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 kk...@apache.org on 2017/12/19 00:08:52 UTC

[01/50] [abbrv] hadoop git commit: HDFS-11902. [READ] Merge BlockFormatProvider and FileRegionProvider. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-6592 0e66d31e2 -> bf2a8ccc0 (forced update)


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 8782e71..40d77f7a 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
@@ -52,11 +52,12 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
-import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -168,49 +169,66 @@ public class TestProvidedImpl {
   }
 
   /**
-   * A simple FileRegion provider for tests.
+   * A simple FileRegion BlockAliasMap for tests.
    */
-  public static class TestFileRegionProvider
-      extends FileRegionProvider implements Configurable {
+  public static class TestFileRegionBlockAliasMap
+      extends BlockAliasMap<FileRegion> {
 
     private Configuration conf;
     private int minId;
     private int numBlocks;
     private Iterator<FileRegion> suppliedIterator;
 
-    TestFileRegionProvider() {
+    TestFileRegionBlockAliasMap() {
       this(null, MIN_BLK_ID, NUM_PROVIDED_BLKS);
     }
 
-    TestFileRegionProvider(Iterator<FileRegion> iterator, int minId,
-        int numBlocks) {
+    TestFileRegionBlockAliasMap(Iterator<FileRegion> iterator, int minId,
+                                int numBlocks) {
       this.suppliedIterator = iterator;
       this.minId = minId;
       this.numBlocks = numBlocks;
     }
 
     @Override
-    public Iterator<FileRegion> iterator() {
-      if (suppliedIterator == null) {
-        return new TestFileRegionIterator(providedBasePath, minId, numBlocks);
-      } else {
-        return suppliedIterator;
-      }
-    }
+    public Reader<FileRegion> getReader(Reader.Options opts)
+        throws IOException {
+
+      BlockAliasMap.Reader<FileRegion> reader =
+          new BlockAliasMap.Reader<FileRegion>() {
+            @Override
+            public Iterator<FileRegion> iterator() {
+              if (suppliedIterator == null) {
+                return new TestFileRegionIterator(providedBasePath, minId,
+                    numBlocks);
+              } else {
+                return suppliedIterator;
+              }
+            }
 
-    @Override
-    public void setConf(Configuration conf) {
-      this.conf = conf;
+            @Override
+            public void close() throws IOException {
+
+            }
+
+            @Override
+            public FileRegion resolve(Block ident) throws IOException {
+              return null;
+            }
+          };
+      return reader;
     }
 
     @Override
-    public Configuration getConf() {
-      return conf;
+    public Writer<FileRegion> getWriter(Writer.Options opts)
+        throws IOException {
+      // not implemented
+      return null;
     }
 
     @Override
-    public void refresh() {
-      //do nothing!
+    public void refresh() throws IOException {
+      // do nothing!
     }
 
     public void setMinBlkId(int minId) {
@@ -359,8 +377,8 @@ public class TestProvidedImpl {
         new ShortCircuitRegistry(conf);
     when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry);
 
-    conf.setClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
-        TestFileRegionProvider.class, FileRegionProvider.class);
+    this.conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TestFileRegionBlockAliasMap.class, BlockAliasMap.class);
     conf.setClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
         TestProvidedVolumeDF.class, ProvidedVolumeDF.class);
 
@@ -496,12 +514,13 @@ public class TestProvidedImpl {
     conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
     for (int i = 0; i < providedVolumes.size(); i++) {
       ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
-      TestFileRegionProvider provider = (TestFileRegionProvider)
-          vol.getFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+      TestFileRegionBlockAliasMap testBlockFormat =
+          (TestFileRegionBlockAliasMap) vol
+              .getBlockFormat(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
       //equivalent to two new blocks appearing
-      provider.setBlockCount(NUM_PROVIDED_BLKS + 2);
+      testBlockFormat.setBlockCount(NUM_PROVIDED_BLKS + 2);
       //equivalent to deleting the first block
-      provider.setMinBlkId(MIN_BLK_ID + 1);
+      testBlockFormat.setMinBlkId(MIN_BLK_ID + 1);
 
       DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf);
       scanner.reconcile();
@@ -525,7 +544,7 @@ public class TestProvidedImpl {
     for (int i = 0; i < providedVolumes.size(); i++) {
       ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
       vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID],
-          new TestFileRegionProvider(fileRegionIterator, minBlockId,
+          new TestFileRegionBlockAliasMap(fileRegionIterator, minBlockId,
               numBlocks));
       ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
       vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 e1e85c1..2e57c9f 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
@@ -29,7 +29,7 @@ import org.apache.commons.cli.PosixParser;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -103,7 +103,7 @@ public class FileSystemImage implements Tool {
         break;
       case "b":
         opts.blocks(
-            Class.forName(o.getValue()).asSubclass(BlockFormat.class));
+            Class.forName(o.getValue()).asSubclass(BlockAliasMap.class));
         break;
       case "i":
         opts.blockIds(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 a3603a1..ea1888a 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
@@ -44,8 +44,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
@@ -88,7 +88,7 @@ public class ImageWriter implements Closeable {
   private final long startBlock;
   private final long startInode;
   private final UGIResolver ugis;
-  private final BlockFormat.Writer<FileRegion> blocks;
+  private final BlockAliasMap.Writer<FileRegion> blocks;
   private final BlockResolver blockIds;
   private final Map<Long, DirEntry.Builder> dircache;
   private final TrackedOutputStream<DigestOutputStream> raw;
@@ -155,8 +155,8 @@ public class ImageWriter implements Closeable {
     ugis = null == opts.ugis
         ? ReflectionUtils.newInstance(opts.ugisClass, opts.getConf())
         : opts.ugis;
-    BlockFormat<FileRegion> fmt = null == opts.blocks
-        ? ReflectionUtils.newInstance(opts.blockFormatClass, opts.getConf())
+    BlockAliasMap<FileRegion> fmt = null == opts.blocks
+        ? ReflectionUtils.newInstance(opts.aliasMap, opts.getConf())
         : opts.blocks;
     blocks = fmt.getWriter(null);
     blockIds = null == opts.blockIds
@@ -509,10 +509,10 @@ public class ImageWriter implements Closeable {
     private long startInode;
     private UGIResolver ugis;
     private Class<? extends UGIResolver> ugisClass;
-    private BlockFormat<FileRegion> blocks;
+    private BlockAliasMap<FileRegion> blocks;
 
     @SuppressWarnings("rawtypes")
-    private Class<? extends BlockFormat> blockFormatClass;
+    private Class<? extends BlockAliasMap> aliasMap;
     private BlockResolver blockIds;
     private Class<? extends BlockResolver> blockIdsClass;
     private FSImageCompression compress =
@@ -524,7 +524,6 @@ public class ImageWriter implements Closeable {
     @Override
     public void setConf(Configuration conf) {
       this.conf = conf;
-      //long lastTxn = conf.getLong(LAST_TXN, 0L);
       String def = new File("hdfs/name").toURI().toString();
       outdir = new Path(conf.get(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, def));
       startBlock = conf.getLong(FixedBlockResolver.START_BLOCK, (1L << 30) + 1);
@@ -532,9 +531,9 @@ public class ImageWriter implements Closeable {
       maxdircache = conf.getInt(CACHE_ENTRY, 100);
       ugisClass = conf.getClass(UGI_CLASS,
           SingleUGIResolver.class, UGIResolver.class);
-      blockFormatClass = conf.getClass(
-          DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-          NullBlockFormat.class, BlockFormat.class);
+      aliasMap = conf.getClass(
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+          NullBlockAliasMap.class, BlockAliasMap.class);
       blockIdsClass = conf.getClass(BLOCK_RESOLVER_CLASS,
           FixedBlockResolver.class, BlockResolver.class);
     }
@@ -584,14 +583,14 @@ public class ImageWriter implements Closeable {
       return this;
     }
 
-    public Options blocks(BlockFormat<FileRegion> blocks) {
+    public Options blocks(BlockAliasMap<FileRegion> blocks) {
       this.blocks = blocks;
       return this;
     }
 
     @SuppressWarnings("rawtypes")
-    public Options blocks(Class<? extends BlockFormat> blocksClass) {
-      this.blockFormatClass = blocksClass;
+    public Options blocks(Class<? extends BlockAliasMap> blocksClass) {
+      this.aliasMap = blocksClass;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
new file mode 100644
index 0000000..4cdf473
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+
+/**
+ * Null sink for region information emitted from FSImage.
+ */
+public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+    return new Reader<FileRegion>() {
+      @Override
+      public Iterator<FileRegion> iterator() {
+        return new Iterator<FileRegion>() {
+          @Override
+          public boolean hasNext() {
+            return false;
+          }
+          @Override
+          public FileRegion next() {
+            throw new NoSuchElementException();
+          }
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+
+      @Override
+      public void close() throws IOException {
+        // do nothing
+      }
+
+      @Override
+      public FileRegion resolve(Block ident) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    return new Writer<FileRegion>() {
+      @Override
+      public void store(FileRegion token) throws IOException {
+        // do nothing
+      }
+
+      @Override
+      public void close() throws IOException {
+        // do nothing
+      }
+    };
+  }
+
+  @Override
+  public void refresh() throws IOException {
+    // do nothing
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockFormat.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockFormat.java
deleted file mode 100644
index aabdf74..0000000
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockFormat.java
+++ /dev/null
@@ -1,87 +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.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
-import org.apache.hadoop.hdfs.server.common.BlockFormat.Reader.Options;
-import org.apache.hadoop.hdfs.server.common.FileRegion;
-
-/**
- * Null sink for region information emitted from FSImage.
- */
-public class NullBlockFormat extends BlockFormat<FileRegion> {
-
-  @Override
-  public Reader<FileRegion> getReader(Options opts) throws IOException {
-    return new Reader<FileRegion>() {
-      @Override
-      public Iterator<FileRegion> iterator() {
-        return new Iterator<FileRegion>() {
-          @Override
-          public boolean hasNext() {
-            return false;
-          }
-          @Override
-          public FileRegion next() {
-            throw new NoSuchElementException();
-          }
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException();
-          }
-        };
-      }
-
-      @Override
-      public void close() throws IOException {
-        // do nothing
-      }
-
-      @Override
-      public FileRegion resolve(Block ident) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-
-  @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
-    return new Writer<FileRegion>() {
-      @Override
-      public void store(FileRegion token) throws IOException {
-        // do nothing
-      }
-
-      @Override
-      public void close() throws IOException {
-        // do nothing
-      }
-    };
-  }
-
-  @Override
-  public void refresh() throws IOException {
-    // do nothing
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 14e6bed..d327363 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,8 +24,8 @@ import com.google.protobuf.ByteString;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
+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;
@@ -70,7 +70,7 @@ public class TreePath {
   }
 
   public INode toINode(UGIResolver ugi, BlockResolver blk,
-      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+                       BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
           throws IOException {
     if (stat.isFile()) {
       return toFile(ugi, blk, out, blockPoolID);
@@ -101,14 +101,14 @@ public class TreePath {
 
   void writeBlock(long blockId, long offset, long length,
       long genStamp, String blockPoolID,
-      BlockFormat.Writer<FileRegion> out) throws IOException {
+      BlockAliasMap.Writer<FileRegion> out) throws IOException {
     FileStatus s = getFileStatus();
     out.store(new FileRegion(blockId, s.getPath(), offset, length,
         blockPoolID, genStamp));
   }
 
   INode toFile(UGIResolver ugi, BlockResolver blk,
-      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+               BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
           throws IOException {
     final FileStatus s = getFileStatus();
     // TODO should this store resolver's user/group?

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 d622b9e..2170baa 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
@@ -44,13 +44,9 @@ 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.blockmanagement.BlockFormatProvider;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
-import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionProvider;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+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_NAMENODE_NAME_DIR_KEY;
@@ -103,18 +99,13 @@ public class TestNameNodeProvidedImplementation {
         DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
 
-    conf.setClass(DFSConfigKeys.DFS_NAMENODE_BLOCK_PROVIDER_CLASS,
-        BlockFormatProvider.class, BlockProvider.class);
-    conf.setClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
-        TextFileRegionProvider.class, FileRegionProvider.class);
-    conf.setClass(DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-        TextFileRegionFormat.class, BlockFormat.class);
-
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_WRITE_PATH,
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TextFileRegionAliasMap.class, BlockAliasMap.class);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH,
         BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
         BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER, ",");
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, ",");
 
     conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
         new File(NAMEPATH.toUri()).toString());
@@ -167,7 +158,7 @@ public class TestNameNodeProvidedImplementation {
     ImageWriter.Options opts = ImageWriter.defaults();
     opts.setConf(conf);
     opts.output(out.toString())
-        .blocks(TextFileRegionFormat.class)
+        .blocks(TextFileRegionAliasMap.class)
         .blockIds(blockIdsClass);
     try (ImageWriter w = new ImageWriter(opts)) {
       for (TreePath e : t) {


---------------------------------------------------------------------
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: HDFS-12903. [READ] Fix closing streams in ImageWriter

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


[10/50] [abbrv] hadoop git commit: HDFS-12093. [READ] Share remoteFS between ProvidedReplica instances.

Posted by kk...@apache.org.
HDFS-12093. [READ] Share remoteFS between ProvidedReplica instances.


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

Branch: refs/heads/YARN-6592
Commit: 2407c9b93aabb021b76c802b19c928fb6cbb0a85
Parents: 663b3c0
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Aug 7 14:31:15 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../datanode/FinalizedProvidedReplica.java      |  6 +++--
 .../hdfs/server/datanode/ProvidedReplica.java   | 25 +++++++++++---------
 .../hdfs/server/datanode/ReplicaBuilder.java    | 11 +++++++--
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 17 +++++++++----
 .../datanode/TestProvidedReplicaImpl.java       |  2 +-
 5 files changed, 40 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2407c9b9/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 722d573..e23d6be 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -31,8 +32,9 @@ public class FinalizedProvidedReplica extends ProvidedReplica {
 
   public FinalizedProvidedReplica(long blockId, URI fileURI,
       long fileOffset, long blockLen, long genStamp,
-      FsVolumeSpi volume, Configuration conf) {
-    super(blockId, fileURI, fileOffset, blockLen, genStamp, volume, conf);
+      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
+    super(blockId, fileURI, fileOffset, blockLen, genStamp, volume, conf,
+        remoteFS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2407c9b9/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 946ab5a..2b3bd13 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
@@ -65,16 +65,23 @@ public abstract class ProvidedReplica extends ReplicaInfo {
    * @param volume the volume this block belongs to
    */
   public ProvidedReplica(long blockId, URI fileURI, long fileOffset,
-      long blockLen, long genStamp, FsVolumeSpi volume, Configuration conf) {
+      long blockLen, long genStamp, FsVolumeSpi volume, Configuration conf,
+      FileSystem remoteFS) {
     super(volume, blockId, blockLen, genStamp);
     this.fileURI = fileURI;
     this.fileOffset = fileOffset;
     this.conf = conf;
-    try {
-      this.remoteFS = FileSystem.get(fileURI, this.conf);
-    } catch (IOException e) {
-      LOG.warn("Failed to obtain filesystem for " + fileURI);
-      this.remoteFS = null;
+    if (remoteFS != null) {
+      this.remoteFS = remoteFS;
+    } else {
+      LOG.warn(
+          "Creating an reference to the remote FS for provided block " + this);
+      try {
+        this.remoteFS = FileSystem.get(fileURI, this.conf);
+      } catch (IOException e) {
+        LOG.warn("Failed to obtain filesystem for " + fileURI);
+        this.remoteFS = null;
+      }
     }
   }
 
@@ -83,11 +90,7 @@ public abstract class ProvidedReplica extends ReplicaInfo {
     this.fileURI = r.fileURI;
     this.fileOffset = r.fileOffset;
     this.conf = r.conf;
-    try {
-      this.remoteFS = FileSystem.newInstance(fileURI, this.conf);
-    } catch (IOException e) {
-      this.remoteFS = null;
-    }
+    this.remoteFS = r.remoteFS;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2407c9b9/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 639467f..c5cb6a5 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.File;
 import java.net.URI;
 
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
@@ -50,6 +51,7 @@ public class ReplicaBuilder {
   private long offset;
   private Configuration conf;
   private FileRegion fileRegion;
+  private FileSystem remoteFS;
 
   public ReplicaBuilder(ReplicaState state) {
     volume = null;
@@ -138,6 +140,11 @@ public class ReplicaBuilder {
     return this;
   }
 
+  public ReplicaBuilder setRemoteFS(FileSystem remoteFS) {
+    this.remoteFS = remoteFS;
+    return this;
+  }
+
   public LocalReplicaInPipeline buildLocalReplicaInPipeline()
       throws IllegalArgumentException {
     LocalReplicaInPipeline info = null;
@@ -275,14 +282,14 @@ public class ReplicaBuilder {
     }
     if (fileRegion == null) {
       info = new FinalizedProvidedReplica(blockId, uri, offset,
-          length, genStamp, volume, conf);
+          length, genStamp, volume, conf, remoteFS);
     } else {
       info = new FinalizedProvidedReplica(fileRegion.getBlock().getBlockId(),
           fileRegion.getPath().toUri(),
           fileRegion.getOffset(),
           fileRegion.getBlock().getNumBytes(),
           fileRegion.getBlock().getGenerationStamp(),
-          volume, conf);
+          volume, conf, remoteFS);
     }
     return info;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2407c9b9/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 5cd28c7..d1a7015 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
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -96,7 +97,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
     }
 
     public void getVolumeMap(ReplicaMap volumeMap,
-        RamDiskReplicaTracker ramDiskReplicaMap) throws IOException {
+        RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
+        throws IOException {
       Iterator<FileRegion> iter = provider.iterator();
       while (iter.hasNext()) {
         FileRegion region = iter.next();
@@ -112,9 +114,10 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
               .setGenerationStamp(region.getBlock().getGenerationStamp())
               .setFsVolume(providedVolume)
               .setConf(conf)
+              .setRemoteFS(remoteFS)
               .build();
-          // check if the replica already exists
-          ReplicaInfo oldReplica = volumeMap.get(bpid, newReplica.getBlockId());
+          ReplicaInfo oldReplica =
+              volumeMap.get(bpid, newReplica.getBlockId());
           if (oldReplica == null) {
             volumeMap.add(bpid, newReplica);
             bpVolumeMap.add(bpid, newReplica);
@@ -163,6 +166,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
       new ConcurrentHashMap<String, ProvidedBlockPoolSlice>();
 
   private ProvidedVolumeDF df;
+  //the remote FileSystem to which this ProvidedVolume points to.
+  private FileSystem remoteFS;
 
   ProvidedVolumeImpl(FsDatasetImpl dataset, String storageID,
       StorageDirectory sd, FileIoProvider fileIoProvider,
@@ -176,6 +181,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
         conf.getClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
             DefaultProvidedVolumeDF.class, ProvidedVolumeDF.class);
     df = ReflectionUtils.newInstance(dfClass, conf);
+    remoteFS = FileSystem.get(baseURI, conf);
   }
 
   @Override
@@ -397,7 +403,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
           throws IOException {
     LOG.info("Creating volumemap for provided volume " + this);
     for(ProvidedBlockPoolSlice s : bpSlices.values()) {
-      s.getVolumeMap(volumeMap, ramDiskReplicaMap);
+      s.getVolumeMap(volumeMap, ramDiskReplicaMap, remoteFS);
     }
   }
 
@@ -414,7 +420,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   void getVolumeMap(String bpid, ReplicaMap volumeMap,
       final RamDiskReplicaTracker ramDiskReplicaMap)
           throws IOException {
-    getProvidedBlockPoolSlice(bpid).getVolumeMap(volumeMap, ramDiskReplicaMap);
+    getProvidedBlockPoolSlice(bpid).getVolumeMap(volumeMap, ramDiskReplicaMap,
+        remoteFS);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2407c9b9/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 8258c21..967e94d 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));
+          currentReplicaLength, 0, null, conf, null));
     }
   }
 


---------------------------------------------------------------------
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: HADOOP-15106. FileSystem::open(PathHandle) should throw a specific exception on validation failure

Posted by kk...@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-6592
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


[22/50] [abbrv] hadoop git commit: HDFS-12809. [READ] Fix the randomized selection of locations in {{ProvidedBlocksBuilder}}.

Posted by kk...@apache.org.
HDFS-12809. [READ] Fix the randomized selection of locations in {{ProvidedBlocksBuilder}}.


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

Branch: refs/heads/YARN-6592
Commit: 4d59dabb7f6ef1d8565bf2bb2d38aeb91bf7f7cc
Parents: 3d3be87
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Nov 27 17:04:20 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../blockmanagement/ProvidedStorageMap.java     | 112 +++++++------------
 .../TestNameNodeProvidedImplementation.java     |  26 ++++-
 2 files changed, 61 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d59dabb/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 6fec977..c85eb2c 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
@@ -19,11 +19,12 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentSkipListMap;
@@ -229,11 +230,8 @@ public class ProvidedStorageMap {
         sids.add(currInfo.getStorageID());
         types.add(storageType);
         if (StorageType.PROVIDED.equals(storageType)) {
-          DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
-          locs.add(
-              new DatanodeInfoWithStorage(
-                  dn, currInfo.getStorageID(), currInfo.getStorageType()));
-          excludedUUids.add(dn.getDatanodeUuid());
+          // Provided location will be added to the list of locations after
+          // examining all local locations.
           isProvidedBlock = true;
         } else {
           locs.add(new DatanodeInfoWithStorage(
@@ -245,11 +243,17 @@ public class ProvidedStorageMap {
 
       int numLocations = locs.size();
       if (isProvidedBlock) {
+        // add the first datanode here
+        DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
+        locs.add(
+            new DatanodeInfoWithStorage(dn, storageId, StorageType.PROVIDED));
+        excludedUUids.add(dn.getDatanodeUuid());
+        numLocations++;
         // add more replicas until we reach the defaultReplication
         for (int count = numLocations + 1;
             count <= defaultReplication && count <= providedDescriptor
                 .activeProvidedDatanodes(); count++) {
-          DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
+          dn = chooseProvidedDatanode(excludedUUids);
           locs.add(new DatanodeInfoWithStorage(
               dn, storageId, StorageType.PROVIDED));
           sids.add(storageId);
@@ -284,6 +288,9 @@ public class ProvidedStorageMap {
 
     private final NavigableMap<String, DatanodeDescriptor> dns =
         new ConcurrentSkipListMap<>();
+    // maintain a separate list of the datanodes with provided storage
+    // to efficiently choose Datanodes when required.
+    private final List<DatanodeDescriptor> dnR = new ArrayList<>();
     public final static String NETWORK_LOCATION = "/REMOTE";
     public final static String NAME = "PROVIDED";
 
@@ -300,8 +307,8 @@ public class ProvidedStorageMap {
 
     DatanodeStorageInfo getProvidedStorage(
         DatanodeDescriptor dn, DatanodeStorage s) {
-      LOG.info("XXXXX adding Datanode " + dn.getDatanodeUuid());
       dns.put(dn.getDatanodeUuid(), dn);
+      dnR.add(dn);
       // TODO: maintain separate RPC ident per dn
       return storageMap.get(s.getStorageID());
     }
@@ -315,84 +322,42 @@ public class ProvidedStorageMap {
     }
 
     DatanodeDescriptor choose(DatanodeDescriptor client) {
-      // exact match for now
-      DatanodeDescriptor dn = client != null ?
-          dns.get(client.getDatanodeUuid()) : null;
-      if (null == dn) {
-        dn = chooseRandom();
-      }
-      return dn;
+      return choose(client, Collections.<String>emptySet());
     }
 
     DatanodeDescriptor choose(DatanodeDescriptor client,
         Set<String> excludedUUids) {
       // exact match for now
-      DatanodeDescriptor dn = client != null ?
-          dns.get(client.getDatanodeUuid()) : null;
-
-      if (null == dn || excludedUUids.contains(client.getDatanodeUuid())) {
-        dn = null;
-        Set<String> exploredUUids = new HashSet<String>();
-
-        while(exploredUUids.size() < dns.size()) {
-          Map.Entry<String, DatanodeDescriptor> d =
-                  dns.ceilingEntry(UUID.randomUUID().toString());
-          if (null == d) {
-            d = dns.firstEntry();
-          }
-          String uuid = d.getValue().getDatanodeUuid();
-          //this node has already been explored, and was not selected earlier
-          if (exploredUUids.contains(uuid)) {
-            continue;
-          }
-          exploredUUids.add(uuid);
-          //this node has been excluded
-          if (excludedUUids.contains(uuid)) {
-            continue;
-          }
-          return dns.get(uuid);
-        }
-      }
-
-      return dn;
-    }
-
-    DatanodeDescriptor chooseRandom(DatanodeStorageInfo[] excludedStorages) {
-      // TODO: Currently this is not uniformly random;
-      // skewed toward sparse sections of the ids
-      Set<DatanodeDescriptor> excludedNodes =
-          new HashSet<DatanodeDescriptor>();
-      if (excludedStorages != null) {
-        for (int i= 0; i < excludedStorages.length; i++) {
-          LOG.info("Excluded: " + excludedStorages[i].getDatanodeDescriptor());
-          excludedNodes.add(excludedStorages[i].getDatanodeDescriptor());
+      if (client != null && !excludedUUids.contains(client.getDatanodeUuid())) {
+        DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
+        if (dn != null) {
+          return dn;
         }
       }
-      Set<DatanodeDescriptor> exploredNodes = new HashSet<DatanodeDescriptor>();
 
-      while(exploredNodes.size() < dns.size()) {
-        Map.Entry<String, DatanodeDescriptor> d =
-            dns.ceilingEntry(UUID.randomUUID().toString());
-        if (null == d) {
-          d = dns.firstEntry();
-        }
-        DatanodeDescriptor node = d.getValue();
-        //this node has already been explored, and was not selected earlier
-        if (exploredNodes.contains(node)) {
-          continue;
+      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;
         }
-        exploredNodes.add(node);
-        //this node has been excluded
-        if (excludedNodes.contains(node)) {
-          continue;
-        }
-        return node;
+        Collections.swap(dnR, i, pos);
       }
       return null;
     }
 
-    DatanodeDescriptor chooseRandom() {
-      return chooseRandom(null);
+    DatanodeDescriptor chooseRandom(DatanodeStorageInfo... excludedStorages) {
+      Set<String> excludedNodes = new HashSet<>();
+      if (excludedStorages != null) {
+        for (int i = 0; i < excludedStorages.length; i++) {
+          DatanodeDescriptor dn = excludedStorages[i].getDatanodeDescriptor();
+          String uuid = dn.getDatanodeUuid();
+          excludedNodes.add(uuid);
+        }
+      }
+      return choose(null, excludedNodes);
     }
 
     @Override
@@ -414,6 +379,7 @@ public class ProvidedStorageMap {
         DatanodeDescriptor storedDN = dns.get(dnToRemove.getDatanodeUuid());
         if (storedDN != null) {
           dns.remove(dnToRemove.getDatanodeUuid());
+          dnR.remove(dnToRemove);
         }
       }
       return dns.size();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d59dabb/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 9c82967..09e8f97 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
@@ -27,8 +27,11 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Random;
+import java.util.Set;
+
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -480,16 +483,31 @@ public class TestNameNodeProvidedImplementation {
     // given the start and length in the above call,
     // only one LocatedBlock in LocatedBlocks
     assertEquals(expectedBlocks, locatedBlocks.getLocatedBlocks().size());
-    LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
-    assertEquals(expectedLocations, locatedBlock.getLocations().length);
-    return locatedBlock.getLocations();
+    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=30000)
+  @Test(timeout=50000)
   public void testSetReplicationForProvidedFiles() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);


---------------------------------------------------------------------
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-12778. [READ] Report multiple locations for PROVIDED blocks

Posted by kk...@apache.org.
HDFS-12778. [READ] Report multiple locations for PROVIDED blocks


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

Branch: refs/heads/YARN-6592
Commit: 3d3be87e301d9f8ab1a220bc5dbeae0f032c5a86
Parents: 3b1d303
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Nov 21 14:54:57 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../blockmanagement/ProvidedStorageMap.java     | 149 +++++++------------
 .../server/namenode/FixedBlockResolver.java     |   3 +-
 .../TestNameNodeProvidedImplementation.java     | 127 +++++++++++-----
 3 files changed, 151 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d3be87e/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 2bc8faa..6fec977 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
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -72,6 +71,7 @@ public class ProvidedStorageMap {
   private final DatanodeStorageInfo providedStorageInfo;
   private boolean providedEnabled;
   private long capacity;
+  private int defaultReplication;
 
   ProvidedStorageMap(RwLock lock, BlockManager bm, Configuration conf)
       throws IOException {
@@ -95,6 +95,8 @@ public class ProvidedStorageMap {
         storageId, State.NORMAL, StorageType.PROVIDED);
     providedDescriptor = new ProvidedDescriptor();
     providedStorageInfo = providedDescriptor.createProvidedStorage(ds);
+    this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
 
     this.bm = bm;
     this.lock = lock;
@@ -198,63 +200,72 @@ public class ProvidedStorageMap {
    */
   class ProvidedBlocksBuilder extends LocatedBlockBuilder {
 
-    private ShadowDatanodeInfoWithStorage pending;
-    private boolean hasProvidedLocations;
-
     ProvidedBlocksBuilder(int maxBlocks) {
       super(maxBlocks);
-      pending = new ShadowDatanodeInfoWithStorage(
-          providedDescriptor, storageId);
-      hasProvidedLocations = false;
+    }
+
+    private DatanodeDescriptor chooseProvidedDatanode(
+        Set<String> excludedUUids) {
+      DatanodeDescriptor dn = providedDescriptor.choose(null, excludedUUids);
+      if (dn == null) {
+        dn = providedDescriptor.choose(null);
+      }
+      return dn;
     }
 
     @Override
     LocatedBlock newLocatedBlock(ExtendedBlock eb,
         DatanodeStorageInfo[] storages, long pos, boolean isCorrupt) {
 
-      DatanodeInfoWithStorage[] locs =
-        new DatanodeInfoWithStorage[storages.length];
-      String[] sids = new String[storages.length];
-      StorageType[] types = new StorageType[storages.length];
+      List<DatanodeInfoWithStorage> locs = new ArrayList<>();
+      List<String> sids = new ArrayList<>();
+      List<StorageType> types = new ArrayList<>();
+      boolean isProvidedBlock = false;
+      Set<String> excludedUUids = new HashSet<>();
+
       for (int i = 0; i < storages.length; ++i) {
-        sids[i] = storages[i].getStorageID();
-        types[i] = storages[i].getStorageType();
-        if (StorageType.PROVIDED.equals(storages[i].getStorageType())) {
-          locs[i] = pending;
-          hasProvidedLocations = true;
+        DatanodeStorageInfo currInfo = storages[i];
+        StorageType storageType = currInfo.getStorageType();
+        sids.add(currInfo.getStorageID());
+        types.add(storageType);
+        if (StorageType.PROVIDED.equals(storageType)) {
+          DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
+          locs.add(
+              new DatanodeInfoWithStorage(
+                  dn, currInfo.getStorageID(), currInfo.getStorageType()));
+          excludedUUids.add(dn.getDatanodeUuid());
+          isProvidedBlock = true;
         } else {
-          locs[i] = new DatanodeInfoWithStorage(
-              storages[i].getDatanodeDescriptor(), sids[i], types[i]);
+          locs.add(new DatanodeInfoWithStorage(
+              currInfo.getDatanodeDescriptor(),
+              currInfo.getStorageID(), storageType));
+          excludedUUids.add(currInfo.getDatanodeDescriptor().getDatanodeUuid());
         }
       }
-      return new LocatedBlock(eb, locs, sids, types, pos, isCorrupt, null);
-    }
 
-    @Override
-    LocatedBlocks build(DatanodeDescriptor client) {
-      // TODO: to support multiple provided storages, need to pass/maintain map
-      if (hasProvidedLocations) {
-        // set all fields of pending DatanodeInfo
-        List<String> excludedUUids = new ArrayList<String>();
-        for (LocatedBlock b : blocks) {
-          DatanodeInfo[] infos = b.getLocations();
-          StorageType[] types = b.getStorageTypes();
-
-          for (int i = 0; i < types.length; i++) {
-            if (!StorageType.PROVIDED.equals(types[i])) {
-              excludedUUids.add(infos[i].getDatanodeUuid());
-            }
-          }
+      int numLocations = locs.size();
+      if (isProvidedBlock) {
+        // add more replicas until we reach the defaultReplication
+        for (int count = numLocations + 1;
+            count <= defaultReplication && count <= providedDescriptor
+                .activeProvidedDatanodes(); count++) {
+          DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
+          locs.add(new DatanodeInfoWithStorage(
+              dn, storageId, StorageType.PROVIDED));
+          sids.add(storageId);
+          types.add(StorageType.PROVIDED);
+          excludedUUids.add(dn.getDatanodeUuid());
         }
-
-        DatanodeDescriptor dn =
-                providedDescriptor.choose(client, excludedUUids);
-        if (dn == null) {
-          dn = providedDescriptor.choose(client);
-        }
-        pending.replaceInternal(dn);
       }
+      return new LocatedBlock(eb,
+          locs.toArray(new DatanodeInfoWithStorage[locs.size()]),
+          sids.toArray(new String[sids.size()]),
+          types.toArray(new StorageType[types.size()]),
+          pos, isCorrupt, null);
+    }
 
+    @Override
+    LocatedBlocks build(DatanodeDescriptor client) {
       return new LocatedBlocks(
           flen, isUC, blocks, last, lastComplete, feInfo, ecPolicy);
     }
@@ -266,53 +277,6 @@ public class ProvidedStorageMap {
   }
 
   /**
-   * An abstract {@link DatanodeInfoWithStorage} to represent provided storage.
-   */
-  static class ShadowDatanodeInfoWithStorage extends DatanodeInfoWithStorage {
-    private String shadowUuid;
-
-    ShadowDatanodeInfoWithStorage(DatanodeDescriptor d, String storageId) {
-      super(d, storageId, StorageType.PROVIDED);
-    }
-
-    @Override
-    public String getDatanodeUuid() {
-      return shadowUuid;
-    }
-
-    public void setDatanodeUuid(String uuid) {
-      shadowUuid = uuid;
-    }
-
-    void replaceInternal(DatanodeDescriptor dn) {
-      updateRegInfo(dn); // overwrite DatanodeID (except UUID)
-      setDatanodeUuid(dn.getDatanodeUuid());
-      setCapacity(dn.getCapacity());
-      setDfsUsed(dn.getDfsUsed());
-      setRemaining(dn.getRemaining());
-      setBlockPoolUsed(dn.getBlockPoolUsed());
-      setCacheCapacity(dn.getCacheCapacity());
-      setCacheUsed(dn.getCacheUsed());
-      setLastUpdate(dn.getLastUpdate());
-      setLastUpdateMonotonic(dn.getLastUpdateMonotonic());
-      setXceiverCount(dn.getXceiverCount());
-      setNetworkLocation(dn.getNetworkLocation());
-      adminState = dn.getAdminState();
-      setUpgradeDomain(dn.getUpgradeDomain());
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return super.equals(obj);
-    }
-
-    @Override
-    public int hashCode() {
-      return super.hashCode();
-    }
-  }
-
-  /**
    * An abstract DatanodeDescriptor to track datanodes with provided storages.
    * NOTE: never resolved through registerDatanode, so not in the topology.
    */
@@ -336,6 +300,7 @@ public class ProvidedStorageMap {
 
     DatanodeStorageInfo getProvidedStorage(
         DatanodeDescriptor dn, DatanodeStorage s) {
+      LOG.info("XXXXX adding Datanode " + dn.getDatanodeUuid());
       dns.put(dn.getDatanodeUuid(), dn);
       // TODO: maintain separate RPC ident per dn
       return storageMap.get(s.getStorageID());
@@ -352,7 +317,7 @@ public class ProvidedStorageMap {
     DatanodeDescriptor choose(DatanodeDescriptor client) {
       // exact match for now
       DatanodeDescriptor dn = client != null ?
-              dns.get(client.getDatanodeUuid()) : null;
+          dns.get(client.getDatanodeUuid()) : null;
       if (null == dn) {
         dn = chooseRandom();
       }
@@ -360,10 +325,10 @@ public class ProvidedStorageMap {
     }
 
     DatanodeDescriptor choose(DatanodeDescriptor client,
-        List<String> excludedUUids) {
+        Set<String> excludedUUids) {
       // exact match for now
       DatanodeDescriptor dn = client != null ?
-              dns.get(client.getDatanodeUuid()) : null;
+          dns.get(client.getDatanodeUuid()) : null;
 
       if (null == dn || excludedUUids.contains(client.getDatanodeUuid())) {
         dn = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d3be87e/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
index 8ff9695..4b3a01f 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
@@ -34,6 +34,7 @@ public class FixedBlockResolver extends BlockResolver implements Configurable {
       "hdfs.image.writer.resolver.fixed.block.size";
   public static final String START_BLOCK =
       "hdfs.image.writer.resolver.fixed.block.start";
+  public static final long BLOCKSIZE_DEFAULT = 256 * (1L << 20);
 
   private Configuration conf;
   private long blocksize = 256 * (1L << 20);
@@ -42,7 +43,7 @@ public class FixedBlockResolver extends BlockResolver implements Configurable {
   @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
-    blocksize = conf.getLong(BLOCKSIZE, 256 * (1L << 20));
+    blocksize = conf.getLong(BLOCKSIZE, BLOCKSIZE_DEFAULT);
     blockIds.set(conf.getLong(START_BLOCK, (1L << 30)));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d3be87e/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 f6d38f6..9c82967 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
@@ -474,12 +474,12 @@ public class TestNameNodeProvidedImplementation {
   }
 
   private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
-      String filename, int expectedLocations) throws IOException {
-    LocatedBlocks locatedBlocks = client.getLocatedBlocks(
-        filename, 0, baseFileLen);
-    //given the start and length in the above call,
-    //only one LocatedBlock in LocatedBlocks
-    assertEquals(1, locatedBlocks.getLocatedBlocks().size());
+      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());
     LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
     assertEquals(expectedLocations, locatedBlock.getLocations().length);
     return locatedBlock.getLocations();
@@ -513,17 +513,20 @@ public class TestNameNodeProvidedImplementation {
         file, newReplication, 10000);
     DFSClient client = new DFSClient(new InetSocketAddress("localhost",
         cluster.getNameNodePort()), cluster.getConfiguration(0));
-    getAndCheckBlockLocations(client, filename, newReplication);
+    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, newReplication, 10000);
-    // the only replica left should be the PROVIDED datanode
-    getAndCheckBlockLocations(client, filename, newReplication);
+        file, (short) defaultReplication, 10000);
+    getAndCheckBlockLocations(client, filename, baseFileLen, 1,
+        defaultReplication);
   }
 
   @Test(timeout=30000)
@@ -545,8 +548,9 @@ public class TestNameNodeProvidedImplementation {
 
     if (numFiles >= 1) {
       String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
-
-      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      // 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(
@@ -564,7 +568,7 @@ public class TestNameNodeProvidedImplementation {
           providedDatanode1.getDatanodeId().getXferAddr());
 
       //should find the block on the 2nd provided datanode
-      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
       assertEquals(providedDatanode2.getDatanodeUuid(),
           dnInfos[0].getDatanodeUuid());
 
@@ -575,14 +579,14 @@ public class TestNameNodeProvidedImplementation {
       BlockManagerTestUtil.noticeDeadDatanode(
           cluster.getNameNode(),
           providedDatanode2.getDatanodeId().getXferAddr());
-      getAndCheckBlockLocations(client, filename, 0);
+      getAndCheckBlockLocations(client, filename, baseFileLen, 1, 0);
 
       //restart the provided datanode
       cluster.restartDataNode(providedDNProperties1, true);
       cluster.waitActive();
 
       //should find the block on the 1st provided datanode now
-      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      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());
@@ -593,20 +597,18 @@ public class TestNameNodeProvidedImplementation {
   public void testTransientDeadDatanodes() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
             FixedBlockResolver.class);
-    // 2 Datanodes, 1 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 2, null,
+    // 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);
-
-    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-        cluster.getNameNodePort()), cluster.getConfiguration(0));
-
     for (int i= 0; i < numFiles; i++) {
-      verifyFileLocation(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(),
@@ -614,7 +616,7 @@ public class TestNameNodeProvidedImplementation {
       cluster.waitActive();
       cluster.triggerHeartbeats();
       Thread.sleep(1000);
-      verifyFileLocation(i);
+      verifyFileLocation(i, 2);
     }
   }
 
@@ -622,17 +624,18 @@ public class TestNameNodeProvidedImplementation {
   public void testNamenodeRestart() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    // 2 Datanodes, 1 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 2, null,
+    // 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);
+    verifyFileLocation(numFiles - 1, 2);
     cluster.restartNameNodes();
     cluster.waitActive();
-    verifyFileLocation(numFiles - 1);
+    verifyFileLocation(numFiles - 1, 2);
   }
 
   /**
@@ -640,18 +643,21 @@ public class TestNameNodeProvidedImplementation {
    * @param fileIndex the index of the file to verify.
    * @throws Exception
    */
-  private void verifyFileLocation(int fileIndex)
+  private void verifyFileLocation(int fileIndex, int replication)
       throws Exception {
-    DataNode providedDatanode = cluster.getDataNodes().get(0);
     DFSClient client = new DFSClient(
         new InetSocketAddress("localhost", cluster.getNameNodePort()),
         cluster.getConfiguration(0));
-    if (fileIndex <= numFiles && fileIndex >= 0) {
-      String filename = "/" + filePrefix + fileIndex + fileSuffix;
-      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
-      // location should be the provided DN
-      assertEquals(providedDatanode.getDatanodeUuid(),
-          dnInfos[0].getDatanodeUuid());
+    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);
     }
   }
 
@@ -669,4 +675,55 @@ public class TestNameNodeProvidedImplementation {
     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);
+    }
+  }
 }


---------------------------------------------------------------------
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: 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 kk...@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-6592
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


[12/50] [abbrv] hadoop git commit: HDFS-12775. [READ] Fix reporting of Provided volumes

Posted by kk...@apache.org.
HDFS-12775. [READ] Fix reporting of Provided volumes


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

Branch: refs/heads/YARN-6592
Commit: 3b1d30301bcd35bbe525a7e122d3e5acfab92c88
Parents: e1a28f9
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 16 03:52:12 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   1 -
 .../server/blockmanagement/BlockManager.java    |  19 ++-
 .../blockmanagement/DatanodeDescriptor.java     |  24 ++--
 .../blockmanagement/DatanodeStatistics.java     |   3 +
 .../server/blockmanagement/DatanodeStats.java   |   4 +-
 .../blockmanagement/HeartbeatManager.java       |   9 +-
 .../blockmanagement/ProvidedStorageMap.java     |  60 +++++++--
 .../blockmanagement/StorageTypeStats.java       |  33 ++++-
 .../fsdataset/impl/DefaultProvidedVolumeDF.java |  58 ---------
 .../fsdataset/impl/ProvidedVolumeDF.java        |  34 -----
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 101 ++++++++++++---
 .../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 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  12 ++
 .../hdfs/server/namenode/NameNodeMXBean.java    |  10 +-
 .../namenode/metrics/FSNamesystemMBean.java     |   7 +-
 .../src/main/proto/FederationProtocol.proto     |   1 +
 .../src/main/resources/hdfs-default.xml         |   8 --
 .../src/main/webapps/hdfs/dfshealth.html        |   1 +
 .../blockmanagement/TestProvidedStorageMap.java |  39 +++---
 .../fsdataset/impl/TestProvidedImpl.java        |  55 ++------
 .../metrics/TestFederationMetrics.java          |   2 +
 .../TestNameNodeProvidedImplementation.java     | 125 ++++++++++++++++---
 29 files changed, 425 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index cb57675..fbdc859 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -331,7 +331,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
   public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;
 
-  public static final String DFS_PROVIDER_DF_CLASS = "dfs.provided.df.class";
   public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";
   public static final String DFS_PROVIDER_STORAGEUUID_DEFAULT =  "DS-PROVIDED";
   public static final String DFS_PROVIDED_ALIASMAP_CLASS = "dfs.provided.aliasmap.class";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 07502c1..f92c4e8 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
@@ -103,6 +103,8 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.hdfs.util.FoldedTreeSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.namenode.CacheManager;
@@ -2402,6 +2404,21 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  public long getProvidedCapacity() {
+    return providedStorageMap.getCapacity();
+  }
+
+  public void updateHeartbeat(DatanodeDescriptor node, StorageReport[] reports,
+      long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes,
+      VolumeFailureSummary volumeFailureSummary) {
+
+    for (StorageReport report: reports) {
+      providedStorageMap.updateStorage(node, report.getStorage());
+    }
+    node.updateHeartbeat(reports, cacheCapacity, cacheUsed, xceiverCount,
+        failedVolumes, volumeFailureSummary);
+  }
+
   /**
    * StatefulBlockInfo is used to build the "toUC" list, which is a list of
    * updates to the information about under-construction blocks.
@@ -2463,7 +2480,7 @@ public class BlockManager implements BlockStatsMXBean {
       // !#! 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, context);
+          providedStorageMap.getStorage(node, storage);
 
       if (storageInfo == null) {
         // We handle this for backwards compatibility.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index c17ab4c..83c608f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -449,24 +449,24 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.volumeFailures = volFailures;
     this.volumeFailureSummary = volumeFailureSummary;
     for (StorageReport report : reports) {
-      totalCapacity += report.getCapacity();
-      totalRemaining += report.getRemaining();
-      totalBlockPoolUsed += report.getBlockPoolUsed();
-      totalDfsUsed += report.getDfsUsed();
-      totalNonDfsUsed += report.getNonDfsUsed();
 
-      // for PROVIDED storages, do not call updateStorage() unless
-      // DatanodeStorageInfo already exists!
-      if (StorageType.PROVIDED.equals(report.getStorage().getStorageType())
-          && storageMap.get(report.getStorage().getStorageID()) == null) {
-        continue;
-      }
-      DatanodeStorageInfo storage = updateStorage(report.getStorage());
+      DatanodeStorageInfo storage =
+          storageMap.get(report.getStorage().getStorageID());
       if (checkFailedStorages) {
         failedStorageInfos.remove(storage);
       }
 
       storage.receivedHeartbeat(report);
+      // skip accounting for capacity of PROVIDED storages!
+      if (StorageType.PROVIDED.equals(storage.getStorageType())) {
+        continue;
+      }
+
+      totalCapacity += report.getCapacity();
+      totalRemaining += report.getRemaining();
+      totalBlockPoolUsed += report.getBlockPoolUsed();
+      totalDfsUsed += report.getDfsUsed();
+      totalNonDfsUsed += report.getNonDfsUsed();
     }
 
     // Update total metrics for the node.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
index 33eca2e..36a9c2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
@@ -77,4 +77,7 @@ public interface DatanodeStatistics {
 
   /** @return Storage Tier statistics*/
   Map<StorageType, StorageTypeStats> getStorageTypeStats();
+
+  /** @return the provided capacity */
+  public long getProvidedCapacity();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
index 8386b27..912d4d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
@@ -183,7 +183,7 @@ class DatanodeStats {
       StorageTypeStats storageTypeStats =
           storageTypeStatsMap.get(storageType);
       if (storageTypeStats == null) {
-        storageTypeStats = new StorageTypeStats();
+        storageTypeStats = new StorageTypeStats(storageType);
         storageTypeStatsMap.put(storageType, storageTypeStats);
       }
       storageTypeStats.addNode(node);
@@ -194,7 +194,7 @@ class DatanodeStats {
       StorageTypeStats storageTypeStats =
           storageTypeStatsMap.get(info.getStorageType());
       if (storageTypeStats == null) {
-        storageTypeStats = new StorageTypeStats();
+        storageTypeStats = new StorageTypeStats(info.getStorageType());
         storageTypeStatsMap.put(info.getStorageType(), storageTypeStats);
       }
       storageTypeStats.addStorage(info, node);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
index a72ad64..1972a6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
@@ -195,6 +195,11 @@ class HeartbeatManager implements DatanodeStatistics {
     return stats.getStatsMap();
   }
 
+  @Override
+  public long getProvidedCapacity() {
+    return blockManager.getProvidedCapacity();
+  }
+
   synchronized void register(final DatanodeDescriptor d) {
     if (!d.isAlive()) {
       addDatanode(d);
@@ -232,8 +237,8 @@ class HeartbeatManager implements DatanodeStatistics {
       int xceiverCount, int failedVolumes,
       VolumeFailureSummary volumeFailureSummary) {
     stats.subtract(node);
-    node.updateHeartbeat(reports, cacheCapacity, cacheUsed,
-      xceiverCount, failedVolumes, volumeFailureSummary);
+    blockManager.updateHeartbeat(node, reports, cacheCapacity, cacheUsed,
+        xceiverCount, failedVolumes, volumeFailureSummary);
     stats.add(node);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 3d19775..2bc8faa 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
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
-import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.common.BlockAlias;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@ -72,6 +71,7 @@ public class ProvidedStorageMap {
   private final ProvidedDescriptor providedDescriptor;
   private final DatanodeStorageInfo providedStorageInfo;
   private boolean providedEnabled;
+  private long capacity;
 
   ProvidedStorageMap(RwLock lock, BlockManager bm, Configuration conf)
       throws IOException {
@@ -112,14 +112,13 @@ public class ProvidedStorageMap {
   /**
    * @param dn datanode descriptor
    * @param s data node storage
-   * @param context the block report context
    * @return the {@link DatanodeStorageInfo} for the specified datanode.
    * If {@code s} corresponds to a provided storage, the storage info
    * representing provided storage is returned.
    * @throws IOException
    */
-  DatanodeStorageInfo getStorage(DatanodeDescriptor dn, DatanodeStorage s,
-      BlockReportContext context) throws IOException {
+  DatanodeStorageInfo getStorage(DatanodeDescriptor dn, DatanodeStorage s)
+      throws IOException {
     if (providedEnabled && storageId.equals(s.getStorageID())) {
       if (StorageType.PROVIDED.equals(s.getStorageType())) {
         if (providedStorageInfo.getState() == State.FAILED
@@ -127,8 +126,10 @@ public class ProvidedStorageMap {
           providedStorageInfo.setState(State.NORMAL);
           LOG.info("Provided storage transitioning to state " + State.NORMAL);
         }
-        processProvidedStorageReport(context);
-        dn.injectStorage(providedStorageInfo);
+        if (dn.getStorageInfo(s.getStorageID()) == null) {
+          dn.injectStorage(providedStorageInfo);
+        }
+        processProvidedStorageReport();
         return providedDescriptor.getProvidedStorage(dn, s);
       }
       LOG.warn("Reserved storage {} reported as non-provided from {}", s, dn);
@@ -136,7 +137,7 @@ public class ProvidedStorageMap {
     return dn.getStorageInfo(s.getStorageID());
   }
 
-  private void processProvidedStorageReport(BlockReportContext context)
+  private void processProvidedStorageReport()
       throws IOException {
     assert lock.hasWriteLock() : "Not holding write lock";
     if (providedStorageInfo.getBlockReportCount() == 0
@@ -172,6 +173,26 @@ public class ProvidedStorageMap {
     }
   }
 
+  public long getCapacity() {
+    if (providedStorageInfo == null) {
+      return 0;
+    }
+    return providedStorageInfo.getCapacity();
+  }
+
+  public void updateStorage(DatanodeDescriptor node, DatanodeStorage storage) {
+    if (providedEnabled && storageId.equals(storage.getStorageID())) {
+      if (StorageType.PROVIDED.equals(storage.getStorageType())) {
+        node.injectStorage(providedStorageInfo);
+        return;
+      } else {
+        LOG.warn("Reserved storage {} reported as non-provided from {}",
+            storage, node);
+      }
+    }
+    node.updateStorage(storage);
+  }
+
   /**
    * Builder used for creating {@link LocatedBlocks} when a block is provided.
    */
@@ -295,10 +316,12 @@ public class ProvidedStorageMap {
    * An abstract DatanodeDescriptor to track datanodes with provided storages.
    * NOTE: never resolved through registerDatanode, so not in the topology.
    */
-  static class ProvidedDescriptor extends DatanodeDescriptor {
+  public static class ProvidedDescriptor extends DatanodeDescriptor {
 
     private final NavigableMap<String, DatanodeDescriptor> dns =
         new ConcurrentSkipListMap<>();
+    public final static String NETWORK_LOCATION = "/REMOTE";
+    public final static String NAME = "PROVIDED";
 
     ProvidedDescriptor() {
       super(new DatanodeID(
@@ -444,6 +467,21 @@ public class ProvidedStorageMap {
     public int hashCode() {
       return super.hashCode();
     }
+
+    @Override
+    public String toString() {
+      return "PROVIDED-LOCATION";
+    }
+
+    @Override
+    public String getNetworkLocation() {
+      return NETWORK_LOCATION;
+    }
+
+    @Override
+    public String getName() {
+      return NAME;
+    }
   }
 
   /**
@@ -480,7 +518,13 @@ public class ProvidedStorageMap {
         super.setState(state);
       }
     }
+
+    @Override
+    public String toString() {
+      return "PROVIDED-STORAGE";
+    }
   }
+
   /**
    * Used to emulate block reports for provided blocks.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
index 978009e..c335ec6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
@@ -22,6 +22,7 @@ import java.beans.ConstructorProperties;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
 
 /**
  * Statistics per StorageType.
@@ -36,6 +37,7 @@ public class StorageTypeStats {
   private long capacityRemaining = 0L;
   private long blockPoolUsed = 0L;
   private int nodesInService = 0;
+  private StorageType storageType;
 
   @ConstructorProperties({"capacityTotal", "capacityUsed", "capacityNonDfsUsed",
       "capacityRemaining", "blockPoolUsed", "nodesInService"})
@@ -51,22 +53,47 @@ public class StorageTypeStats {
   }
 
   public long getCapacityTotal() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityTotal/nodesInService;
+    }
     return capacityTotal;
   }
 
   public long getCapacityUsed() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityUsed/nodesInService;
+    }
     return capacityUsed;
   }
 
   public long getCapacityNonDfsUsed() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityNonDfsUsed/nodesInService;
+    }
     return capacityNonDfsUsed;
   }
 
   public long getCapacityRemaining() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityRemaining/nodesInService;
+    }
     return capacityRemaining;
   }
 
   public long getBlockPoolUsed() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return blockPoolUsed/nodesInService;
+    }
     return blockPoolUsed;
   }
 
@@ -74,7 +101,9 @@ public class StorageTypeStats {
     return nodesInService;
   }
 
-  StorageTypeStats() {}
+  StorageTypeStats(StorageType storageType) {
+    this.storageType = storageType;
+  }
 
   StorageTypeStats(StorageTypeStats other) {
     capacityTotal = other.capacityTotal;
@@ -87,6 +116,7 @@ public class StorageTypeStats {
 
   void addStorage(final DatanodeStorageInfo info,
       final DatanodeDescriptor node) {
+    assert storageType == info.getStorageType();
     capacityUsed += info.getDfsUsed();
     capacityNonDfsUsed += info.getNonDfsUsed();
     blockPoolUsed += info.getBlockPoolUsed();
@@ -106,6 +136,7 @@ public class StorageTypeStats {
 
   void subtractStorage(final DatanodeStorageInfo info,
       final DatanodeDescriptor node) {
+    assert storageType == info.getStorageType();
     capacityUsed -= info.getDfsUsed();
     capacityNonDfsUsed -= info.getNonDfsUsed();
     blockPoolUsed -= info.getBlockPoolUsed();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
deleted file mode 100644
index 24921c4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
+++ /dev/null
@@ -1,58 +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.datanode.fsdataset.impl;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * The default usage statistics for a provided volume.
- */
-public class DefaultProvidedVolumeDF
-    implements ProvidedVolumeDF, Configurable {
-
-  @Override
-  public void setConf(Configuration conf) {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return null;
-  }
-
-  @Override
-  public long getCapacity() {
-    return Long.MAX_VALUE;
-  }
-
-  @Override
-  public long getSpaceUsed() {
-    return 0;
-  }
-
-  @Override
-  public long getBlockPoolUsed(String bpid) {
-    return 0;
-  }
-
-  @Override
-  public long getAvailable() {
-    return Long.MAX_VALUE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
deleted file mode 100644
index 4d28883..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
+++ /dev/null
@@ -1,34 +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.datanode.fsdataset.impl;
-
-/**
- * This interface is used to define the usage statistics
- * of the provided storage.
- */
-public interface ProvidedVolumeDF {
-
-  long getCapacity();
-
-  long getSpaceUsed();
-
-  long getBlockPoolUsed(String bpid);
-
-  long getAvailable();
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 d103b64..65487f9 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
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -89,6 +90,30 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
     return suffix;
   }
 
+  /**
+   * Class to keep track of the capacity usage statistics for provided volumes.
+   */
+  public static class ProvidedVolumeDF {
+
+    private AtomicLong used = new AtomicLong();
+
+    public long getSpaceUsed() {
+      return used.get();
+    }
+
+    public void decDfsUsed(long value) {
+      used.addAndGet(-value);
+    }
+
+    public void incDfsUsed(long value) {
+      used.addAndGet(value);
+    }
+
+    public long getCapacity() {
+      return getSpaceUsed();
+    }
+  }
+
   static class ProvidedBlockPoolSlice {
     private ProvidedVolumeImpl providedVolume;
 
@@ -96,6 +121,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
     private Configuration conf;
     private String bpid;
     private ReplicaMap bpVolumeMap;
+    private ProvidedVolumeDF df;
+    private AtomicLong numOfBlocks = new AtomicLong();
 
     ProvidedBlockPoolSlice(String bpid, ProvidedVolumeImpl volume,
         Configuration conf) {
@@ -107,6 +134,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
       aliasMap = ReflectionUtils.newInstance(fmt, conf);
       this.conf = conf;
       this.bpid = bpid;
+      this.df = new ProvidedVolumeDF();
       bpVolumeMap.initBlockPool(bpid);
       LOG.info("Created alias map using class: " + aliasMap.getClass());
     }
@@ -155,6 +183,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
           if (oldReplica == null) {
             volumeMap.add(bpid, newReplica);
             bpVolumeMap.add(bpid, newReplica);
+            incrNumBlocks();
+            incDfsUsed(region.getBlock().getNumBytes());
           } else {
             throw new IOException("A block with id " + newReplica.getBlockId()
                 + " already exists in the volumeMap");
@@ -163,6 +193,10 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
       }
     }
 
+    private void incrNumBlocks() {
+      numOfBlocks.incrementAndGet();
+    }
+
     public boolean isEmpty() {
       return bpVolumeMap.replicas(bpid).size() == 0;
     }
@@ -199,6 +233,18 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
         }
       }
     }
+
+    public long getNumOfBlocks() {
+      return numOfBlocks.get();
+    }
+
+    long getDfsUsed() throws IOException {
+      return df.getSpaceUsed();
+    }
+
+    void incDfsUsed(long value) {
+      df.incDfsUsed(value);
+    }
   }
 
   private URI baseURI;
@@ -217,10 +263,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
       "Only provided storages must use ProvidedVolume";
 
     baseURI = getStorageLocation().getUri();
-    Class<? extends ProvidedVolumeDF> dfClass =
-        conf.getClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
-            DefaultProvidedVolumeDF.class, ProvidedVolumeDF.class);
-    df = ReflectionUtils.newInstance(dfClass, conf);
+    df = new ProvidedVolumeDF();
     remoteFS = FileSystem.get(baseURI, conf);
   }
 
@@ -231,34 +274,47 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
   @Override
   public long getCapacity() {
-    if (configuredCapacity < 0) {
-      return df.getCapacity();
+    try {
+      // default to whatever is the space used!
+      return getDfsUsed();
+    } catch (IOException e) {
+      LOG.warn("Exception when trying to get capacity of ProvidedVolume: {}",
+          e);
     }
-    return configuredCapacity;
+    return 0L;
   }
 
   @Override
   public long getDfsUsed() throws IOException {
-    return df.getSpaceUsed();
+    long dfsUsed = 0;
+    synchronized(getDataset()) {
+      for(ProvidedBlockPoolSlice s : bpSlices.values()) {
+        dfsUsed += s.getDfsUsed();
+      }
+    }
+    return dfsUsed;
   }
 
   @Override
   long getBlockPoolUsed(String bpid) throws IOException {
-    if (bpSlices.containsKey(bpid)) {
-      return df.getBlockPoolUsed(bpid);
-    } else {
-      throw new IOException("block pool " + bpid + " is not found");
-    }
+    return getProvidedBlockPoolSlice(bpid).getDfsUsed();
   }
 
   @Override
   public long getAvailable() throws IOException {
-    return df.getAvailable();
+    long remaining = getCapacity() - getDfsUsed();
+    // do not report less than 0 remaining space for PROVIDED storage
+    // to prevent marking it as over capacity on NN
+    if (remaining < 0L) {
+      LOG.warn("Volume {} has less than 0 available space", this);
+      return 0L;
+    }
+    return remaining;
   }
 
   @Override
   long getActualNonDfsUsed() throws IOException {
-    return df.getSpaceUsed();
+    return 0L;
   }
 
   @Override
@@ -267,6 +323,21 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   }
 
   @Override
+  long getNumBlocks() {
+    long numBlocks = 0;
+    for (ProvidedBlockPoolSlice s : bpSlices.values()) {
+      numBlocks += s.getNumOfBlocks();
+    }
+    return numBlocks;
+  }
+
+  @Override
+  void incDfsUsedAndNumBlocks(String bpid, long value) {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
   public URI getBaseURI() {
     return baseURI;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
index cb4245a..8abfc6e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
@@ -65,6 +65,12 @@ public interface FederationMBean {
   long getRemainingCapacity();
 
   /**
+   * Get the total remote storage capacity mounted in the federated cluster.
+   * @return Remote capacity of the federated cluster.
+   */
+  long getProvidedSpace();
+
+  /**
    * Get the number of nameservices in the federation.
    * @return Number of nameservices in the federation.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
index 7844a2e..4582825 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
@@ -272,6 +272,11 @@ public class FederationMetrics implements FederationMBean {
   }
 
   @Override
+  public long getProvidedSpace() {
+    return getNameserviceAggregatedLong(MembershipStats::getProvidedSpace);
+  }
+
+  @Override
   public long getUsedCapacity() {
     return getTotalCapacity() - getRemainingCapacity();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
index 23cd675..c4e5b5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
@@ -169,6 +169,11 @@ public class NamenodeBeanMetrics
   }
 
   @Override
+  public long getProvidedCapacity() {
+    return getFederationMetrics().getProvidedSpace();
+  }
+
+  @Override
   public String getSafemode() {
     // We assume that the global federated view is never in safe mode
     return "";
@@ -450,6 +455,11 @@ public class NamenodeBeanMetrics
   }
 
   @Override
+  public long getProvidedCapacityTotal() {
+    return getProvidedCapacity();
+  }
+
+  @Override
   public long getFilesTotal() {
     return getFederationMetrics().getNumFiles();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
index 98ddd22..b87eeec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
@@ -236,6 +236,7 @@ public class MembershipNamenodeResolver
           report.getNumOfBlocksPendingDeletion());
       stats.setAvailableSpace(report.getAvailableSpace());
       stats.setTotalSpace(report.getTotalSpace());
+      stats.setProvidedSpace(report.getProvidedSpace());
       stats.setNumOfDecommissioningDatanodes(
           report.getNumDecommissioningDatanodes());
       stats.setNumOfActiveDatanodes(report.getNumLiveDatanodes());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
index 555e2ee..d3c6d87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
@@ -58,6 +58,7 @@ public class NamenodeStatusReport {
   private long numOfBlocksUnderReplicated = -1;
   private long numOfBlocksPendingDeletion = -1;
   private long totalSpace = -1;
+  private long providedSpace = -1;
 
   /** If the fields are valid. */
   private boolean registrationValid = false;
@@ -296,7 +297,7 @@ public class NamenodeStatusReport {
   public void setNamesystemInfo(long available, long total,
       long numFiles, long numBlocks, long numBlocksMissing,
       long numBlocksPendingReplication, long numBlocksUnderReplicated,
-      long numBlocksPendingDeletion) {
+      long numBlocksPendingDeletion, long providedSpace) {
     this.totalSpace = total;
     this.availableSpace = available;
     this.numOfBlocks = numBlocks;
@@ -306,6 +307,7 @@ public class NamenodeStatusReport {
     this.numOfBlocksPendingDeletion = numBlocksPendingDeletion;
     this.numOfFiles = numFiles;
     this.statsValid = true;
+    this.providedSpace = providedSpace;
   }
 
   /**
@@ -345,6 +347,14 @@ public class NamenodeStatusReport {
   }
 
   /**
+   * Get the space occupied by provided storage.
+   *
+   * @return the provided capacity.
+   */
+  public long getProvidedSpace() {
+    return this.providedSpace;
+  }
+  /**
    * Get the number of missing blocks.
    *
    * @return Number of missing blocks.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
index 7d69a26..aaf2817 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
@@ -350,7 +350,8 @@ public class NamenodeHeartbeatService extends PeriodicService {
                 jsonObject.getLong("MissingBlocks"),
                 jsonObject.getLong("PendingReplicationBlocks"),
                 jsonObject.getLong("UnderReplicatedBlocks"),
-                jsonObject.getLong("PendingDeletionBlocks"));
+                jsonObject.getLong("PendingDeletionBlocks"),
+                jsonObject.getLong("ProvidedCapacityTotal"));
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
index 0bd19d9..654140c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
@@ -45,6 +45,10 @@ public abstract class MembershipStats extends BaseRecord {
 
   public abstract long getAvailableSpace();
 
+  public abstract void setProvidedSpace(long capacity);
+
+  public abstract long getProvidedSpace();
+
   public abstract void setNumOfFiles(long files);
 
   public abstract long getNumOfFiles();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
index 9f0a167..3347bc6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
@@ -78,6 +78,16 @@ public class MembershipStatsPBImpl extends MembershipStats
   }
 
   @Override
+  public void setProvidedSpace(long capacity) {
+    this.translator.getBuilder().setProvidedSpace(capacity);
+  }
+
+  @Override
+  public long getProvidedSpace() {
+    return this.translator.getProtoOrBuilder().getProvidedSpace();
+  }
+
+  @Override
   public void setNumOfFiles(long files) {
     this.translator.getBuilder().setNumOfFiles(files);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index dedb737..286c41c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4166,6 +4166,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return datanodeStatistics.getCapacityRemaining();
   }
 
+  @Override // FSNamesystemMBean
+  @Metric({"ProvidedCapacityTotal",
+      "Total space used in PROVIDED storage in bytes" })
+  public long getProvidedCapacityTotal() {
+    return datanodeStatistics.getProvidedCapacity();
+  }
+
   @Metric({"CapacityRemainingGB", "Remaining capacity in GB"})
   public float getCapacityRemainingGB() {
     return DFSUtil.roundBytesToGB(getCapacityRemaining());
@@ -5730,6 +5737,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // NameNodeMXBean
+  public long getProvidedCapacity() {
+    return this.getProvidedCapacityTotal();
+  }
+
+  @Override // NameNodeMXBean
   public String getSafemode() {
     if (!this.isInSafeMode())
       return "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
index 82cec33..e4ed3a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
@@ -65,8 +65,14 @@ public interface NameNodeMXBean {
    * @return the total raw bytes including non-dfs used space
    */
   public long getTotal();
-  
-  
+
+  /**
+   * Gets capacity of the provided storage mounted, in bytes.
+   *
+   * @return the total raw bytes present in the provided storage.
+   */
+  public long getProvidedCapacity();
+
   /**
    * Gets the safemode status
    * 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
index ebdbc12..c25bafd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
@@ -69,7 +69,12 @@ public interface FSNamesystemMBean {
    * @return -  used capacity in bytes
    */
   public long getCapacityUsed();
- 
+
+  /**
+   * Total PROVIDED storage capacity.
+   * @return -  total PROVIDED storage capacity in bytes
+   */
+  public long getProvidedCapacityTotal();
 
   /**
    * Total number of files and directories

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
index 88acd08..043a21a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
@@ -30,6 +30,7 @@ package hadoop.hdfs;
 message NamenodeMembershipStatsRecordProto {
   optional uint64 totalSpace = 1;
   optional uint64 availableSpace = 2;
+  optional uint64 providedSpace = 3;
 
   optional uint64 numOfFiles = 10;
   optional uint64 numOfBlocks = 11;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 835d8c4..655f9cb 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
@@ -4630,14 +4630,6 @@
   </property>
 
   <property>
-    <name>dfs.provided.df.class</name>
-    <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
-    <description>
-      The class that is used to measure usage statistics of provided stores.
-    </description>
-  </property>
-
-  <property>
     <name>dfs.provided.storage.id</name>
     <value>DS-PROVIDED</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
index 6ae3960..45aee1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
@@ -162,6 +162,7 @@
 {#nn}
 <table class="table table-bordered table-striped">
   <tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
+  <tr><th> Configured Remote Capacity:</th><td>{ProvidedCapacity|fmt_bytes}</td></tr>
   <tr><th> DFS Used:</th><td>{Used|fmt_bytes} ({PercentUsed|fmt_percentage})</td></tr>
   <tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
   <tr><th> DFS Remaining:</th><td>{Free|fmt_bytes} ({PercentRemaining|fmt_percentage})</td></tr>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 89741b5..1ef2f2b 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
@@ -63,15 +63,15 @@ public class TestProvidedStorageMap {
 
   private DatanodeDescriptor createDatanodeDescriptor(int port) {
     return DFSTestUtil.getDatanodeDescriptor("127.0.0.1", port, "defaultRack",
-            "localhost");
+        "localhost");
   }
 
   @Test
   public void testProvidedStorageMap() throws IOException {
     ProvidedStorageMap providedMap = new ProvidedStorageMap(
-            nameSystemLock, bm, conf);
+        nameSystemLock, bm, conf);
     DatanodeStorageInfo providedMapStorage =
-            providedMap.getProvidedStorageInfo();
+        providedMap.getProvidedStorageInfo();
     //the provided storage cannot be null
     assertNotNull(providedMapStorage);
 
@@ -80,41 +80,40 @@ public class TestProvidedStorageMap {
 
     //associate two storages to the datanode
     DatanodeStorage dn1ProvidedStorage = new DatanodeStorage(
-            providedStorageID,
-            DatanodeStorage.State.NORMAL,
-            StorageType.PROVIDED);
+        providedStorageID,
+        DatanodeStorage.State.NORMAL,
+        StorageType.PROVIDED);
     DatanodeStorage dn1DiskStorage = new DatanodeStorage(
-            "sid-1", DatanodeStorage.State.NORMAL, StorageType.DISK);
+        "sid-1", DatanodeStorage.State.NORMAL, StorageType.DISK);
 
     when(nameSystemLock.hasWriteLock()).thenReturn(true);
-    DatanodeStorageInfo dns1Provided = providedMap.getStorage(dn1,
-            dn1ProvidedStorage, null);
-    DatanodeStorageInfo dns1Disk = providedMap.getStorage(dn1,
-            dn1DiskStorage, null);
+    DatanodeStorageInfo dns1Provided =
+        providedMap.getStorage(dn1, dn1ProvidedStorage);
+    DatanodeStorageInfo dns1Disk = providedMap.getStorage(dn1, dn1DiskStorage);
 
     assertTrue("The provided storages should be equal",
-            dns1Provided == providedMapStorage);
+        dns1Provided == providedMapStorage);
     assertTrue("Disk storage has not yet been registered with block manager",
-            dns1Disk == null);
+        dns1Disk == null);
     //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, null));
+        dnsDisk == providedMap.getStorage(dn1, dn1DiskStorage));
 
     //create a 2nd datanode
     DatanodeDescriptor dn2 = createDatanodeDescriptor(5010);
     //associate a provided storage with the datanode
     DatanodeStorage dn2ProvidedStorage = new DatanodeStorage(
-            providedStorageID,
-            DatanodeStorage.State.NORMAL,
-            StorageType.PROVIDED);
+        providedStorageID,
+        DatanodeStorage.State.NORMAL,
+        StorageType.PROVIDED);
 
     DatanodeStorageInfo dns2Provided = providedMap.getStorage(
-            dn2, dn2ProvidedStorage, null);
+        dn2, dn2ProvidedStorage);
     assertTrue("The provided storages should be equal",
-            dns2Provided == providedMapStorage);
+        dns2Provided == providedMapStorage);
     assertTrue("The DatanodeDescriptor should contain the provided storage",
-            dn2.getStorageInfo(providedStorageID) == providedMapStorage);
+        dn2.getStorageInfo(providedStorageID) == providedMapStorage);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 ecab06b..52112f7 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
@@ -46,7 +46,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
@@ -102,6 +101,7 @@ public class TestProvidedImpl {
   private FsDatasetImpl dataset;
   private static Map<Long, String> blkToPathMap;
   private static List<FsVolumeImpl> providedVolumes;
+  private static long spaceUsed = 0;
 
   /**
    * A simple FileRegion iterator for tests.
@@ -142,6 +142,7 @@ public class TestProvidedImpl {
             }
             writer.flush();
             writer.close();
+            spaceUsed += BLK_LEN;
           } catch (IOException e) {
             e.printStackTrace();
           }
@@ -240,39 +241,6 @@ public class TestProvidedImpl {
     }
   }
 
-  public static class TestProvidedVolumeDF
-      implements ProvidedVolumeDF, Configurable {
-
-    @Override
-    public void setConf(Configuration conf) {
-    }
-
-    @Override
-    public Configuration getConf() {
-      return null;
-    }
-
-    @Override
-    public long getCapacity() {
-      return Long.MAX_VALUE;
-    }
-
-    @Override
-    public long getSpaceUsed() {
-      return -1;
-    }
-
-    @Override
-    public long getBlockPoolUsed(String bpid) {
-      return -1;
-    }
-
-    @Override
-    public long getAvailable() {
-      return Long.MAX_VALUE;
-    }
-  }
-
   private static Storage.StorageDirectory createLocalStorageDirectory(
       File root, Configuration conf)
       throws SecurityException, IOException {
@@ -370,6 +338,8 @@ public class TestProvidedImpl {
     when(datanode.getConf()).thenReturn(conf);
     final DNConf dnConf = new DNConf(datanode);
     when(datanode.getDnConf()).thenReturn(dnConf);
+    // reset the space used
+    spaceUsed = 0;
 
     final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
     when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
@@ -379,8 +349,6 @@ public class TestProvidedImpl {
 
     this.conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
         TestFileRegionBlockAliasMap.class, BlockAliasMap.class);
-    conf.setClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
-        TestProvidedVolumeDF.class, ProvidedVolumeDF.class);
 
     blkToPathMap = new HashMap<Long, String>();
     providedVolumes = new LinkedList<FsVolumeImpl>();
@@ -410,8 +378,6 @@ public class TestProvidedImpl {
     assertEquals(NUM_PROVIDED_INIT_VOLUMES, providedVolumes.size());
     assertEquals(0, dataset.getNumFailedVolumes());
 
-    TestProvidedVolumeDF df = new TestProvidedVolumeDF();
-
     for (int i = 0; i < providedVolumes.size(); i++) {
       //check basic information about provided volume
       assertEquals(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT,
@@ -419,18 +385,17 @@ public class TestProvidedImpl {
       assertEquals(StorageType.PROVIDED,
           providedVolumes.get(i).getStorageType());
 
+      long space = providedVolumes.get(i).getBlockPoolUsed(
+              BLOCK_POOL_IDS[CHOSEN_BP_ID]);
       //check the df stats of the volume
-      assertEquals(df.getAvailable(), providedVolumes.get(i).getAvailable());
-      assertEquals(df.getBlockPoolUsed(BLOCK_POOL_IDS[CHOSEN_BP_ID]),
-          providedVolumes.get(i).getBlockPoolUsed(
-              BLOCK_POOL_IDS[CHOSEN_BP_ID]));
+      assertEquals(spaceUsed, space);
+      assertEquals(NUM_PROVIDED_BLKS, providedVolumes.get(i).getNumBlocks());
 
       providedVolumes.get(i).shutdownBlockPool(
           BLOCK_POOL_IDS[1 - CHOSEN_BP_ID], null);
       try {
-        assertEquals(df.getBlockPoolUsed(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]),
-            providedVolumes.get(i).getBlockPoolUsed(
-                BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]));
+        assertEquals(0, providedVolumes.get(i)
+            .getBlockPoolUsed(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]));
         //should not be triggered
         assertTrue(false);
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
index 61fda0e..b445df5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
@@ -190,6 +190,8 @@ public class TestFederationMetrics extends TestMetricsBase {
           json.getLong("numOfDecomActiveDatanodes"));
       assertEquals(stats.getNumOfDecomDeadDatanodes(),
           json.getLong("numOfDecomDeadDatanodes"));
+      assertEquals(stats.getProvidedSpace(),
+          json.getLong("providedSpace"));
       nameservicesFound++;
     }
     assertEquals(getNameservices().size(), nameservicesFound);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b1d3030/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 22f00aa..f6d38f6 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
@@ -27,6 +27,7 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
+import java.util.Iterator;
 import java.util.Random;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.conf.Configuration;
@@ -44,13 +45,23 @@ 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.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.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.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 
 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;
@@ -59,6 +70,7 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
 import static org.junit.Assert.*;
 
 public class TestNameNodeProvidedImplementation {
@@ -79,6 +91,7 @@ public class TestNameNodeProvidedImplementation {
   private final String filePrefix = "file";
   private final String fileSuffix = ".dat";
   private final int baseFileLen = 1024;
+  private long providedDataSize = 0;
 
   Configuration conf;
   MiniDFSCluster cluster;
@@ -135,6 +148,7 @@ public class TestNameNodeProvidedImplementation {
           }
           writer.flush();
           writer.close();
+          providedDataSize += newFile.length();
         } catch (IOException e) {
           e.printStackTrace();
         }
@@ -206,13 +220,14 @@ public class TestNameNodeProvidedImplementation {
     cluster.waitActive();
   }
 
-  @Test(timeout = 20000)
+  @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},
-        null, false);
+    startCluster(NNDIRPATH, 0,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
 
     FileSystem fs = cluster.getFileSystem();
     for (TreePath e : new RandomTreeWalk(seed)) {
@@ -231,14 +246,83 @@ public class TestNameNodeProvidedImplementation {
     }
   }
 
-  @Test(timeout=20000)
-  public void testBlockLoad() throws Exception {
+  @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);
-    startCluster(NNDIRPATH, 1, new StorageType[] {StorageType.PROVIDED},
-        null, false);
+    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)
@@ -250,8 +334,8 @@ public class TestNameNodeProvidedImplementation {
     // make the last Datanode with only DISK
     startCluster(NNDIRPATH, 3, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
     // wait for the replication to finish
@@ -308,8 +392,9 @@ public class TestNameNodeProvidedImplementation {
         FsUGIResolver.class, UGIResolver.class);
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3, new StorageType[] {StorageType.PROVIDED},
-        null, false);
+    startCluster(NNDIRPATH, 3,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
     FileSystem fs = cluster.getFileSystem();
     Thread.sleep(2000);
     int count = 0;
@@ -371,7 +456,7 @@ public class TestNameNodeProvidedImplementation {
     return fs.getFileBlockLocations(path, 0, fileLen);
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testClusterWithEmptyImage() throws IOException {
     // start a cluster with 2 datanodes without any provided storage
     startCluster(NNDIRPATH, 2, null,
@@ -404,7 +489,7 @@ public class TestNameNodeProvidedImplementation {
    * Tests setting replication of provided files.
    * @throws Exception
    */
-  @Test
+  @Test(timeout=30000)
   public void testSetReplicationForProvidedFiles() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
@@ -441,14 +526,14 @@ public class TestNameNodeProvidedImplementation {
     getAndCheckBlockLocations(client, filename, newReplication);
   }
 
-  @Test
+  @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.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
@@ -511,7 +596,7 @@ public class TestNameNodeProvidedImplementation {
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
@@ -540,7 +625,7 @@ public class TestNameNodeProvidedImplementation {
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
@@ -570,7 +655,7 @@ public class TestNameNodeProvidedImplementation {
     }
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testSetClusterID() throws Exception {
     String clusterID = "PROVIDED-CLUSTER";
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
@@ -578,7 +663,7 @@ public class TestNameNodeProvidedImplementation {
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
     NameNode nn = cluster.getNameNode();


---------------------------------------------------------------------
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: YARN-7522. Introduce AllocationTagsManager to associate allocation tags to nodes. (Wangda Tan via asuresh)

Posted by kk...@apache.org.
YARN-7522. Introduce AllocationTagsManager to associate allocation tags to nodes. (Wangda Tan via asuresh)


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

Branch: refs/heads/YARN-6592
Commit: bf2a8ccc0908d94b974a78bb4aba0035149dfc77
Parents: ca28a79
Author: Arun Suresh <as...@apache.org>
Authored: Fri Dec 8 00:24:00 2017 -0800
Committer: Konstantinos Karanasos <kk...@apache.org>
Committed: Mon Dec 18 16:07:00 2017 -0800

----------------------------------------------------------------------
 .../resourcemanager/RMActiveServiceContext.java |  15 +
 .../yarn/server/resourcemanager/RMContext.java  |   5 +
 .../server/resourcemanager/RMContextImpl.java   |  12 +
 .../server/resourcemanager/ResourceManager.java |   9 +
 .../constraint/AllocationTagsManager.java       | 431 +++++++++++++++++++
 .../constraint/AllocationTagsNamespaces.java    |  31 ++
 .../InvalidAllocationTagsQueryException.java    |  35 ++
 .../rmcontainer/RMContainer.java                |   8 +
 .../rmcontainer/RMContainerImpl.java            |  21 +
 .../constraint/TestAllocationTagsManager.java   | 328 ++++++++++++++
 .../rmcontainer/TestRMContainerImpl.java        | 124 ++++++
 .../scheduler/capacity/TestUtils.java           |   9 +
 .../scheduler/fifo/TestFifoScheduler.java       |   5 +
 13 files changed, 1033 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.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/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 9dc5945..6ee3a4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -107,6 +108,7 @@ public class RMActiveServiceContext {
 
   private RMAppLifetimeMonitor rmAppLifetimeMonitor;
   private QueueLimitCalculator queueLimitCalculator;
+  private AllocationTagsManager allocationTagsManager;
 
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
@@ -398,6 +400,19 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
+  public AllocationTagsManager getAllocationTagsManager() {
+    return allocationTagsManager;
+  }
+
+  @Private
+  @Unstable
+  public void setAllocationTagsManager(
+      AllocationTagsManager allocationTagsManager) {
+    this.allocationTagsManager = allocationTagsManager;
+  }
+
+  @Private
+  @Unstable
   public RMDelegatedNodeLabelsUpdater getRMDelegatedNodeLabelsUpdater() {
     return rmDelegatedNodeLabelsUpdater;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index ec94030..62899d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -166,4 +167,8 @@ public interface RMContext extends ApplicationMasterServiceContext {
   void setResourceProfilesManager(ResourceProfilesManager mgr);
 
   String getAppProxyUrl(Configuration conf, ApplicationId applicationId);
+
+  AllocationTagsManager getAllocationTagsManager();
+
+  void setAllocationTagsManager(AllocationTagsManager allocationTagsManager);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.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/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 80a9109..315fdc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -504,6 +505,17 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public AllocationTagsManager getAllocationTagsManager() {
+    return activeServiceContext.getAllocationTagsManager();
+  }
+
+  @Override
+  public void setAllocationTagsManager(
+      AllocationTagsManager allocationTagsManager) {
+    activeServiceContext.setAllocationTagsManager(allocationTagsManager);
+  }
+
+  @Override
   public RMDelegatedNodeLabelsUpdater getRMDelegatedNodeLabelsUpdater() {
     return activeServiceContext.getRMDelegatedNodeLabelsUpdater();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index a0317f6..8d1000e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Pu
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@@ -490,6 +491,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       throws InstantiationException, IllegalAccessException {
     return new RMNodeLabelsManager();
   }
+
+  protected AllocationTagsManager createAllocationTagsManager() {
+    return new AllocationTagsManager();
+  }
   
   protected DelegationTokenRenewer createDelegationTokenRenewer() {
     return new DelegationTokenRenewer();
@@ -609,6 +614,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       addService(nlm);
       rmContext.setNodeLabelManager(nlm);
 
+      AllocationTagsManager allocationTagsManager =
+          createAllocationTagsManager();
+      rmContext.setAllocationTagsManager(allocationTagsManager);
+
       RMDelegatedNodeLabelsUpdater delegatedNodeLabelsUpdater =
           createRMDelegatedNodeLabelsUpdater();
       if (delegatedNodeLabelsUpdater != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsManager.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/constraint/AllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsManager.java
new file mode 100644
index 0000000..b67fab9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsManager.java
@@ -0,0 +1,431 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.constraint;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.LongBinaryOperator;
+
+/**
+ * Support storing maps between container-tags/applications and
+ * nodes. This will be required by affinity/anti-affinity implementation and
+ * cardinality.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class AllocationTagsManager {
+
+  private static final Logger LOG = Logger.getLogger(
+      AllocationTagsManager.class);
+
+  private ReentrantReadWriteLock.ReadLock readLock;
+  private ReentrantReadWriteLock.WriteLock writeLock;
+
+  // Application's tags to node
+  private Map<ApplicationId, NodeToCountedTags> perAppMappings =
+      new HashMap<>();
+
+  // Global tags to node mapping (used to fast return aggregated tags
+  // cardinality across apps)
+  private NodeToCountedTags globalMapping = new NodeToCountedTags();
+
+  /**
+   * Store node to counted tags.
+   */
+  @VisibleForTesting
+  static class NodeToCountedTags {
+    // Map<NodeId, Map<Tag, Count>>
+    private Map<NodeId, Map<String, Long>> nodeToTagsWithCount =
+        new HashMap<>();
+
+    // protected by external locks
+    private void addTagsToNode(NodeId nodeId, Set<String> tags) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.computeIfAbsent(nodeId,
+          k -> new HashMap<>());
+
+      for (String tag : tags) {
+        Long count = innerMap.get(tag);
+        if (count == null) {
+          innerMap.put(tag, 1L);
+        } else{
+          innerMap.put(tag, count + 1);
+        }
+      }
+    }
+
+    // protected by external locks
+    private void addTagToNode(NodeId nodeId, String tag) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.computeIfAbsent(nodeId,
+          k -> new HashMap<>());
+
+      Long count = innerMap.get(tag);
+      if (count == null) {
+        innerMap.put(tag, 1L);
+      } else{
+        innerMap.put(tag, count + 1);
+      }
+    }
+
+    private void removeTagFromInnerMap(Map<String, Long> innerMap, String tag) {
+      Long count = innerMap.get(tag);
+      if (count > 1) {
+        innerMap.put(tag, count - 1);
+      } else {
+        if (count <= 0) {
+          LOG.warn(
+              "Trying to remove tags from node, however the count already"
+                  + " becomes 0 or less, it could be a potential bug.");
+        }
+        innerMap.remove(tag);
+      }
+    }
+
+    private void removeTagsFromNode(NodeId nodeId, Set<String> tags) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        LOG.warn("Failed to find node=" + nodeId
+            + " while trying to remove tags, please double check.");
+        return;
+      }
+
+      for (String tag : tags) {
+        removeTagFromInnerMap(innerMap, tag);
+      }
+
+      if (innerMap.isEmpty()) {
+        nodeToTagsWithCount.remove(nodeId);
+      }
+    }
+
+    private void removeTagFromNode(NodeId nodeId, String tag) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        LOG.warn("Failed to find node=" + nodeId
+            + " while trying to remove tags, please double check.");
+        return;
+      }
+
+      removeTagFromInnerMap(innerMap, tag);
+
+      if (innerMap.isEmpty()) {
+        nodeToTagsWithCount.remove(nodeId);
+      }
+    }
+
+    private long getCardinality(NodeId nodeId, String tag) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        return 0;
+      }
+      Long value = innerMap.get(tag);
+      return value == null ? 0 : value;
+    }
+
+    private long getCardinality(NodeId nodeId, Set<String> tags,
+        LongBinaryOperator op) {
+      Map<String, Long> innerMap = nodeToTagsWithCount.get(nodeId);
+      if (innerMap == null) {
+        return 0;
+      }
+
+      long returnValue = 0;
+      boolean firstTag = true;
+
+      if (tags != null && !tags.isEmpty()) {
+        for (String tag : tags) {
+          Long value = innerMap.get(tag);
+          if (value == null) {
+            value = 0L;
+          }
+
+          if (firstTag) {
+            returnValue = value;
+            firstTag = false;
+            continue;
+          }
+
+          returnValue = op.applyAsLong(returnValue, value);
+        }
+      } else {
+        // Similar to above if, but only iterate values for better performance
+        for (long value : innerMap.values()) {
+          // For the first value, we will not apply op
+          if (firstTag) {
+            returnValue = value;
+            firstTag = false;
+            continue;
+          }
+          returnValue = op.applyAsLong(returnValue, value);
+        }
+      }
+      return returnValue;
+    }
+
+    private boolean isEmpty() {
+      return nodeToTagsWithCount.isEmpty();
+    }
+
+    @VisibleForTesting
+    public Map<NodeId, Map<String, Long>> getNodeToTagsWithCount() {
+      return nodeToTagsWithCount;
+    }
+  }
+
+  @VisibleForTesting
+  Map<ApplicationId, NodeToCountedTags> getPerAppMappings() {
+    return perAppMappings;
+  }
+
+  @VisibleForTesting
+  NodeToCountedTags getGlobalMapping() {
+    return globalMapping;
+  }
+
+  public AllocationTagsManager() {
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  /**
+   * Notify container allocated on a node.
+   *
+   * @param nodeId         allocated node.
+   * @param applicationId  applicationId
+   * @param containerId    container id.
+   * @param allocationTags allocation tags, see
+   *                       {@link SchedulingRequest#getAllocationTags()}
+   *                       application_id will be added to allocationTags.
+   */
+  public void addContainer(NodeId nodeId, ApplicationId applicationId,
+      ContainerId containerId, Set<String> allocationTags) {
+    String applicationIdTag =
+        AllocationTagsNamespaces.APP_ID + applicationId.toString();
+
+    boolean useSet = false;
+    if (allocationTags != null && !allocationTags.isEmpty()) {
+      // Copy before edit it.
+      allocationTags = new HashSet<>(allocationTags);
+      allocationTags.add(applicationIdTag);
+      useSet = true;
+    }
+
+    writeLock.lock();
+    try {
+      NodeToCountedTags perAppTagsMapping = perAppMappings.computeIfAbsent(
+          applicationId, k -> new NodeToCountedTags());
+
+      if (useSet) {
+        perAppTagsMapping.addTagsToNode(nodeId, allocationTags);
+        globalMapping.addTagsToNode(nodeId, allocationTags);
+      } else {
+        perAppTagsMapping.addTagToNode(nodeId, applicationIdTag);
+        globalMapping.addTagToNode(nodeId, applicationIdTag);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Added container=" + containerId + " with tags=[" + StringUtils
+                .join(allocationTags, ",") + "]");
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Notify container removed.
+   *
+   * @param nodeId         nodeId
+   * @param applicationId  applicationId
+   * @param containerId    containerId.
+   * @param allocationTags allocation tags for given container
+   */
+  public void removeContainer(NodeId nodeId, ApplicationId applicationId,
+      ContainerId containerId, Set<String> allocationTags) {
+    String applicationIdTag =
+        AllocationTagsNamespaces.APP_ID + applicationId.toString();
+    boolean useSet = false;
+
+    if (allocationTags != null && !allocationTags.isEmpty()) {
+      // Copy before edit it.
+      allocationTags = new HashSet<>(allocationTags);
+      allocationTags.add(applicationIdTag);
+      useSet = true;
+    }
+
+    writeLock.lock();
+    try {
+      NodeToCountedTags perAppTagsMapping = perAppMappings.get(applicationId);
+      if (perAppTagsMapping == null) {
+        return;
+      }
+
+      if (useSet) {
+        perAppTagsMapping.removeTagsFromNode(nodeId, allocationTags);
+        globalMapping.removeTagsFromNode(nodeId, allocationTags);
+      } else {
+        perAppTagsMapping.removeTagFromNode(nodeId, applicationIdTag);
+        globalMapping.removeTagFromNode(nodeId, applicationIdTag);
+      }
+
+      if (perAppTagsMapping.isEmpty()) {
+        perAppMappings.remove(applicationId);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Removed container=" + containerId + " with tags=[" + StringUtils
+                .join(allocationTags, ",") + "]");
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Get cardinality for following conditions. External can pass-in a binary op
+   * to implement customized logic.   *
+   * @param nodeId        nodeId, required.
+   * @param applicationId applicationId. When null is specified, return
+   *                      aggregated cardinality among all nodes.
+   * @param tag           allocation tag, see
+   *                      {@link SchedulingRequest#getAllocationTags()},
+   *                      When multiple tags specified. Returns cardinality
+   *                      depends on op. If a specified tag doesn't exist,
+   *                      0 will be its cardinality.
+   *                      When null/empty tags specified, all tags
+   *                      (of the node/app) will be considered.
+   * @return cardinality of specified query on the node.
+   * @throws InvalidAllocationTagsQueryException when illegal query
+   *                                            parameter specified
+   */
+  public long getNodeCardinality(NodeId nodeId, ApplicationId applicationId,
+      String tag) throws InvalidAllocationTagsQueryException {
+    readLock.lock();
+
+    try {
+      if (nodeId == null) {
+        throw new InvalidAllocationTagsQueryException(
+            "Must specify nodeId/tags/op to query cardinality");
+      }
+
+      NodeToCountedTags mapping;
+      if (applicationId != null) {
+        mapping = perAppMappings.get(applicationId);
+      } else{
+        mapping = globalMapping;
+      }
+
+      if (mapping == null) {
+        return 0;
+      }
+
+      return mapping.getCardinality(nodeId, tag);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Check if given tag exists on node.
+   *
+   * @param nodeId        nodeId, required.
+   * @param applicationId applicationId. When null is specified, return
+   *                      aggregated cardinality among all nodes.
+   * @param tag           allocation tag, see
+   *                      {@link SchedulingRequest#getAllocationTags()},
+   *                      When multiple tags specified. Returns cardinality
+   *                      depends on op. If a specified tag doesn't exist,
+   *                      0 will be its cardinality.
+   *                      When null/empty tags specified, all tags
+   *                      (of the node/app) will be considered.
+   * @return cardinality of specified query on the node.
+   * @throws InvalidAllocationTagsQueryException when illegal query
+   *                                            parameter specified
+   */
+  public boolean allocationTagExistsOnNode(NodeId nodeId,
+      ApplicationId applicationId, String tag)
+      throws InvalidAllocationTagsQueryException {
+    return getNodeCardinality(nodeId, applicationId, tag) > 0;
+  }
+
+  /**
+   * Get cardinality for following conditions. External can pass-in a binary op
+   * to implement customized logic.
+   *
+   * @param nodeId        nodeId, required.
+   * @param applicationId applicationId. When null is specified, return
+   *                      aggregated cardinality among all nodes.
+   * @param tags          allocation tags, see
+   *                      {@link SchedulingRequest#getAllocationTags()},
+   *                      When multiple tags specified. Returns cardinality
+   *                      depends on op. If a specified tag doesn't exist, 0
+   *                      will be its cardinality. When null/empty tags
+   *                      specified, all tags (of the node/app) will be
+   *                      considered.
+   * @param op            operator. Such as Long::max, Long::sum, etc. Required.
+   *                      This sparameter only take effect when #values >= 2.
+   * @return cardinality of specified query on the node.
+   * @throws InvalidAllocationTagsQueryException when illegal query
+   *                                            parameter specified
+   */
+  public long getNodeCardinalityByOp(NodeId nodeId, ApplicationId applicationId,
+      Set<String> tags, LongBinaryOperator op)
+      throws InvalidAllocationTagsQueryException {
+    readLock.lock();
+
+    try {
+      if (nodeId == null || op == null) {
+        throw new InvalidAllocationTagsQueryException(
+            "Must specify nodeId/tags/op to query cardinality");
+      }
+
+      NodeToCountedTags mapping;
+      if (applicationId != null) {
+        mapping = perAppMappings.get(applicationId);
+      } else{
+        mapping = globalMapping;
+      }
+
+      if (mapping == null) {
+        return 0;
+      }
+
+      return mapping.getCardinality(nodeId, tags, op);
+    } finally {
+      readLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsNamespaces.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/constraint/AllocationTagsNamespaces.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsNamespaces.java
new file mode 100644
index 0000000..893ff1c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/AllocationTagsNamespaces.java
@@ -0,0 +1,31 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.constraint;
+
+/**
+ * Predefined namespaces for tags
+ *
+ * Same as namespace  of resource types. Namespaces of placement tags are start
+ * with alphabets and ended with "/"
+ */
+public class AllocationTagsNamespaces {
+  public static final String APP_ID = "yarn_app_id/";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/InvalidAllocationTagsQueryException.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/constraint/InvalidAllocationTagsQueryException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/InvalidAllocationTagsQueryException.java
new file mode 100644
index 0000000..5519e39
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/InvalidAllocationTagsQueryException.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.server.resourcemanager.constraint;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception when invalid parameter specified to do placement tags related
+ * queries.
+ */
+public class InvalidAllocationTagsQueryException extends YarnException {
+  private static final long serialVersionUID = 12312831974894L;
+
+  public InvalidAllocationTagsQueryException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.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/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index f3cbf63..8f751b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -30,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -115,4 +117,10 @@ public interface RMContainer extends EventHandler<RMContainerEvent>,
   boolean completed();
 
   NodeId getNodeId();
+
+  /**
+   * Return {@link SchedulingRequest#getAllocationTags()} specified by AM.
+   * @return allocation tags, could be null/empty
+   */
+  Set<String> getAllocationTags();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index e26689e..184cdfc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -189,6 +190,9 @@ public class RMContainerImpl implements RMContainer {
   private boolean isExternallyAllocated;
   private SchedulerRequestKey allocatedSchedulerKey;
 
+  // TODO, set it when container allocated by scheduler (From SchedulingRequest)
+  private Set<String> allocationTags = null;
+
   public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext) {
@@ -501,6 +505,11 @@ public class RMContainerImpl implements RMContainer {
     return nodeId;
   }
 
+  @Override
+  public Set<String> getAllocationTags() {
+    return allocationTags;
+  }
+
   private static class BaseTransition implements
       SingleArcTransition<RMContainerImpl, RMContainerEvent> {
 
@@ -565,6 +574,12 @@ public class RMContainerImpl implements RMContainer {
 
     @Override
     public void transition(RMContainerImpl container, RMContainerEvent event) {
+      // Notify placementManager
+      container.rmContext.getAllocationTagsManager().addContainer(
+          container.getNodeId(),
+          container.getApplicationAttemptId().getApplicationId(),
+          container.getContainerId(), container.getAllocationTags());
+
       container.eventHandler.handle(new RMAppAttemptEvent(
           container.appAttemptId, RMAppAttemptEventType.CONTAINER_ALLOCATED));
     }
@@ -676,6 +691,12 @@ public class RMContainerImpl implements RMContainer {
 
     @Override
     public void transition(RMContainerImpl container, RMContainerEvent event) {
+      // Notify placementManager
+      container.rmContext.getAllocationTagsManager().removeContainer(
+          container.getNodeId(),
+          container.getApplicationAttemptId().getApplicationId(),
+          container.getContainerId(), container.getAllocationTags());
+
       RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) event;
 
       container.finishTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/TestAllocationTagsManager.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/constraint/TestAllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/TestAllocationTagsManager.java
new file mode 100644
index 0000000..0358792
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/constraint/TestAllocationTagsManager.java
@@ -0,0 +1,328 @@
+/*
+ * *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ * /
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.constraint;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test functionality of AllocationTagsManager.
+ */
+public class TestAllocationTagsManager {
+  @Test
+  public void testAllocationTagsManagerSimpleCases()
+      throws InvalidAllocationTagsQueryException {
+    AllocationTagsManager atm = new AllocationTagsManager();
+
+    /**
+     * Construct test case:
+     * Node1:
+     *    container_1_1 (mapper/reducer/app_1)
+     *    container_1_3 (service/app_1)
+     *
+     * Node2:
+     *    container_1_2 (mapper/reducer/app_1)
+     *    container_1_4 (reducer/app_1)
+     *    container_2_1 (service/app_2)
+     */
+
+    // 3 Containers from app1
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    // 1 Container from app2
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Get Cardinality of app1 on node1, with tag "mapper"
+    Assert.assertEquals(1,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node1:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of("mapper"),
+            Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=min
+    Assert.assertEquals(1,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::min));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=max
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=sum
+    Assert.assertEquals(3,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::sum));
+
+    // Get Cardinality by passing single tag.
+    Assert.assertEquals(1,
+        atm.getNodeCardinality(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), "mapper"));
+
+    Assert.assertEquals(2,
+        atm.getNodeCardinality(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), "reducer"));
+
+    // Get Cardinality of app1 on node2, with tag "no_existed/reducer", op=min
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("no_existed", "reducer"), Long::min));
+
+    // Get Cardinality of app1 on node2, with tag "<applicationId>", op=max
+    // (Expect this returns #containers from app1 on node2)
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet
+                .of(AllocationTagsNamespaces.APP_ID + TestUtils
+                    .getMockApplicationId(1).toString()), Long::max));
+
+    // Get Cardinality of app1 on node2, with empty tag set, op=max
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
+
+    // Get Cardinality of all apps on node2, with empty tag set, op=sum
+    Assert.assertEquals(7,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"), null,
+            ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(5,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(2,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(2), ImmutableSet.of(), Long::sum));
+
+    // Finish all containers:
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Expect all cardinality to be 0
+    // Get Cardinality of app1 on node1, with tag "mapper"
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node1:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of("mapper"),
+            Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=min
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::min));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=max
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::max));
+
+    // Get Cardinality of app1 on node2, with tag "mapper/reducer", op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of("mapper", "reducer"), Long::sum));
+
+    // Get Cardinality of app1 on node2, with tag "<applicationId>", op=max
+    // (Expect this returns #containers from app1 on node2)
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            ImmutableSet.of(TestUtils.getMockApplicationId(1).toString()),
+            Long::max));
+
+    Assert.assertEquals(0,
+        atm.getNodeCardinality(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1),
+            TestUtils.getMockApplicationId(1).toString()));
+
+    // Get Cardinality of app1 on node2, with empty tag set, op=max
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
+
+    // Get Cardinality of all apps on node2, with empty tag set, op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"), null,
+            ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::sum));
+
+    // Get Cardinality of app_1 on node2, with empty tag set, op=sum
+    Assert.assertEquals(0,
+        atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+            TestUtils.getMockApplicationId(2), ImmutableSet.of(), Long::sum));
+  }
+
+  @Test
+  public void testAllocationTagsManagerMemoryAfterCleanup()
+      throws InvalidAllocationTagsQueryException {
+    /**
+     * Make sure YARN cleans up all memory once container/app finishes.
+     */
+
+    AllocationTagsManager atm = new AllocationTagsManager();
+
+    // Add a bunch of containers
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Remove all these containers
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.removeContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.removeContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // Check internal data structure
+    Assert.assertEquals(0,
+        atm.getGlobalMapping().getNodeToTagsWithCount().size());
+    Assert.assertEquals(0, atm.getPerAppMappings().size());
+  }
+
+  @Test
+  public void testQueryCardinalityWithIllegalParameters()
+      throws InvalidAllocationTagsQueryException {
+    /**
+     * Make sure YARN cleans up all memory once container/app finishes.
+     */
+
+    AllocationTagsManager atm = new AllocationTagsManager();
+
+    // Add a bunch of containers
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 1),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 2),
+        ImmutableSet.of("mapper", "reducer"));
+
+    atm.addContainer(NodeId.fromString("node1:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 3),
+        ImmutableSet.of("service"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(1), TestUtils.getMockContainerId(1, 4),
+        ImmutableSet.of("reducer"));
+
+    atm.addContainer(NodeId.fromString("node2:1234"),
+        TestUtils.getMockApplicationId(2), TestUtils.getMockContainerId(2, 3),
+        ImmutableSet.of("service"));
+
+    // No node-id
+    boolean caughtException = false;
+    try {
+      atm.getNodeCardinalityByOp(null, TestUtils.getMockApplicationId(2),
+          ImmutableSet.of("mapper"), Long::min);
+    } catch (InvalidAllocationTagsQueryException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue("should fail because of nodeId specified",
+        caughtException);
+
+    // No op
+    caughtException = false;
+    try {
+      atm.getNodeCardinalityByOp(NodeId.fromString("node2:1234"),
+          TestUtils.getMockApplicationId(2), ImmutableSet.of("mapper"), null);
+    } catch (InvalidAllocationTagsQueryException e) {
+      caughtException = true;
+    }
+    Assert.assertTrue("should fail because of nodeId specified",
+        caughtException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index 6c189b3..27ff311 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -109,6 +110,8 @@ public class TestRMContainerImpl {
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getRMApps()).thenReturn(rmApps);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    when(rmContext.getAllocationTagsManager()).thenReturn(ptm);
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(
         YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
@@ -209,6 +212,8 @@ public class TestRMContainerImpl {
     when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    when(rmContext.getAllocationTagsManager()).thenReturn(ptm);
 
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(
@@ -367,4 +372,123 @@ public class TestRMContainerImpl {
     verify(publisher, times(1)).containerCreated(any(RMContainer.class), anyLong());
     verify(publisher, times(1)).containerFinished(any(RMContainer.class), anyLong());
   }
+
+  @Test
+  public void testContainerTransitionNotifyPlacementTagsManager()
+      throws Exception {
+    DrainDispatcher drainDispatcher = new DrainDispatcher();
+    EventHandler<RMAppAttemptEvent> appAttemptEventHandler = mock(
+        EventHandler.class);
+    EventHandler generic = mock(EventHandler.class);
+    drainDispatcher.register(RMAppAttemptEventType.class,
+        appAttemptEventHandler);
+    drainDispatcher.register(RMNodeEventType.class, generic);
+    drainDispatcher.init(new YarnConfiguration());
+    drainDispatcher.start();
+    NodeId nodeId = BuilderUtils.newNodeId("host", 3425);
+    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 1);
+    ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
+    ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
+
+    Resource resource = BuilderUtils.newResource(512, 1);
+    Priority priority = BuilderUtils.newPriority(5);
+
+    Container container = BuilderUtils.newContainer(containerId, nodeId,
+        "host:3465", resource, priority, null);
+    ConcurrentMap<ApplicationId, RMApp> rmApps =
+        spy(new ConcurrentHashMap<ApplicationId, RMApp>());
+    RMApp rmApp = mock(RMApp.class);
+    when(rmApp.getRMAppAttempt(Matchers.any())).thenReturn(null);
+    Mockito.doReturn(rmApp).when(rmApps).get(Matchers.any());
+
+    RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
+    SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
+    AllocationTagsManager tagsManager = new AllocationTagsManager();
+    RMContext rmContext = mock(RMContext.class);
+    when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
+    when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer);
+    when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
+    when(rmContext.getRMApps()).thenReturn(rmApps);
+    when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getAllocationTagsManager()).thenReturn(tagsManager);
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(
+        YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+        true);
+    when(rmContext.getYarnConfiguration()).thenReturn(conf);
+
+    /* First container: ALLOCATED -> KILLED */
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
+        nodeId, "user", rmContext);
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+
+    Assert.assertEquals(1,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerFinishedEvent(containerId, ContainerStatus
+        .newInstance(containerId, ContainerState.COMPLETE, "", 0),
+        RMContainerEventType.KILL));
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    /* Second container: ACQUIRED -> FINISHED */
+    rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
+        nodeId, "user", rmContext);
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+
+    Assert.assertEquals(1,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(
+        new RMContainerEvent(containerId, RMContainerEventType.ACQUIRED));
+
+    rmContainer.handle(new RMContainerFinishedEvent(containerId, ContainerStatus
+        .newInstance(containerId, ContainerState.COMPLETE, "", 0),
+        RMContainerEventType.FINISHED));
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    /* Third container: RUNNING -> FINISHED */
+    rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
+        nodeId, "user", rmContext);
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(new RMContainerEvent(containerId,
+        RMContainerEventType.START));
+
+    Assert.assertEquals(1,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+
+    rmContainer.handle(
+        new RMContainerEvent(containerId, RMContainerEventType.ACQUIRED));
+
+    rmContainer.handle(
+        new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+
+    rmContainer.handle(new RMContainerFinishedEvent(containerId, ContainerStatus
+        .newInstance(containerId, ContainerState.COMPLETE, "", 0),
+        RMContainerEventType.FINISHED));
+
+    Assert.assertEquals(0,
+        tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.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/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index e3326c7..61a5555 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -135,6 +136,9 @@ public class TestUtils {
         new DefaultResourceCalculator());
     rmContext.setScheduler(mockScheduler);
 
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    rmContext.setAllocationTagsManager(ptm);
+
     return rmContext;
   }
   
@@ -234,6 +238,11 @@ public class TestUtils {
     doReturn(id).when(containerId).getContainerId();
     return containerId;
   }
+
+  public static ContainerId getMockContainerId(int appId, int containerId) {
+    ApplicationAttemptId attemptId = getMockApplicationAttemptId(appId, 1);
+    return ContainerId.newContainerId(attemptId, containerId);
+  }
   
   public static Container getMockContainer(
       ContainerId containerId, NodeId nodeId, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2a8ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.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/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 3f97b59..4b902a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -234,6 +235,8 @@ public class TestFifoScheduler {
     FifoScheduler scheduler = new FifoScheduler();
     RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
         null, containerTokenSecretManager, nmTokenSecretManager, null, scheduler);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
+    rmContext.setAllocationTagsManager(ptm);
     rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
     rmContext.setRMApplicationHistoryWriter(
         mock(RMApplicationHistoryWriter.class));
@@ -312,12 +315,14 @@ public class TestFifoScheduler {
     FifoScheduler scheduler = new FifoScheduler();
     RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
         null, containerTokenSecretManager, nmTokenSecretManager, null, scheduler);
+    AllocationTagsManager ptm = mock(AllocationTagsManager.class);
     rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
     rmContext.setRMApplicationHistoryWriter(mock(RMApplicationHistoryWriter.class));
     ((RMContextImpl) rmContext).setYarnConfiguration(new YarnConfiguration());
     NullRMNodeLabelsManager nlm = new NullRMNodeLabelsManager();
     nlm.init(new Configuration());
     rmContext.setNodeLabelManager(nlm);
+    rmContext.setAllocationTagsManager(ptm);
 
     scheduler.setRMContext(rmContext);
     ((RMContextImpl) rmContext).setScheduler(scheduler);


---------------------------------------------------------------------
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: HDFS-12777. [READ] Reduce memory and CPU footprint for PROVIDED volumes.

Posted by kk...@apache.org.
HDFS-12777. [READ] Reduce memory and CPU footprint for PROVIDED volumes.


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

Branch: refs/heads/YARN-6592
Commit: e1a28f95b8ffcb86300148f10a23b710f8388341
Parents: 6cd80b2
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Nov 10 10:19:33 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/DirectoryScanner.java  |  4 +
 .../datanode/FinalizedProvidedReplica.java      |  8 ++
 .../hdfs/server/datanode/ProvidedReplica.java   | 77 +++++++++++++++++++-
 .../hdfs/server/datanode/ReplicaBuilder.java    | 37 +++++++++-
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 30 +++++++-
 .../fsdataset/impl/TestProvidedImpl.java        | 76 ++++++++++++-------
 6 files changed, 196 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1a28f95/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 3b6d06c..8fb8551 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
@@ -530,6 +530,10 @@ public class DirectoryScanner implements Runnable {
           new HashMap<Integer, Future<ScanInfoPerBlockPool>>();
 
       for (int i = 0; i < volumes.size(); i++) {
+        if (volumes.get(i).getStorageType() == StorageType.PROVIDED) {
+          // Disable scanning PROVIDED volumes to keep overhead low
+          continue;
+        }
         ReportCompiler reportCompiler =
             new ReportCompiler(datanode, volumes.get(i));
         Future<ScanInfoPerBlockPool> result =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1a28f95/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 e23d6be..bcc9a38 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
@@ -21,6 +21,7 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -37,6 +38,13 @@ public class FinalizedProvidedReplica extends ProvidedReplica {
         remoteFS);
   }
 
+  public FinalizedProvidedReplica(long blockId, Path pathPrefix,
+      String pathSuffix, long fileOffset, long blockLen, long genStamp,
+      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
+    super(blockId, pathPrefix, pathSuffix, fileOffset, blockLen,
+        genStamp, volume, conf, remoteFS);
+  }
+
   @Override
   public ReplicaState getState() {
     return ReplicaState.FINALIZED;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1a28f95/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 2b3bd13..8681421 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
@@ -23,6 +23,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -51,18 +52,23 @@ public abstract class ProvidedReplica extends ReplicaInfo {
   static final byte[] NULL_CHECKSUM_ARRAY =
       FsDatasetUtil.createNullChecksumByteArray();
   private URI fileURI;
+  private Path pathPrefix;
+  private String pathSuffix;
   private long fileOffset;
   private Configuration conf;
   private FileSystem remoteFS;
 
   /**
    * Constructor.
+   *
    * @param blockId block id
    * @param fileURI remote URI this block is to be read from
    * @param fileOffset the offset in the remote URI
    * @param blockLen the length of the block
    * @param genStamp the generation stamp of the block
    * @param volume the volume this block belongs to
+   * @param conf the configuration
+   * @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,
@@ -85,23 +91,86 @@ public abstract class ProvidedReplica extends ReplicaInfo {
     }
   }
 
+  /**
+   * Constructor.
+   *
+   * @param blockId block id
+   * @param pathPrefix A prefix of the {@link Path} associated with this replica
+   *          on the remote {@link FileSystem}.
+   * @param pathSuffix A suffix of the {@link Path} associated with this replica
+   *          on the remote {@link FileSystem}. Resolving the {@code pathSuffix}
+   *          against the {@code pathPrefix} should provide the exact
+   *          {@link Path} of the data associated with this replica on the
+   *          remote {@link FileSystem}.
+   * @param fileOffset the offset in the remote URI
+   * @param blockLen the length of the block
+   * @param genStamp the generation stamp of the block
+   * @param volume the volume this block belongs to
+   * @param conf the configuration
+   * @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) {
+    super(volume, blockId, blockLen, genStamp);
+    this.fileURI = null;
+    this.pathPrefix = pathPrefix;
+    this.pathSuffix = pathSuffix;
+    this.fileOffset = fileOffset;
+    this.conf = conf;
+    if (remoteFS != null) {
+      this.remoteFS = remoteFS;
+    } else {
+      LOG.warn(
+          "Creating an reference to the remote FS for provided block " + this);
+      try {
+        this.remoteFS = FileSystem.get(pathPrefix.toUri(), this.conf);
+      } catch (IOException e) {
+        LOG.warn("Failed to obtain filesystem for " + pathPrefix);
+        this.remoteFS = null;
+      }
+    }
+  }
+
   public ProvidedReplica(ProvidedReplica r) {
     super(r);
     this.fileURI = r.fileURI;
     this.fileOffset = r.fileOffset;
     this.conf = r.conf;
     this.remoteFS = r.remoteFS;
+    this.pathPrefix = r.pathPrefix;
+    this.pathSuffix = r.pathSuffix;
   }
 
   @Override
   public URI getBlockURI() {
-    return this.fileURI;
+    return getRemoteURI();
+  }
+
+  @VisibleForTesting
+  public String getPathSuffix() {
+    return pathSuffix;
+  }
+
+  @VisibleForTesting
+  public Path getPathPrefix() {
+    return pathPrefix;
+  }
+
+  private URI getRemoteURI() {
+    if (fileURI != null) {
+      return fileURI;
+    } else if (pathPrefix == null) {
+      return new Path(pathSuffix).toUri();
+    } else {
+      return new Path(pathPrefix, pathSuffix).toUri();
+    }
   }
 
   @Override
   public InputStream getDataInputStream(long seekOffset) throws IOException {
     if (remoteFS != null) {
-      FSDataInputStream ins = remoteFS.open(new Path(fileURI));
+      FSDataInputStream ins = remoteFS.open(new Path(getRemoteURI()));
       ins.seek(fileOffset + seekOffset);
       return new BoundedInputStream(
           new FSDataInputStream(ins), getBlockDataLength());
@@ -132,7 +201,7 @@ public abstract class ProvidedReplica extends ReplicaInfo {
   public boolean blockDataExists() {
     if(remoteFS != null) {
       try {
-        return remoteFS.exists(new Path(fileURI));
+        return remoteFS.exists(new Path(getRemoteURI()));
       } catch (IOException e) {
         return false;
       }
@@ -220,7 +289,7 @@ public abstract class ProvidedReplica extends ReplicaInfo {
   public int compareWith(ScanInfo info) {
     //local scanning cannot find any provided blocks.
     if (info.getFileRegion().equals(
-        new FileRegion(this.getBlockId(), new Path(fileURI),
+        new FileRegion(this.getBlockId(), new Path(getRemoteURI()),
             fileOffset, this.getNumBytes(), this.getGenerationStamp()))) {
       return 0;
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1a28f95/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 c5cb6a5..de68e2d 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
@@ -21,6 +21,7 @@ import java.io.File;
 import java.net.URI;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
@@ -52,6 +53,8 @@ public class ReplicaBuilder {
   private Configuration conf;
   private FileRegion fileRegion;
   private FileSystem remoteFS;
+  private String pathSuffix;
+  private Path pathPrefix;
 
   public ReplicaBuilder(ReplicaState state) {
     volume = null;
@@ -145,6 +148,28 @@ public class ReplicaBuilder {
     return this;
   }
 
+  /**
+   * Set the suffix of the {@link Path} associated with the replica.
+   * Intended to be use only for {@link ProvidedReplica}s.
+   * @param suffix the path suffix.
+   * @return the builder with the path suffix set.
+   */
+  public ReplicaBuilder setPathSuffix(String suffix) {
+    this.pathSuffix = suffix;
+    return this;
+  }
+
+  /**
+   * Set the prefix of the {@link Path} associated with the replica.
+   * Intended to be use only for {@link ProvidedReplica}s.
+   * @param prefix the path prefix.
+   * @return the builder with the path prefix set.
+   */
+  public ReplicaBuilder setPathPrefix(Path prefix) {
+    this.pathPrefix = prefix;
+    return this;
+  }
+
   public LocalReplicaInPipeline buildLocalReplicaInPipeline()
       throws IllegalArgumentException {
     LocalReplicaInPipeline info = null;
@@ -275,14 +300,20 @@ public class ReplicaBuilder {
       throw new IllegalArgumentException("Finalized PROVIDED replica " +
           "cannot be constructed from another replica");
     }
-    if (fileRegion == null && uri == null) {
+    if (fileRegion == null && uri == null &&
+        (pathPrefix == null || pathSuffix == null)) {
       throw new IllegalArgumentException(
           "Trying to construct a provided replica on " + volume +
           " without enough information");
     }
     if (fileRegion == null) {
-      info = new FinalizedProvidedReplica(blockId, uri, offset,
-          length, genStamp, volume, conf, remoteFS);
+      if (uri != null) {
+        info = new FinalizedProvidedReplica(blockId, uri, offset,
+            length, genStamp, volume, conf, remoteFS);
+      } else {
+        info = new FinalizedProvidedReplica(blockId, pathPrefix, pathSuffix,
+            offset, length, genStamp, volume, conf, remoteFS);
+      }
     } else {
       info = new FinalizedProvidedReplica(fileRegion.getBlock().getBlockId(),
           fileRegion.getPath().toUri(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1a28f95/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 092672d..d103b64 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
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -65,6 +66,29 @@ import org.apache.hadoop.util.Time;
  */
 public class ProvidedVolumeImpl extends FsVolumeImpl {
 
+  /**
+   * Get a suffix of the full path, excluding the given prefix.
+   *
+   * @param prefix a prefix of the path.
+   * @param fullPath the full path whose suffix is needed.
+   * @return the suffix of the path, which when resolved against {@code prefix}
+   *         gets back the {@code fullPath}.
+   */
+  @VisibleForTesting
+  protected static String getSuffix(final Path prefix, final Path fullPath) {
+    String prefixStr = prefix.toString();
+    String pathStr = fullPath.toString();
+    if (!pathStr.startsWith(prefixStr)) {
+      LOG.debug("Path {} is not a prefix of the path {}", prefix, fullPath);
+      return pathStr;
+    }
+    String suffix = pathStr.replaceFirst("^" + prefixStr, "");
+    if (suffix.startsWith("/")) {
+      suffix = suffix.substring(1);
+    }
+    return suffix;
+  }
+
   static class ProvidedBlockPoolSlice {
     private ProvidedVolumeImpl providedVolume;
 
@@ -106,15 +130,19 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
         return;
       }
       Iterator<FileRegion> iter = reader.iterator();
+      Path blockPrefixPath = new Path(providedVolume.getBaseURI());
       while (iter.hasNext()) {
         FileRegion region = iter.next();
         if (region.getBlockPoolId() != null
             && region.getBlockPoolId().equals(bpid)
             && containsBlock(providedVolume.baseURI,
                 region.getPath().toUri())) {
+          String blockSuffix =
+              getSuffix(blockPrefixPath, new Path(region.getPath().toUri()));
           ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
               .setBlockId(region.getBlock().getBlockId())
-              .setURI(region.getPath().toUri())
+              .setPathPrefix(blockPrefixPath)
+              .setPathSuffix(blockSuffix)
               .setOffset(region.getOffset())
               .setLength(region.getBlock().getNumBytes())
               .setGenerationStamp(region.getBlock().getGenerationStamp())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1a28f95/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 40d77f7a..ecab06b 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
@@ -62,7 +62,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;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@@ -509,33 +509,6 @@ public class TestProvidedImpl {
     }
   }
 
-  @Test
-  public void testRefresh() throws IOException {
-    conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
-    for (int i = 0; i < providedVolumes.size(); i++) {
-      ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
-      TestFileRegionBlockAliasMap testBlockFormat =
-          (TestFileRegionBlockAliasMap) vol
-              .getBlockFormat(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
-      //equivalent to two new blocks appearing
-      testBlockFormat.setBlockCount(NUM_PROVIDED_BLKS + 2);
-      //equivalent to deleting the first block
-      testBlockFormat.setMinBlkId(MIN_BLK_ID + 1);
-
-      DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf);
-      scanner.reconcile();
-      ReplicaInfo info = dataset.getBlockReplica(
-          BLOCK_POOL_IDS[CHOSEN_BP_ID], NUM_PROVIDED_BLKS + 1);
-      //new replica should be added to the dataset
-      assertTrue(info != null);
-      try {
-        info = dataset.getBlockReplica(BLOCK_POOL_IDS[CHOSEN_BP_ID], 0);
-      } catch(Exception ex) {
-        LOG.info("Exception expected: " + ex);
-      }
-    }
-  }
-
   private int getBlocksInProvidedVolumes(String basePath, int numBlocks,
       int minBlockId) throws IOException {
     TestFileRegionIterator fileRegionIterator =
@@ -621,4 +594,51 @@ public class TestProvidedImpl {
         ProvidedVolumeImpl.containsBlock(new URI("/bucket1/dir1/"),
             new URI("s3a:/bucket1/dir1/temp.txt")));
   }
+
+  @Test
+  public void testProvidedReplicaSuffixExtraction() {
+    assertEquals("B.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/"), new Path("file:///A/B.txt")));
+    assertEquals("B/C.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/"), new Path("file:///A/B/C.txt")));
+    assertEquals("B/C/D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/"), new Path("file:///A/B/C/D.txt")));
+    assertEquals("D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/B/C/"), new Path("file:///A/B/C/D.txt")));
+    assertEquals("file:/A/B/C/D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///X/B/C/"), new Path("file:///A/B/C/D.txt")));
+    assertEquals("D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("/A/B/C"), new Path("/A/B/C/D.txt")));
+    assertEquals("D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("/A/B/C/"), new Path("/A/B/C/D.txt")));
+
+    assertEquals("data/current.csv", ProvidedVolumeImpl.getSuffix(
+        new Path("wasb:///users/alice/"),
+        new Path("wasb:///users/alice/data/current.csv")));
+    assertEquals("current.csv", ProvidedVolumeImpl.getSuffix(
+        new Path("wasb:///users/alice/data"),
+        new Path("wasb:///users/alice/data/current.csv")));
+
+    assertEquals("wasb:/users/alice/data/current.csv",
+        ProvidedVolumeImpl.getSuffix(
+            new Path("wasb:///users/bob/"),
+            new Path("wasb:///users/alice/data/current.csv")));
+  }
+
+  @Test
+  public void testProvidedReplicaPrefix() throws Exception {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      FsVolumeImpl vol = providedVolumes.get(i);
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(volumeMap, null);
+
+      Path expectedPrefix = new Path(
+          StorageLocation.normalizeFileURI(new File(providedBasePath).toURI()));
+      for (ReplicaInfo info : volumeMap
+          .replicas(BLOCK_POOL_IDS[CHOSEN_BP_ID])) {
+        ProvidedReplica pInfo = (ProvidedReplica) info;
+        assertEquals(expectedPrefix, pInfo.getPathPrefix());
+      }
+    }
+  }
 }


---------------------------------------------------------------------
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: YARN-6594. [API] Introduce SchedulingRequest object. (Konstantinos Karanasos via wangda)

Posted by kk...@apache.org.
YARN-6594. [API] Introduce SchedulingRequest object. (Konstantinos Karanasos via wangda)


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

Branch: refs/heads/YARN-6592
Commit: 16cbed8998648439e16fa2c40decadffc0679d98
Parents: 45b1ca6
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Oct 30 16:54:02 2017 -0700
Committer: Konstantinos Karanasos <kk...@apache.org>
Committed: Mon Dec 18 16:07:00 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/ResourceSizing.java |  64 +++++
 .../yarn/api/records/SchedulingRequest.java     | 205 ++++++++++++++
 .../src/main/proto/yarn_protos.proto            |  14 +
 .../records/impl/pb/ResourceSizingPBImpl.java   | 117 ++++++++
 .../impl/pb/SchedulingRequestPBImpl.java        | 266 +++++++++++++++++++
 5 files changed, 666 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/16cbed89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
new file mode 100644
index 0000000..d82be11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
@@ -0,0 +1,64 @@
+/**
+ * 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;
+
+/**
+ * {@code ResourceSizing} contains information for the size of a
+ * {@link SchedulingRequest}, such as the number of requested allocations and
+ * the resources for each allocation.
+ */
+@Public
+@Unstable
+public abstract class ResourceSizing {
+
+  @Public
+  @Unstable
+  public static ResourceSizing newInstance(Resource resources) {
+    return ResourceSizing.newInstance(1, resources);
+  }
+
+  @Public
+  @Unstable
+  public static ResourceSizing newInstance(int numAllocations, Resource resources) {
+    ResourceSizing resourceSizing = Records.newRecord(ResourceSizing.class);
+    resourceSizing.setNumAllocations(numAllocations);
+    resourceSizing.setResources(resources);
+    return resourceSizing;
+  }
+
+  @Public
+  @Unstable
+  public abstract int getNumAllocations();
+
+  @Public
+  @Unstable
+  public abstract void setNumAllocations(int numAllocations);
+
+  @Public
+  @Unstable
+  public abstract Resource getResources();
+
+  @Public
+  @Unstable
+  public abstract void setResources(Resource resources);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16cbed89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
new file mode 100644
index 0000000..47a0697
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
@@ -0,0 +1,205 @@
+/**
+ * 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 java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@code SchedulingRequest} represents a request made by an application to the
+ * {@code ResourceManager} to obtain an allocation. It is similar to the
+ * {@link ResourceRequest}. However, it is more complete than the latter, as it
+ * allows applications to specify allocation tags (e.g., to express that an
+ * allocation belongs to {@code Spark} or is an {@code HBase-master}), as well
+ * as involved {@link PlacementConstraint}s (e.g., anti-affinity between Spark
+ * and HBase allocations).
+ *
+ * The size specification of the allocation is in {@code ResourceSizing}.
+ */
+@Public
+@Unstable
+public abstract class SchedulingRequest {
+
+  @Public
+  @Unstable
+  public static SchedulingRequest newInstance(long allocationRequestId,
+      Priority priority, ExecutionTypeRequest executionType,
+      Set<String> allocationTags, ResourceSizing resourceSizing,
+      PlacementConstraint placementConstraintExpression) {
+    return SchedulingRequest.newBuilder()
+        .allocationRequestId(allocationRequestId).priority(priority)
+        .executionType(executionType).allocationTags(allocationTags)
+        .placementConstraintExpression(placementConstraintExpression).build();
+  }
+
+  @Public
+  @Unstable
+  public static SchedulingRequestBuilder newBuilder() {
+    return new SchedulingRequestBuilder();
+  }
+
+  /**
+   * Class to construct instances of {@link SchedulingRequest} with specific
+   * options.
+   */
+  @Public
+  @Unstable
+  public static final class SchedulingRequestBuilder {
+    private SchedulingRequest schedulingRequest =
+            Records.newRecord(SchedulingRequest.class);
+
+    private SchedulingRequestBuilder() {
+      schedulingRequest.setAllocationRequestId(0);
+      schedulingRequest.setPriority(Priority.newInstance(0));
+      schedulingRequest.setExecutionType(ExecutionTypeRequest.newInstance());
+    }
+
+    /**
+     * Set the <code>allocationRequestId</code> of the request.
+     * 
+     * @see SchedulingRequest#setAllocationRequestId(long)
+     * @param allocationRequestId <code>allocationRequestId</code> of the
+     *          request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder allocationRequestId(
+            long allocationRequestId) {
+      schedulingRequest.setAllocationRequestId(allocationRequestId);
+      return this;
+    }
+
+    /**
+     * Set the <code>priority</code> of the request.
+     *
+     * @param priority <code>priority</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     * @see SchedulingRequest#setPriority(Priority)
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder priority(Priority priority) {
+      schedulingRequest.setPriority(priority);
+      return this;
+    }
+
+    /**
+     * Set the <code>executionType</code> of the request.
+     * 
+     * @see SchedulingRequest#setExecutionType(ExecutionTypeRequest)
+     * @param executionType <code>executionType</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder executionType(
+        ExecutionTypeRequest executionType) {
+      schedulingRequest.setExecutionType(executionType);
+      return this;
+    }
+    
+    /**
+     * Set the <code>allocationTags</code> of the request.
+     *
+     * @see SchedulingRequest#setAllocationTags(Set)
+     * @param allocationTags <code>allocationsTags</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder allocationTags(Set<String> allocationTags) {
+      schedulingRequest.setAllocationTags(allocationTags);
+      return this;
+    }
+
+    /**
+     * Set the <code>executionType</code> of the request.
+     *
+     * @see SchedulingRequest#setResourceSizing(ResourceSizing)
+     * @param resourceSizing <code>resourceSizing</code> of the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder resourceSizing(
+        ResourceSizing resourceSizing) {
+      schedulingRequest.setResourceSizing(resourceSizing);
+      return this;
+    }
+
+    /**
+     * Set the <code>placementConstraintExpression</code> of the request.
+     *
+     * @see SchedulingRequest#setPlacementConstraint(
+     *      PlacementConstraint)
+     * @param placementConstraintExpression <code>placementConstraints</code> of
+     *          the request
+     * @return {@link SchedulingRequest.SchedulingRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequestBuilder placementConstraintExpression(
+        PlacementConstraint placementConstraintExpression) {
+      schedulingRequest
+          .setPlacementConstraint(placementConstraintExpression);
+      return this;
+    }
+
+    /**
+     * Return generated {@link SchedulingRequest} object.
+     * 
+     * @return {@link SchedulingRequest}
+     */
+    @Public
+    @Unstable
+    public SchedulingRequest build() {
+      return schedulingRequest;
+    }
+  }
+
+  public abstract long getAllocationRequestId();
+
+  public abstract void setAllocationRequestId(long allocationRequestId);
+
+  public abstract Priority getPriority();
+
+  public abstract void setPriority(Priority priority);
+
+  public abstract ExecutionTypeRequest getExecutionType();
+
+  public abstract void setExecutionType(ExecutionTypeRequest executionType);
+
+  public abstract Set<String> getAllocationTags();
+
+  public abstract void setAllocationTags(Set<String> allocationTags);
+
+  public abstract ResourceSizing getResourceSizing();
+
+  public abstract void setResourceSizing(ResourceSizing resourceSizing);
+
+  public abstract PlacementConstraint getPlacementConstraint();
+
+  public abstract void setPlacementConstraint(
+      PlacementConstraint placementConstraint);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16cbed89/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 968b75e..2dbdefb 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
@@ -405,6 +405,20 @@ message ExecutionTypeRequestProto {
   optional bool enforce_execution_type = 2 [default = false];
 }
 
+message SchedulingRequestProto {
+  optional int64 allocationRequestId = 1 [default = 0];
+  optional PriorityProto priority = 2;
+  optional ExecutionTypeRequestProto executionType = 3;
+  repeated string allocationTags = 4;
+  optional ResourceSizingProto resourceSizing = 5;
+  optional PlacementConstraintProto placementConstraint = 6;
+}
+
+message ResourceSizingProto {
+  optional int32 numAllocations = 1;
+  optional ResourceProto resources = 2;
+}
+
 enum AMCommandProto {
   AM_RESYNC = 1;
   AM_SHUTDOWN = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16cbed89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
new file mode 100644
index 0000000..05bb3bd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
@@ -0,0 +1,117 @@
+/**
+ * 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.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProtoOrBuilder;
+
+@Private
+@Unstable
+public class ResourceSizingPBImpl extends ResourceSizing {
+  ResourceSizingProto proto = ResourceSizingProto.getDefaultInstance();
+  ResourceSizingProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Resource resources = null;
+
+  public ResourceSizingPBImpl() {
+    builder = ResourceSizingProto.newBuilder();
+  }
+
+  public ResourceSizingPBImpl(ResourceSizingProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ResourceSizingProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.resources != null) {
+      builder.setResources(convertToProtoFormat(this.resources));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ResourceSizingProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int getNumAllocations() {
+    ResourceSizingProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getNumAllocations());
+  }
+
+  @Override
+  public void setNumAllocations(int numAllocations) {
+    maybeInitBuilder();
+    builder.setNumAllocations(numAllocations);
+  }
+
+  @Override
+  public Resource getResources() {
+    ResourceSizingProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.resources != null) {
+      return this.resources;
+    }
+    if (!p.hasResources()) {
+      return null;
+    }
+    this.resources = convertFromProtoFormat(p.getResources());
+    return this.resources;
+  }
+
+  @Override
+  public void setResources(Resource resources) {
+    maybeInitBuilder();
+    if (resources == null) {
+      builder.clearResources();
+    }
+    this.resources = resources;
+  }
+
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto r) {
+    return new ResourcePBImpl(r);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource r) {
+    return ((ResourcePBImpl) r).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16cbed89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
new file mode 100644
index 0000000..7826b36
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
@@ -0,0 +1,266 @@
+/**
+ * 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 java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProtoOrBuilder;
+
+@Private
+@Unstable
+public class SchedulingRequestPBImpl extends SchedulingRequest {
+  SchedulingRequestProto proto = SchedulingRequestProto.getDefaultInstance();
+  SchedulingRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Priority priority = null;
+  private ExecutionTypeRequest executionType = null;
+  private Set<String> allocationTags = null;
+  private ResourceSizing resourceSizing = null;
+  private PlacementConstraint placementConstraint = null;
+
+  public SchedulingRequestPBImpl() {
+    builder = SchedulingRequestProto.newBuilder();
+  }
+
+  public SchedulingRequestPBImpl(SchedulingRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SchedulingRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.priority != null) {
+      builder.setPriority(convertToProtoFormat(this.priority));
+    }
+    if (this.executionType != null) {
+      builder.setExecutionType(convertToProtoFormat(this.executionType));
+    }
+    if (this.allocationTags != null) {
+      builder.clearAllocationTags();
+      builder.addAllAllocationTags(this.allocationTags);
+    }
+    if (this.resourceSizing != null) {
+      builder.setResourceSizing(convertToProtoFormat(this.resourceSizing));
+    }
+    if (this.placementConstraint != null) {
+      builder.setPlacementConstraint(
+          convertToProtoFormat(this.placementConstraint));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SchedulingRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public long getAllocationRequestId() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getAllocationRequestId());
+  }
+
+  @Override
+  public void setAllocationRequestId(long allocationRequestId) {
+    maybeInitBuilder();
+    builder.setAllocationRequestId(allocationRequestId);
+  }
+
+  @Override
+  public Priority getPriority() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.priority != null) {
+      return this.priority;
+    }
+    if (!p.hasPriority()) {
+      return null;
+    }
+    this.priority = convertFromProtoFormat(p.getPriority());
+    return this.priority;
+  }
+
+  @Override
+  public void setPriority(Priority priority) {
+    maybeInitBuilder();
+    if (priority == null) {
+      builder.clearPriority();
+    }
+    this.priority = priority;
+  }
+
+  @Override
+  public ExecutionTypeRequest getExecutionType() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.executionType != null) {
+      return this.executionType;
+    }
+    if (!p.hasExecutionType()) {
+      return null;
+    }
+    this.executionType = convertFromProtoFormat(p.getExecutionType());
+    return this.executionType;
+  }
+
+  @Override
+  public void setExecutionType(ExecutionTypeRequest executionType) {
+    maybeInitBuilder();
+    if (executionType == null) {
+      builder.clearExecutionType();
+    }
+    this.executionType = executionType;
+  }
+
+  @Override
+  public Set<String> getAllocationTags() {
+    initAllocationTags();
+    return this.allocationTags;
+  }
+
+  @Override
+  public void setAllocationTags(Set<String> allocationTags) {
+    maybeInitBuilder();
+    builder.clearAllocationTags();
+    this.allocationTags = allocationTags;
+  }
+
+  @Override
+  public ResourceSizing getResourceSizing() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.resourceSizing != null) {
+      return this.resourceSizing;
+    }
+    if (!p.hasResourceSizing()) {
+      return null;
+    }
+    this.resourceSizing = convertFromProtoFormat(p.getResourceSizing());
+    return this.resourceSizing;
+  }
+
+  @Override
+  public void setResourceSizing(ResourceSizing resourceSizing) {
+    maybeInitBuilder();
+    if (resourceSizing == null) {
+      builder.clearResourceSizing();
+    }
+    this.resourceSizing = resourceSizing;
+  }
+
+  @Override
+  public PlacementConstraint getPlacementConstraint() {
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.placementConstraint != null) {
+      return this.placementConstraint;
+    }
+    if (!p.hasPlacementConstraint()) {
+      return null;
+    }
+    this.placementConstraint =
+        convertFromProtoFormat(p.getPlacementConstraint());
+    return this.placementConstraint;
+  }
+
+  @Override
+  public void setPlacementConstraint(PlacementConstraint placementConstraint) {
+    maybeInitBuilder();
+    if (placementConstraint == null) {
+      builder.clearPlacementConstraint();
+    }
+    this.placementConstraint = placementConstraint;
+  }
+
+  private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
+    return new PriorityPBImpl(p);
+  }
+
+  private PriorityProto convertToProtoFormat(Priority p) {
+    return ((PriorityPBImpl) p).getProto();
+  }
+
+  private ExecutionTypeRequestPBImpl convertFromProtoFormat(
+      ExecutionTypeRequestProto p) {
+    return new ExecutionTypeRequestPBImpl(p);
+  }
+
+  private ExecutionTypeRequestProto convertToProtoFormat(
+      ExecutionTypeRequest p) {
+    return ((ExecutionTypeRequestPBImpl) p).getProto();
+  }
+
+  private ResourceSizingPBImpl convertFromProtoFormat(ResourceSizingProto p) {
+    return new ResourceSizingPBImpl(p);
+  }
+
+  private ResourceSizingProto convertToProtoFormat(ResourceSizing p) {
+    return ((ResourceSizingPBImpl) p).getProto();
+  }
+
+  private PlacementConstraint convertFromProtoFormat(
+      PlacementConstraintProto c) {
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(c);
+    return fromProtoConverter.convert();
+  }
+
+  private PlacementConstraintProto convertToProtoFormat(PlacementConstraint c) {
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(c);
+    return toProtoConverter.convert();
+  }
+
+  private void initAllocationTags() {
+    if (this.allocationTags != null) {
+      return;
+    }
+    SchedulingRequestProtoOrBuilder p = viaProto ? proto : builder;
+    this.allocationTags = new HashSet<>();
+    this.allocationTags.addAll(p.getAllocationTagsList());
+  }
+}


---------------------------------------------------------------------
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-12874. Documentation for provided storage. Contributed by Virajith Jalaparti

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


[15/50] [abbrv] hadoop git commit: HDFS-12607. [READ] Even one dead datanode with PROVIDED storage results in ProvidedStorageInfo being marked as FAILED

Posted by kk...@apache.org.
HDFS-12607. [READ] Even one dead datanode with PROVIDED storage results in ProvidedStorageInfo being marked as FAILED


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

Branch: refs/heads/YARN-6592
Commit: 71d0a825711387fe06396323a9ca6a5af0ade415
Parents: 98f5ed5
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Nov 6 11:05:59 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../blockmanagement/DatanodeDescriptor.java     |  6 ++-
 .../blockmanagement/ProvidedStorageMap.java     | 40 +++++++++++++-------
 .../TestNameNodeProvidedImplementation.java     | 40 ++++++++++++++++++++
 3 files changed, 71 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71d0a825/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index e3d6582..c17ab4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -455,8 +455,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
       totalDfsUsed += report.getDfsUsed();
       totalNonDfsUsed += report.getNonDfsUsed();
 
-      if (StorageType.PROVIDED.equals(
-          report.getStorage().getStorageType())) {
+      // for PROVIDED storages, do not call updateStorage() unless
+      // DatanodeStorageInfo already exists!
+      if (StorageType.PROVIDED.equals(report.getStorage().getStorageType())
+          && storageMap.get(report.getStorage().getStorageID()) == null) {
         continue;
       }
       DatanodeStorageInfo storage = updateStorage(report.getStorage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71d0a825/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 a848d50..3d19775 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
@@ -66,7 +66,6 @@ public class ProvidedStorageMap {
   // limit to a single provider for now
   private RwLock lock;
   private BlockManager bm;
-  private boolean hasDNs = false;
   private BlockAliasMap aliasMap;
 
   private final String storageId;
@@ -123,6 +122,11 @@ public class ProvidedStorageMap {
       BlockReportContext context) throws IOException {
     if (providedEnabled && storageId.equals(s.getStorageID())) {
       if (StorageType.PROVIDED.equals(s.getStorageType())) {
+        if (providedStorageInfo.getState() == State.FAILED
+            && s.getState() == State.NORMAL) {
+          providedStorageInfo.setState(State.NORMAL);
+          LOG.info("Provided storage transitioning to state " + State.NORMAL);
+        }
         processProvidedStorageReport(context);
         dn.injectStorage(providedStorageInfo);
         return providedDescriptor.getProvidedStorage(dn, s);
@@ -135,21 +139,14 @@ public class ProvidedStorageMap {
   private void processProvidedStorageReport(BlockReportContext context)
       throws IOException {
     assert lock.hasWriteLock() : "Not holding write lock";
-    if (hasDNs) {
-      return;
-    }
-    if (providedStorageInfo.getBlockReportCount() == 0) {
+    if (providedStorageInfo.getBlockReportCount() == 0
+        || providedDescriptor.activeProvidedDatanodes() == 0) {
       LOG.info("Calling process first blk report from storage: "
           + providedStorageInfo);
       // first pass; periodic refresh should call bm.processReport
       bm.processFirstBlockReport(providedStorageInfo,
           new ProvidedBlockList(aliasMap.getReader(null).iterator()));
-    } else {
-      bm.processReport(providedStorageInfo,
-          new ProvidedBlockList(aliasMap.getReader(null).iterator()),
-          context);
     }
-    hasDNs = true;
   }
 
   @VisibleForTesting
@@ -167,9 +164,10 @@ public class ProvidedStorageMap {
   public void removeDatanode(DatanodeDescriptor dnToRemove) {
     if (providedEnabled) {
       assert lock.hasWriteLock() : "Not holding write lock";
-      int remainingDatanodes = providedDescriptor.remove(dnToRemove);
-      if (remainingDatanodes == 0) {
-        hasDNs = false;
+      providedDescriptor.remove(dnToRemove);
+      // if all datanodes fail, set the block report count to 0
+      if (providedDescriptor.activeProvidedDatanodes() == 0) {
+        providedStorageInfo.setBlockReportCount(0);
       }
     }
   }
@@ -466,6 +464,22 @@ public class ProvidedStorageMap {
         return false;
       }
     }
+
+    @Override
+    void setState(DatanodeStorage.State state) {
+      if (state == State.FAILED) {
+        // The state should change to FAILED only when there are no active
+        // datanodes with PROVIDED storage.
+        ProvidedDescriptor dn = (ProvidedDescriptor) getDatanodeDescriptor();
+        if (dn.activeProvidedDatanodes() == 0) {
+          LOG.info("Provided storage {} transitioning to state {}",
+              this, State.FAILED);
+          super.setState(state);
+        }
+      } else {
+        super.setState(state);
+      }
+    }
   }
   /**
    * Used to emulate block reports for provided blocks.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71d0a825/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 2170baa..aae04be 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
@@ -492,4 +492,44 @@ public class TestNameNodeProvidedImplementation {
           dnInfos[0].getXferAddr());
     }
   }
+
+  @Test(timeout=300000)
+  public void testTransientDeadDatanodes() throws Exception {
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+            FixedBlockResolver.class);
+    // 2 Datanodes, 1 PROVIDED and other DISK
+    startCluster(NNDIRPATH, 2, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
+        false);
+
+    DataNode providedDatanode = cluster.getDataNodes().get(0);
+
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+            cluster.getNameNodePort()), cluster.getConfiguration(0));
+
+    for (int i= 0; i < numFiles; i++) {
+      String filename = "/" + filePrefix + i + fileSuffix;
+
+      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      // location should be the provided DN.
+      assertTrue(dnInfos[0].getDatanodeUuid()
+          .equals(providedDatanode.getDatanodeUuid()));
+
+      // NameNode thinks the datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode.getDatanodeId().getXferAddr());
+      cluster.waitActive();
+      cluster.triggerHeartbeats();
+      Thread.sleep(1000);
+
+      // should find the block on the 2nd provided datanode.
+      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      assertTrue(
+          dnInfos[0].getDatanodeUuid()
+              .equals(providedDatanode.getDatanodeUuid()));
+    }
+  }
 }


---------------------------------------------------------------------
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-12605. [READ] TestNameNodeProvidedImplementation#testProvidedDatanodeFailures fails after rebase

Posted by kk...@apache.org.
HDFS-12605. [READ] TestNameNodeProvidedImplementation#testProvidedDatanodeFailures fails after rebase


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

Branch: refs/heads/YARN-6592
Commit: d6a9a8997339939b59ce36246225f7cc45b21da5
Parents: 17052c4
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Oct 18 13:53:11 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/DatanodeDescriptor.java | 12 ++++++++++++
 .../namenode/TestNameNodeProvidedImplementation.java    |  6 +++---
 2 files changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6a9a899/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 28a3d1a..e3d6582 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -489,6 +489,18 @@ public class DatanodeDescriptor extends DatanodeInfo {
     synchronized (storageMap) {
       DatanodeStorageInfo storage = storageMap.get(s.getStorageID());
       if (null == storage) {
+        LOG.info("Adding new storage ID {} for DN {}", s.getStorageID(),
+            getXferAddr());
+        DFSTopologyNodeImpl parent = null;
+        if (getParent() instanceof DFSTopologyNodeImpl) {
+          parent = (DFSTopologyNodeImpl) getParent();
+        }
+        StorageType type = s.getStorageType();
+        if (!hasStorageType(type) && parent != null) {
+          // we are about to add a type this node currently does not have,
+          // inform the parent that a new type is added to this datanode
+          parent.childAddStorage(getName(), type);
+        }
         storageMap.put(s.getStorageID(), s);
       } else {
         assert storage == s : "found " + storage + " expected " + s;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6a9a899/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 3f937c4..d622b9e 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
@@ -481,13 +481,13 @@ public class TestNameNodeProvidedImplementation {
       assertEquals(providedDatanode2.getDatanodeUuid(),
           dnInfos[0].getDatanodeUuid());
 
-      //stop the 2nd provided datanode
-      cluster.stopDataNode(1);
+      // 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, 0);
 
       //restart the provided datanode


---------------------------------------------------------------------
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: HDFS-12712. [9806] Code style cleanup

Posted by kk...@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-6592
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


[24/50] [abbrv] hadoop git commit: HDFS-12591. [READ] Implement LevelDBFileRegionFormat. Contributed by Ewan Higgs.

Posted by kk...@apache.org.
HDFS-12591. [READ] Implement LevelDBFileRegionFormat. Contributed by Ewan Higgs.


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

Branch: refs/heads/YARN-6592
Commit: b634053c4daec181511abb314aeef0a8fe851086
Parents: 352f994
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Sat Dec 2 12:22:00 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   2 +
 .../impl/LevelDBFileRegionAliasMap.java         | 257 +++++++++++++++++++
 .../impl/TestLevelDBFileRegionAliasMap.java     | 115 +++++++++
 3 files changed, 374 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b634053c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 00976f9..7db0a8d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -352,6 +352,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_PROVIDED_ALIASMAP_TEXT_CODEC = "dfs.provided.aliasmap.text.codec";
   public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH = "dfs.provided.aliasmap.text.write.path";
 
+  public static final String DFS_PROVIDED_ALIASMAP_LEVELDB_PATH = "dfs.provided.aliasmap.leveldb.read.path";
+
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;
   public static final String  DFS_CONTENT_SUMMARY_LIMIT_KEY = "dfs.content-summary.limit";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b634053c/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
new file mode 100644
index 0000000..66971a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
@@ -0,0 +1,257 @@
+/**
+ * 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.common.blockaliasmap.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import static org.fusesource.leveldbjni.JniDBFactory.factory;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.fromBlockBytes;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.fromProvidedStorageLocationBytes;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.toProtoBufBytes;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A LevelDB based implementation of {@link BlockAliasMap}.
+ */
+public class LevelDBFileRegionAliasMap
+      extends BlockAliasMap<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private LevelDBOptions opts = new LevelDBOptions();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(LevelDBFileRegionAliasMap.class);
+
+  @Override
+  public void setConf(Configuration conf) {
+    opts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+    if (null == opts) {
+      opts = this.opts;
+    }
+    if (!(opts instanceof LevelDBOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    LevelDBOptions o = (LevelDBOptions) opts;
+    return new LevelDBFileRegionAliasMap.LevelDBReader(
+        createDB(o.levelDBPath, false));
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    if (null == opts) {
+      opts = this.opts;
+    }
+    if (!(opts instanceof LevelDBOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    LevelDBOptions o = (LevelDBOptions) opts;
+    return new LevelDBFileRegionAliasMap.LevelDBWriter(
+        createDB(o.levelDBPath, true));
+  }
+
+  private static DB createDB(String levelDBPath, boolean createIfMissing)
+      throws IOException {
+    if (levelDBPath == null || levelDBPath.length() == 0) {
+      throw new IllegalArgumentException(
+          "A valid path needs to be specified for "
+              + LevelDBFileRegionAliasMap.class + " using the parameter "
+              + DFS_PROVIDED_ALIASMAP_LEVELDB_PATH);
+    }
+    org.iq80.leveldb.Options options = new org.iq80.leveldb.Options();
+    options.createIfMissing(createIfMissing);
+    return factory.open(new File(levelDBPath), options);
+  }
+
+  @Override
+  public void refresh() throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Do nothing.
+  }
+
+  /**
+   * Class specifying reader options for the {@link LevelDBFileRegionAliasMap}.
+   */
+  public static class LevelDBOptions implements LevelDBReader.Options,
+      LevelDBWriter.Options, Configurable {
+    private Configuration conf;
+    private String levelDBPath;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      this.levelDBPath = conf.get(DFS_PROVIDED_ALIASMAP_LEVELDB_PATH);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public LevelDBOptions filename(String levelDBPath) {
+      this.levelDBPath = levelDBPath;
+      return this;
+    }
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as LevelDB files.
+   */
+  public static class LevelDBReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link LevelDBReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(String levelDBPath);
+    }
+
+    private DB db;
+
+    LevelDBReader(DB db) {
+      this.db = db;
+    }
+
+    @Override
+    public Optional<FileRegion> resolve(Block block) throws IOException {
+      if (db == null) {
+        return Optional.empty();
+      }
+      // consider layering index w/ composable format
+      byte[] key = toProtoBufBytes(block);
+      byte[] value = db.get(key);
+      ProvidedStorageLocation psl = fromProvidedStorageLocationBytes(value);
+      return Optional.of(new FileRegion(block, psl));
+    }
+
+    static class FRIterator implements Iterator<FileRegion> {
+      private final DBIterator internal;
+
+      FRIterator(DBIterator internal) {
+        this.internal = internal;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return internal.hasNext();
+      }
+
+      @Override
+      public FileRegion next() {
+        Map.Entry<byte[], byte[]> entry = internal.next();
+        if (entry == null) {
+          return null;
+        }
+        try {
+          Block block = fromBlockBytes(entry.getKey());
+          ProvidedStorageLocation psl =
+              fromProvidedStorageLocationBytes(entry.getValue());
+          return new FileRegion(block, psl);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    public Iterator<FileRegion> iterator() {
+      if (db == null) {
+        return null;
+      }
+      DBIterator iterator = db.iterator();
+      iterator.seekToFirst();
+      return new FRIterator(iterator);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (db != null) {
+        db.close();
+      }
+    }
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as LevelDB files.
+   */
+  public static class LevelDBWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options filename(String levelDBPath);
+    }
+
+    private final DB db;
+
+    LevelDBWriter(DB db) {
+      this.db = db;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      byte[] key = toProtoBufBytes(token.getBlock());
+      byte[] value = toProtoBufBytes(token.getProvidedStorageLocation());
+      db.put(key, value);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (db != null) {
+        db.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b634053c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
new file mode 100644
index 0000000..21199e1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
@@ -0,0 +1,115 @@
+/**
+ * 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.common.blockaliasmap.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Tests for the {@link LevelDBFileRegionAliasMap}.
+ */
+public class TestLevelDBFileRegionAliasMap {
+
+  /**
+   * A basic test to verify that we can write data and read it back again.
+   * @throws Exception
+   */
+  @Test
+  public void testReadBack() throws Exception {
+    File dbFile = Files.createTempDirectory("fileregionformat")
+        .toFile();
+    try {
+      LevelDBFileRegionAliasMap frf = new LevelDBFileRegionAliasMap();
+      LevelDBFileRegionAliasMap.LevelDBOptions opts =
+          new LevelDBFileRegionAliasMap.LevelDBOptions()
+              .filename(dbFile.getAbsolutePath());
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts);
+
+      FileRegion fr = new FileRegion(1, new Path("/file"), 1, 1, 1);
+      writer.store(fr);
+      writer.close();
+
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts);
+      FileRegion fr2 = reader.resolve(new Block(1, 1, 1)).get();
+      assertEquals(fr, fr2);
+      reader.close();
+    } finally {
+      dbFile.delete();
+    }
+  }
+
+  @Test
+  /**
+   * A basic test to verify that we can read a bunch of data that we've written.
+   */
+  public void testIterate() throws Exception {
+    FileRegion[] regions = new FileRegion[10];
+    regions[0] = new FileRegion(1, new Path("/file1"), 0, 1024, 1);
+    regions[1] = new FileRegion(2, new Path("/file1"), 1024, 1024, 1);
+    regions[2] = new FileRegion(3, new Path("/file1"), 2048, 1024, 1);
+    regions[3] = new FileRegion(4, new Path("/file2"), 0, 1024, 1);
+    regions[4] = new FileRegion(5, new Path("/file2"), 1024, 1024, 1);
+    regions[5] = new FileRegion(6, new Path("/file2"), 2048, 1024, 1);
+    regions[6] = new FileRegion(7, new Path("/file2"), 3072, 1024, 1);
+    regions[7] = new FileRegion(8, new Path("/file3"), 0, 1024, 1);
+    regions[8] = new FileRegion(9, new Path("/file4"), 0, 1024, 1);
+    regions[9] = new FileRegion(10, new Path("/file5"), 0, 1024,  1);
+    File dbFile = Files.createTempDirectory("fileregionformat")
+        .toFile();
+    try {
+      LevelDBFileRegionAliasMap frf = new LevelDBFileRegionAliasMap();
+      LevelDBFileRegionAliasMap.LevelDBOptions opts =
+          new LevelDBFileRegionAliasMap.LevelDBOptions()
+              .filename(dbFile.getAbsolutePath());
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts);
+
+      for (FileRegion fr : regions) {
+        writer.store(fr);
+      }
+      writer.close();
+
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts);
+      Iterator<FileRegion> it = reader.iterator();
+      int last = -1;
+      int count = 0;
+      while(it.hasNext()) {
+        FileRegion fr = it.next();
+        int blockId = (int)fr.getBlock().getBlockId();
+        assertEquals(regions[blockId-1], fr);
+        assertNotEquals(blockId, last);
+        last = blockId;
+        count++;
+      }
+      assertEquals(count, 10);
+
+      reader.close();
+    } finally {
+      dbFile.delete();
+    }
+  }
+}


---------------------------------------------------------------------
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: HDFS-12893. [READ] Support replication of Provided blocks with non-default topologies.

Posted by kk...@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-6592
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


[30/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 kk...@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-6592
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


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

Posted by kk...@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


[09/50] [abbrv] hadoop git commit: HDFS-12289. [READ] HDFS-12091 breaks the tests for provided block reads

Posted by kk...@apache.org.
HDFS-12289. [READ] HDFS-12091 breaks the tests for provided block reads


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

Branch: refs/heads/YARN-6592
Commit: aca023b72cdb325ca66d196443218f6107efa1ca
Parents: 2407c9b
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Aug 14 10:29:47 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 30 +++++++++++++++++++-
 .../TestNameNodeProvidedImplementation.java     |  4 ++-
 2 files changed, 32 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca023b7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 2d710be..c694854 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -147,6 +147,9 @@ public class MiniDFSCluster implements AutoCloseable {
       GenericTestUtils.SYSPROP_TEST_DATA_DIR;
   /** Configuration option to set the data dir: {@value} */
   public static final String HDFS_MINIDFS_BASEDIR = "hdfs.minidfs.basedir";
+  /** Configuration option to set the provided data dir: {@value} */
+  public static final String HDFS_MINIDFS_BASEDIR_PROVIDED =
+      "hdfs.minidfs.basedir.provided";
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
       = DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + ".testing";
   public static final String  DFS_NAMENODE_DECOMMISSION_INTERVAL_TESTING_KEY
@@ -1397,7 +1400,12 @@ public class MiniDFSCluster implements AutoCloseable {
       if ((storageTypes != null) && (j >= storageTypes.length)) {
         break;
       }
-      File dir = getInstanceStorageDir(dnIndex, j);
+      File dir;
+      if (storageTypes != null && storageTypes[j] == StorageType.PROVIDED) {
+        dir = getProvidedStorageDir(dnIndex, j);
+      } else {
+        dir = getInstanceStorageDir(dnIndex, j);
+      }
       dir.mkdirs();
       if (!dir.isDirectory()) {
         throw new IOException("Mkdirs failed to create directory for DataNode " + dir);
@@ -2847,6 +2855,26 @@ public class MiniDFSCluster implements AutoCloseable {
   }
 
   /**
+   * Get a storage directory for PROVIDED storages.
+   * The PROVIDED directory to return can be set by using the configuration
+   * parameter {@link #HDFS_MINIDFS_BASEDIR_PROVIDED}. If this parameter is
+   * not set, this function behaves exactly the same as
+   * {@link #getInstanceStorageDir(int, int)}. Currently, the two parameters
+   * are ignored as only one PROVIDED storage is supported in HDFS-9806.
+   *
+   * @param dnIndex datanode index (starts from 0)
+   * @param dirIndex directory index
+   * @return Storage directory
+   */
+  public File getProvidedStorageDir(int dnIndex, int dirIndex) {
+    String base = conf.get(HDFS_MINIDFS_BASEDIR_PROVIDED, null);
+    if (base == null) {
+      return getInstanceStorageDir(dnIndex, dirIndex);
+    }
+    return new File(base);
+  }
+
+  /**
    * Get a storage directory for a datanode.
    * <ol>
    * <li><base directory>/data/data<2*dnIndex + 1></li>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca023b7/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 60b306f..3f937c4 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
@@ -74,7 +74,7 @@ public class TestNameNodeProvidedImplementation {
   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 NAMEPATH = new Path(BASE, "providedDir");
   final Path NNDIRPATH = new Path(BASE, "nnDir");
   final Path BLOCKFILE = new Path(NNDIRPATH, "blocks.csv");
   final String SINGLEUSER = "usr1";
@@ -116,6 +116,8 @@ public class TestNameNodeProvidedImplementation {
         BLOCKFILE.toString());
     conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_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);


---------------------------------------------------------------------
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: HDFS-12776. [READ] Increasing replication for PROVIDED files should create local replicas

Posted by kk...@apache.org.
HDFS-12776. [READ] Increasing replication for PROVIDED files should create local replicas


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

Branch: refs/heads/YARN-6592
Commit: 90d1b47a2a400e07e2b6b812c4bbd9c4f2877786
Parents: 87dc026
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 9 13:03:41 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockInfo.java  |  7 ++--
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 25 +++++++++++---
 .../TestNameNodeProvidedImplementation.java     | 36 +++++++++++---------
 3 files changed, 45 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/90d1b47a/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 eb09b7b..8f59df6 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
@@ -187,20 +187,23 @@ public abstract class BlockInfo extends Block
    */
   DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
     int len = getCapacity();
+    DatanodeStorageInfo providedStorageInfo = null;
     for(int idx = 0; idx < len; idx++) {
       DatanodeStorageInfo cur = getStorageInfo(idx);
       if(cur != null) {
         if (cur.getStorageType() == StorageType.PROVIDED) {
           //if block resides on provided storage, only match the storage ids
           if (dn.getStorageInfo(cur.getStorageID()) != null) {
-            return cur;
+            // do not return here as we have to check the other
+            // DatanodeStorageInfos for this block which could be local
+            providedStorageInfo = cur;
           }
         } else if (cur.getDatanodeDescriptor() == dn) {
           return cur;
         }
       }
     }
-    return null;
+    return providedStorageInfo;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90d1b47a/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 db8d60c..fd06a56 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
@@ -1512,6 +1512,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
   }
 
+  private boolean isReplicaProvided(ReplicaInfo replicaInfo) {
+    if (replicaInfo == null) {
+      return false;
+    }
+    return replicaInfo.getVolume().getStorageType() == StorageType.PROVIDED;
+  }
+
   @Override // FsDatasetSpi
   public ReplicaHandler createTemporary(StorageType storageType,
       String storageId, ExtendedBlock b, boolean isTransfer)
@@ -1530,12 +1537,14 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           isInPipeline = currentReplicaInfo.getState() == ReplicaState.TEMPORARY
               || currentReplicaInfo.getState() == ReplicaState.RBW;
           /*
-           * If the current block is old, reject.
+           * If the current block is not PROVIDED and old, reject.
            * else If transfer request, then accept it.
            * else if state is not RBW/Temporary, then reject
+           * If current block is PROVIDED, ignore the replica.
            */
-          if ((currentReplicaInfo.getGenerationStamp() >= b.getGenerationStamp())
-              || (!isTransfer && !isInPipeline)) {
+          if (((currentReplicaInfo.getGenerationStamp() >= b
+              .getGenerationStamp()) || (!isTransfer && !isInPipeline))
+              && !isReplicaProvided(currentReplicaInfo)) {
             throw new ReplicaAlreadyExistsException("Block " + b
                 + " already exists in state " + currentReplicaInfo.getState()
                 + " and thus cannot be created.");
@@ -1555,11 +1564,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             + " after " + writerStopMs + " miniseconds.");
       }
 
+      // if lastFoundReplicaInfo is PROVIDED and FINALIZED,
+      // stopWriter isn't required.
+      if (isReplicaProvided(lastFoundReplicaInfo) &&
+          lastFoundReplicaInfo.getState() == ReplicaState.FINALIZED) {
+        continue;
+      }
       // Stop the previous writer
       ((ReplicaInPipeline)lastFoundReplicaInfo).stopWriter(writerStopTimeoutMs);
     } while (true);
-
-    if (lastFoundReplicaInfo != null) {
+    if (lastFoundReplicaInfo != null
+        && !isReplicaProvided(lastFoundReplicaInfo)) {
       // Old blockfile should be deleted synchronously as it might collide
       // with the new block if allocated in same volume.
       // Do the deletion outside of lock as its DISK IO.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90d1b47a/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 f0303b5..1f6aebb 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
@@ -401,33 +401,37 @@ public class TestNameNodeProvidedImplementation {
   public void testSetReplicationForProvidedFiles() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 2, null,
-        new StorageType[][]{
-            {StorageType.PROVIDED},
-            {StorageType.DISK}},
+    // 10 Datanodes with both DISK and PROVIDED storage
+    startCluster(NNDIRPATH, 10,
+        new StorageType[]{
+            StorageType.PROVIDED, StorageType.DISK},
+        null,
         false);
 
     String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
     Path file = new Path(filename);
     FileSystem fs = cluster.getFileSystem();
 
-    //set the replication to 2, and test that the file has
-    //the required replication.
-    fs.setReplication(file, (short) 2);
+    // 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, (short) 2, 10000);
+        file, newReplication, 10000);
     DFSClient client = new DFSClient(new InetSocketAddress("localhost",
         cluster.getNameNodePort()), cluster.getConfiguration(0));
-    getAndCheckBlockLocations(client, filename, 2);
+    getAndCheckBlockLocations(client, filename, newReplication);
 
-    //set the replication back to 1
-    fs.setReplication(file, (short) 1);
+    // set the replication back to 1
+    newReplication = 1;
+    LOG.info("Setting replication of file {} back to {}",
+        filename, newReplication);
+    fs.setReplication(file, newReplication);
     DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
-        file, (short) 1, 10000);
-    //the only replica left should be the PROVIDED datanode
-    DatanodeInfo[] infos = getAndCheckBlockLocations(client, filename, 1);
-    assertEquals(cluster.getDataNodes().get(0).getDatanodeUuid(),
-        infos[0].getDatanodeUuid());
+        file, newReplication, 10000);
+    // the only replica left should be the PROVIDED datanode
+    getAndCheckBlockLocations(client, filename, newReplication);
   }
 
   @Test


---------------------------------------------------------------------
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: HDFS-12894. [READ] Skip setting block count of ProvidedDatanodeStorageInfo on DN registration update

Posted by kk...@apache.org.
HDFS-12894. [READ] Skip setting block count of ProvidedDatanodeStorageInfo on DN registration update


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

Branch: refs/heads/YARN-6592
Commit: fb996a32a98a25c0fe34a8ebb28563b53cd6e20e
Parents: 9c35be8
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Dec 5 17:55:32 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  5 +++++
 .../blockmanagement/DatanodeDescriptor.java     |  4 +++-
 .../TestNameNodeProvidedImplementation.java     | 20 +++++++++++++++++++-
 3 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb996a32/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 f92c4e8..916cbaa 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
@@ -4943,4 +4943,9 @@ public class BlockManager implements BlockStatsMXBean {
   public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
     pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
   }
+
+  @VisibleForTesting
+  public ProvidedStorageMap getProvidedStorageMap() {
+    return providedStorageMap;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb996a32/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 83c608f..fc58708 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -919,7 +919,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
     
     // must re-process IBR after re-registration
     for(DatanodeStorageInfo storage : getStorageInfos()) {
-      storage.setBlockReportCount(0);
+      if (storage.getStorageType() != StorageType.PROVIDED) {
+        storage.setBlockReportCount(0);
+      }
     }
     heartbeatedSinceRegistration = false;
     forceRegistration = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb996a32/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 deaf9d5..d057247 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
@@ -559,7 +559,9 @@ public class TestNameNodeProvidedImplementation {
     DataNode providedDatanode2 = cluster.getDataNodes().get(1);
 
     DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-            cluster.getNameNodePort()), cluster.getConfiguration(0));
+        cluster.getNameNodePort()), cluster.getConfiguration(0));
+
+    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
 
     if (numFiles >= 1) {
       String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
@@ -596,10 +598,15 @@ public class TestNameNodeProvidedImplementation {
           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.
@@ -621,6 +628,8 @@ public class TestNameNodeProvidedImplementation {
         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);
@@ -631,10 +640,19 @@ public class TestNameNodeProvidedImplementation {
       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,


---------------------------------------------------------------------
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: HDFS-11640. [READ] Datanodes should use a unique identifier when reading from external stores

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


[21/50] [abbrv] hadoop git commit: HDFS-12665. [AliasMap] Create a version of the AliasMap that runs in memory in the Namenode (leveldb). Contributed by Ewan Higgs.

Posted by kk...@apache.org.
HDFS-12665. [AliasMap] Create a version of the AliasMap that runs in memory in the Namenode (leveldb). Contributed by Ewan Higgs.


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

Branch: refs/heads/YARN-6592
Commit: 352f994b6484524cdcfcda021046c59905b62f31
Parents: cc933cb
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 30 10:37:28 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../hdfs/protocol/ProvidedStorageLocation.java  |  85 +++++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  32 ++
 .../src/main/proto/hdfs.proto                   |  14 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   9 +
 .../hdfs/protocolPB/AliasMapProtocolPB.java     |  35 ++
 .../AliasMapProtocolServerSideTranslatorPB.java | 120 +++++++
 ...yAliasMapProtocolClientSideTranslatorPB.java | 159 +++++++++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  28 ++
 .../hdfs/server/aliasmap/InMemoryAliasMap.java  | 213 ++++++++++++
 .../aliasmap/InMemoryAliasMapProtocol.java      |  92 +++++
 .../aliasmap/InMemoryLevelDBAliasMapServer.java | 141 ++++++++
 .../hadoop/hdfs/server/common/FileRegion.java   |  89 ++---
 .../common/blockaliasmap/BlockAliasMap.java     |  19 +-
 .../impl/InMemoryLevelDBAliasMapClient.java     | 156 +++++++++
 .../impl/TextFileRegionAliasMap.java            |  40 ++-
 .../datanode/FinalizedProvidedReplica.java      |  11 +
 .../hdfs/server/datanode/ReplicaBuilder.java    |   7 +-
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  38 +--
 .../hadoop/hdfs/server/namenode/NameNode.java   |  21 ++
 .../src/main/proto/AliasMapProtocol.proto       |  60 ++++
 .../src/main/resources/hdfs-default.xml         |  34 ++
 .../server/aliasmap/ITestInMemoryAliasMap.java  | 126 +++++++
 .../server/aliasmap/TestInMemoryAliasMap.java   |  45 +++
 .../blockmanagement/TestProvidedStorageMap.java |   1 -
 .../impl/TestInMemoryLevelDBAliasMapClient.java | 341 +++++++++++++++++++
 .../impl/TestLevelDbMockAliasMapClient.java     | 116 +++++++
 .../fsdataset/impl/TestProvidedImpl.java        |   9 +-
 hadoop-project/pom.xml                          |   8 +-
 hadoop-tools/hadoop-fs2img/pom.xml              |   6 +
 .../hdfs/server/namenode/NullBlockAliasMap.java |   9 +-
 .../TestNameNodeProvidedImplementation.java     |  65 +++-
 32 files changed, 2016 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
new file mode 100644
index 0000000..eee58ba
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.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.hdfs.protocol;
+
+import org.apache.hadoop.fs.Path;
+
+import javax.annotation.Nonnull;
+import java.util.Arrays;
+
+/**
+ * ProvidedStorageLocation is a location in an external storage system
+ * containing the data for a block (~Replica).
+ */
+public class ProvidedStorageLocation {
+  private final Path path;
+  private final long offset;
+  private final long length;
+  private final byte[] nonce;
+
+  public ProvidedStorageLocation(Path path, long offset, long length,
+      byte[] nonce) {
+    this.path = path;
+    this.offset = offset;
+    this.length = length;
+    this.nonce = Arrays.copyOf(nonce, nonce.length);
+  }
+
+  public @Nonnull Path getPath() {
+    return path;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public @Nonnull byte[] getNonce() {
+    // create a copy of the nonce and return it.
+    return Arrays.copyOf(nonce, nonce.length);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ProvidedStorageLocation that = (ProvidedStorageLocation) o;
+
+    if ((offset != that.offset) || (length != that.length)
+        || !path.equals(that.path)) {
+      return false;
+    }
+    return Arrays.equals(nonce, that.nonce);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = path.hashCode();
+    result = 31 * result + (int) (offset ^ (offset >>> 32));
+    result = 31 * result + (int) (length ^ (length >>> 32));
+    result = 31 * result + Arrays.hashCode(nonce);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 8ff9bfe..813083f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -96,6 +96,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
@@ -3227,4 +3228,35 @@ public class PBHelperClient {
     }
     return ret;
   }
+
+  public static ProvidedStorageLocation convert(
+      HdfsProtos.ProvidedStorageLocationProto providedStorageLocationProto) {
+    if (providedStorageLocationProto == null) {
+      return null;
+    }
+    String path = providedStorageLocationProto.getPath();
+    long length = providedStorageLocationProto.getLength();
+    long offset = providedStorageLocationProto.getOffset();
+    ByteString nonce = providedStorageLocationProto.getNonce();
+
+    if (path == null || length == -1 || offset == -1 || nonce == null) {
+      return null;
+    } else {
+      return new ProvidedStorageLocation(new Path(path), offset, length,
+          nonce.toByteArray());
+    }
+  }
+
+  public static HdfsProtos.ProvidedStorageLocationProto convert(
+      ProvidedStorageLocation providedStorageLocation) {
+    String path = providedStorageLocation.getPath().toString();
+    return HdfsProtos.ProvidedStorageLocationProto.newBuilder()
+        .setPath(path)
+        .setLength(providedStorageLocation.getLength())
+        .setOffset(providedStorageLocation.getOffset())
+        .setNonce(ByteString.copyFrom(providedStorageLocation.getNonce()))
+        .build();
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 25f1339..405495f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -45,6 +45,20 @@ message ExtendedBlockProto {
                                                // here for historical reasons
 }
 
+
+/**
+* ProvidedStorageLocation will contain the exact location in the provided
+  storage. The path, offset and length will result in ranged read. The nonce
+  is there to verify that you receive what you expect.
+*/
+
+message ProvidedStorageLocationProto {
+  required string path = 1;
+  required int64 offset = 2;
+  required int64 length = 3;
+  required bytes nonce = 4;
+}
+
 /**
  * Identifies a Datanode
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index cfdadf2..fe51071 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -191,7 +191,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <dependency>
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
-      <version>1.8</version>
     </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
@@ -208,6 +207,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>curator-test</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+        <groupId>org.assertj</groupId>
+        <artifactId>assertj-core</artifactId>
+        <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -341,6 +345,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>fsimage.proto</include>
                   <include>FederationProtocol.proto</include>
                   <include>RouterProtocol.proto</include>
+                  <include>AliasMapProtocol.proto</include>
                 </includes>
               </source>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index fbdc859..00976f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -95,6 +95,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
   public static final String  DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50105";
   public static final String  DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.backup.dnrpc-address";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS = "dfs.provided.aliasmap.inmemory.dnrpc-address";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT = "0.0.0.0:50200";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR = "dfs.provided.aliasmap.inmemory.leveldb.dir";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE = "dfs.provided.aliasmap.inmemory.batch-size";
+  public static final int DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT = 500;
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED = "dfs.provided.aliasmap.inmemory.enabled";
+  public static final boolean DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED_DEFAULT = false;
+
   public static final String  DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY;
   public static final long    DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_DEFAULT =
@@ -1633,4 +1641,5 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   @Deprecated
   public static final long    DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       HdfsClientConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
new file mode 100644
index 0000000..98b3ee1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.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.hdfs.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos;
+import org.apache.hadoop.ipc.ProtocolInfo;
+
+/**
+ * Protocol between the Namenode and the Datanode to read the AliasMap
+ * used for Provided storage.
+ * TODO add Kerberos support
+ */
+@ProtocolInfo(
+    protocolName =
+        "org.apache.hadoop.hdfs.server.aliasmap.AliasMapProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface AliasMapProtocolPB extends
+    AliasMapProtocolProtos.AliasMapProtocolService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..808c43b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
@@ -0,0 +1,120 @@
+/*
+ * 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.protocolPB;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.ReadResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.WriteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.WriteResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.*;
+
+/**
+ * AliasMapProtocolServerSideTranslatorPB is responsible for translating RPC
+ * calls and forwarding them to the internal InMemoryAliasMap.
+ */
+public class AliasMapProtocolServerSideTranslatorPB
+    implements AliasMapProtocolPB {
+
+  private final InMemoryAliasMapProtocol aliasMap;
+
+  public AliasMapProtocolServerSideTranslatorPB(
+      InMemoryAliasMapProtocol aliasMap) {
+    this.aliasMap = aliasMap;
+  }
+
+  private static final WriteResponseProto VOID_WRITE_RESPONSE =
+      WriteResponseProto.newBuilder().build();
+
+  @Override
+  public WriteResponseProto write(RpcController controller,
+      WriteRequestProto request) throws ServiceException {
+    try {
+      FileRegion toWrite =
+          PBHelper.convert(request.getKeyValuePair());
+
+      aliasMap.write(toWrite.getBlock(), toWrite.getProvidedStorageLocation());
+      return VOID_WRITE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReadResponseProto read(RpcController controller,
+      ReadRequestProto request) throws ServiceException {
+    try {
+      Block toRead =  PBHelperClient.convert(request.getKey());
+
+      Optional<ProvidedStorageLocation> optionalResult =
+          aliasMap.read(toRead);
+
+      ReadResponseProto.Builder builder = ReadResponseProto.newBuilder();
+      if (optionalResult.isPresent()) {
+        ProvidedStorageLocation providedStorageLocation = optionalResult.get();
+        builder.setValue(PBHelperClient.convert(providedStorageLocation));
+      }
+
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListResponseProto list(RpcController controller,
+      ListRequestProto request) throws ServiceException {
+    try {
+      BlockProto marker = request.getMarker();
+      IterationResult iterationResult;
+      if (marker.isInitialized()) {
+        iterationResult =
+            aliasMap.list(Optional.of(PBHelperClient.convert(marker)));
+      } else {
+        iterationResult = aliasMap.list(Optional.empty());
+      }
+      ListResponseProto.Builder responseBuilder =
+          ListResponseProto.newBuilder();
+      List<FileRegion> fileRegions = iterationResult.getFileRegions();
+
+      List<KeyValueProto> keyValueProtos = fileRegions.stream()
+          .map(PBHelper::convert).collect(Collectors.toList());
+      responseBuilder.addAllFileRegions(keyValueProtos);
+      Optional<Block> nextMarker = iterationResult.getNextBlock();
+      nextMarker
+          .map(m -> responseBuilder.setNextMarker(PBHelperClient.convert(m)));
+
+      return responseBuilder.build();
+
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..a79360f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
@@ -0,0 +1,159 @@
+/*
+ * 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.protocolPB;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS;
+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.protocol.proto.HdfsProtos.*;
+
+/**
+ * This class is the client side translator to translate requests made to the
+ * {@link InMemoryAliasMapProtocol} interface to the RPC server implementing
+ * {@link AliasMapProtocolPB}.
+ */
+public class InMemoryAliasMapProtocolClientSideTranslatorPB
+    implements InMemoryAliasMapProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory
+          .getLogger(InMemoryAliasMapProtocolClientSideTranslatorPB.class);
+
+  private AliasMapProtocolPB rpcProxy;
+
+  public InMemoryAliasMapProtocolClientSideTranslatorPB(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT);
+    InetSocketAddress aliasMapAddr = NetUtils.createSocketAddr(addr);
+
+    RPC.setProtocolEngine(conf, AliasMapProtocolPB.class,
+        ProtobufRpcEngine.class);
+    LOG.info("Connecting to address: " + addr);
+    try {
+      rpcProxy = RPC.getProxy(AliasMapProtocolPB.class,
+          RPC.getProtocolVersion(AliasMapProtocolPB.class), aliasMapAddr, null,
+          conf, NetUtils.getDefaultSocketFactory(conf), 0);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Override
+  public InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException {
+    ListRequestProto.Builder builder = ListRequestProto.newBuilder();
+    if (marker.isPresent()) {
+      builder.setMarker(PBHelperClient.convert(marker.get()));
+    }
+    ListRequestProto request = builder.build();
+    try {
+      ListResponseProto response = rpcProxy.list(null, request);
+      List<KeyValueProto> fileRegionsList = response.getFileRegionsList();
+
+      List<FileRegion> fileRegions = fileRegionsList
+          .stream()
+          .map(kv -> new FileRegion(
+              PBHelperClient.convert(kv.getKey()),
+              PBHelperClient.convert(kv.getValue()),
+              null
+          ))
+          .collect(Collectors.toList());
+      BlockProto nextMarker = response.getNextMarker();
+
+      if (nextMarker.isInitialized()) {
+        return new InMemoryAliasMap.IterationResult(fileRegions,
+            Optional.of(PBHelperClient.convert(nextMarker)));
+      } else {
+        return new InMemoryAliasMap.IterationResult(fileRegions,
+            Optional.empty());
+      }
+
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Nonnull
+  @Override
+  public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+
+    ReadRequestProto request =
+        ReadRequestProto
+            .newBuilder()
+            .setKey(PBHelperClient.convert(block))
+            .build();
+    try {
+      ReadResponseProto response = rpcProxy.read(null, request);
+
+      ProvidedStorageLocationProto providedStorageLocation =
+          response.getValue();
+      if (providedStorageLocation.isInitialized()) {
+        return Optional.of(PBHelperClient.convert(providedStorageLocation));
+      }
+      return Optional.empty();
+
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    WriteRequestProto request =
+        WriteRequestProto
+            .newBuilder()
+            .setKeyValuePair(KeyValueProto.newBuilder()
+                .setKey(PBHelperClient.convert(block))
+                .setValue(PBHelperClient.convert(providedStorageLocation))
+                .build())
+            .build();
+
+    try {
+      rpcProxy.write(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  public void stop() {
+    RPC.stopProxy(rpcProxy);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 6539d32..2952a5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECReconstructionCommandProto;
@@ -56,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstr
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
@@ -80,6 +83,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -1096,4 +1100,28 @@ public class PBHelper {
         DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION,
         blkECReconstructionInfos);
   }
+
+  public static KeyValueProto convert(FileRegion fileRegion) {
+    return KeyValueProto
+        .newBuilder()
+        .setKey(PBHelperClient.convert(fileRegion.getBlock()))
+        .setValue(PBHelperClient.convert(
+            fileRegion.getProvidedStorageLocation()))
+        .build();
+  }
+
+  public static FileRegion
+      convert(KeyValueProto keyValueProto) {
+    BlockProto blockProto =
+        keyValueProto.getKey();
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        keyValueProto.getValue();
+
+    Block block =
+        PBHelperClient.convert(blockProto);
+    ProvidedStorageLocation providedStorageLocation =
+        PBHelperClient.convert(providedStorageLocationProto);
+
+    return new FileRegion(block, providedStorageLocation, null);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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
new file mode 100644
index 0000000..be891e5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
@@ -0,0 +1,213 @@
+/*
+ * 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.aliasmap;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.fusesource.leveldbjni.JniDBFactory;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import org.iq80.leveldb.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * InMemoryAliasMap is an implementation of the InMemoryAliasMapProtocol for
+ * use with LevelDB.
+ */
+public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
+    Configurable {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(InMemoryAliasMap.class);
+
+  private final DB levelDb;
+  private Configuration conf;
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @VisibleForTesting
+  static String createPathErrorMessage(String directory) {
+    return new StringBuilder()
+        .append("Configured directory '")
+        .append(directory)
+        .append("' doesn't exist")
+        .toString();
+  }
+
+  public static @Nonnull InMemoryAliasMap init(Configuration conf)
+      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()) {
+      String error = createPathErrorMessage(directory);
+      throw new IOException(error);
+    }
+    DB levelDb = JniDBFactory.factory.open(path, options);
+    InMemoryAliasMap aliasMap = new InMemoryAliasMap(levelDb);
+    aliasMap.setConf(conf);
+    return aliasMap;
+  }
+
+  @VisibleForTesting
+  InMemoryAliasMap(DB levelDb) {
+    this.levelDb = levelDb;
+  }
+
+  @Override
+  public IterationResult list(Optional<Block> marker) throws IOException {
+    return withIterator((DBIterator iterator) -> {
+      Integer batchSize =
+          conf.getInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT);
+      if (marker.isPresent()) {
+        iterator.seek(toProtoBufBytes(marker.get()));
+      } else {
+        iterator.seekToFirst();
+      }
+      int i = 0;
+      ArrayList<FileRegion> batch =
+          Lists.newArrayListWithExpectedSize(batchSize);
+      while (iterator.hasNext() && i < batchSize) {
+        Map.Entry<byte[], byte[]> entry = iterator.next();
+        Block block = fromBlockBytes(entry.getKey());
+        ProvidedStorageLocation providedStorageLocation =
+            fromProvidedStorageLocationBytes(entry.getValue());
+        batch.add(new FileRegion(block, providedStorageLocation, null));
+        ++i;
+      }
+      if (iterator.hasNext()) {
+        Block nextMarker = fromBlockBytes(iterator.next().getKey());
+        return new IterationResult(batch, Optional.of(nextMarker));
+      } else {
+        return new IterationResult(batch, Optional.empty());
+      }
+
+    });
+  }
+
+  public @Nonnull Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+
+    byte[] extendedBlockDbFormat = toProtoBufBytes(block);
+    byte[] providedStorageLocationDbFormat = levelDb.get(extendedBlockDbFormat);
+    if (providedStorageLocationDbFormat == null) {
+      return Optional.empty();
+    } else {
+      ProvidedStorageLocation providedStorageLocation =
+          fromProvidedStorageLocationBytes(providedStorageLocationDbFormat);
+      return Optional.of(providedStorageLocation);
+    }
+  }
+
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    byte[] extendedBlockDbFormat = toProtoBufBytes(block);
+    byte[] providedStorageLocationDbFormat =
+        toProtoBufBytes(providedStorageLocation);
+    levelDb.put(extendedBlockDbFormat, providedStorageLocationDbFormat);
+  }
+
+  public void close() throws IOException {
+    levelDb.close();
+  }
+
+  @Nonnull
+  public static ProvidedStorageLocation fromProvidedStorageLocationBytes(
+      @Nonnull byte[] providedStorageLocationDbFormat)
+      throws InvalidProtocolBufferException {
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        ProvidedStorageLocationProto
+            .parseFrom(providedStorageLocationDbFormat);
+    return PBHelperClient.convert(providedStorageLocationProto);
+  }
+
+  @Nonnull
+  public static Block fromBlockBytes(@Nonnull byte[] blockDbFormat)
+      throws InvalidProtocolBufferException {
+    BlockProto blockProto = BlockProto.parseFrom(blockDbFormat);
+    return PBHelperClient.convert(blockProto);
+  }
+
+  public static byte[] toProtoBufBytes(@Nonnull ProvidedStorageLocation
+      providedStorageLocation) throws IOException {
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        PBHelperClient.convert(providedStorageLocation);
+    ByteArrayOutputStream providedStorageLocationOutputStream =
+        new ByteArrayOutputStream();
+    providedStorageLocationProto.writeTo(providedStorageLocationOutputStream);
+    return providedStorageLocationOutputStream.toByteArray();
+  }
+
+  public static byte[] toProtoBufBytes(@Nonnull Block block)
+      throws IOException {
+    BlockProto blockProto =
+        PBHelperClient.convert(block);
+    ByteArrayOutputStream blockOutputStream = new ByteArrayOutputStream();
+    blockProto.writeTo(blockOutputStream);
+    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 <R> Return type.
+   */
+  @FunctionalInterface
+  public interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
new file mode 100644
index 0000000..fb6e8b3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.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.hdfs.server.aliasmap;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Protocol used by clients to read/write data about aliases of
+ * provided blocks for an in-memory implementation of the
+ * {@link org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap}.
+ */
+public interface InMemoryAliasMapProtocol {
+
+  /**
+   * The result of a read from the in-memory aliasmap. It contains the
+   * a list of FileRegions that are returned, along with the next block
+   * from which the read operation must continue.
+   */
+  class IterationResult {
+
+    private final List<FileRegion> batch;
+    private final Optional<Block> nextMarker;
+
+    public IterationResult(List<FileRegion> batch, Optional<Block> nextMarker) {
+      this.batch = batch;
+      this.nextMarker = nextMarker;
+    }
+
+    public List<FileRegion> getFileRegions() {
+      return batch;
+    }
+
+    public Optional<Block> getNextBlock() {
+      return nextMarker;
+    }
+  }
+
+  /**
+   * List the next batch of {@link FileRegion}s in the alias map starting from
+   * the given {@code marker}. To retrieve all {@link FileRegion}s stored in the
+   * alias map, multiple calls to this function might be required.
+   * @param marker the next block to get fileregions from.
+   * @return the {@link IterationResult} with a set of
+   * FileRegions and the next marker.
+   * @throws IOException
+   */
+  InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException;
+
+  /**
+   * Gets the {@link ProvidedStorageLocation} associated with the
+   * specified block.
+   * @param block the block to lookup
+   * @return the associated {@link ProvidedStorageLocation}.
+   * @throws IOException
+   */
+  @Nonnull
+  Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException;
+
+  /**
+   * Stores the block and it's associated {@link ProvidedStorageLocation}
+   * in the alias map.
+   * @param block
+   * @param providedStorageLocation
+   * @throws IOException
+   */
+  void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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
new file mode 100644
index 0000000..91b1e83
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
@@ -0,0 +1,141 @@
+/*
+ * 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.aliasmap;
+
+import com.google.protobuf.BlockingService;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolServerSideTranslatorPB;
+import org.apache.hadoop.ipc.RPC;
+import javax.annotation.Nonnull;
+import java.io.Closeable;
+import java.io.IOException;
+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;
+
+/**
+ * InMemoryLevelDBAliasMapServer is the entry point from the Namenode into
+ * the {@link InMemoryAliasMap}.
+ */
+public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
+    Configurable, Closeable {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(InMemoryLevelDBAliasMapServer.class);
+  private final CheckedFunction<Configuration, InMemoryAliasMap> initFun;
+  private RPC.Server aliasMapServer;
+  private Configuration conf;
+  private InMemoryAliasMap aliasMap;
+
+  public InMemoryLevelDBAliasMapServer(
+      CheckedFunction<Configuration, InMemoryAliasMap> initFun) {
+    this.initFun = initFun;
+
+  }
+
+  public void start() throws IOException {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      throw new UnsupportedOperationException("Unable to start "
+          + "InMemoryLevelDBAliasMapServer as security is enabled");
+    }
+    RPC.setProtocolEngine(getConf(), AliasMapProtocolPB.class,
+        ProtobufRpcEngine.class);
+    AliasMapProtocolServerSideTranslatorPB aliasMapProtocolXlator =
+        new AliasMapProtocolServerSideTranslatorPB(this);
+
+    BlockingService aliasMapProtocolService =
+        AliasMapProtocolService
+            .newReflectiveBlockingService(aliasMapProtocolXlator);
+
+    String rpcAddress =
+        conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+            DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT);
+    String[] split = rpcAddress.split(":");
+    String bindHost = split[0];
+    Integer port = Integer.valueOf(split[1]);
+
+    aliasMapServer = new RPC.Builder(conf)
+        .setProtocol(AliasMapProtocolPB.class)
+        .setInstance(aliasMapProtocolService)
+        .setBindAddress(bindHost)
+        .setPort(port)
+        .setNumHandlers(1)
+        .setVerbose(true)
+        .build();
+
+    LOG.info("Starting InMemoryLevelDBAliasMapServer on ", rpcAddress);
+    aliasMapServer.start();
+  }
+
+  @Override
+  public InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException {
+    return aliasMap.list(marker);
+  }
+
+  @Nonnull
+  @Override
+  public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+    return aliasMap.read(block);
+  }
+
+  @Override
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    aliasMap.write(block, providedStorageLocation);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    try {
+      this.aliasMap = initFun.apply(conf);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void close() {
+    LOG.info("Stopping InMemoryLevelDBAliasMapServer");
+    try {
+      aliasMap.close();
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    }
+    aliasMapServer.stop();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 c568b90..5d04640 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
@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.common;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 
 /**
  * This class is used to represent provided blocks that are file regions,
@@ -27,95 +29,70 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  */
 public class FileRegion implements BlockAlias {
 
-  private final Path path;
-  private final long offset;
-  private final long length;
-  private final long blockId;
+  private final Pair<Block, ProvidedStorageLocation> pair;
   private final String bpid;
-  private final long genStamp;
 
   public FileRegion(long blockId, Path path, long offset,
       long length, String bpid, long genStamp) {
-    this.path = path;
-    this.offset = offset;
-    this.length = length;
-    this.blockId = blockId;
-    this.bpid = bpid;
-    this.genStamp = genStamp;
+    this(new Block(blockId, length, genStamp),
+        new ProvidedStorageLocation(path, offset, length, new byte[0]), bpid);
   }
 
   public FileRegion(long blockId, Path path, long offset,
       long length, String bpid) {
     this(blockId, path, offset, length, bpid,
         HdfsConstants.GRANDFATHER_GENERATION_STAMP);
-
   }
 
   public FileRegion(long blockId, Path path, long offset,
       long length, long genStamp) {
     this(blockId, path, offset, length, null, genStamp);
+  }
 
+  public FileRegion(Block block,
+      ProvidedStorageLocation providedStorageLocation) {
+    this.pair  = Pair.of(block, providedStorageLocation);
+    this.bpid = null;
+  }
+
+  public FileRegion(Block block,
+      ProvidedStorageLocation providedStorageLocation, String bpid) {
+    this.pair  = Pair.of(block, providedStorageLocation);
+    this.bpid = bpid;
   }
 
   public FileRegion(long blockId, Path path, long offset, long length) {
     this(blockId, path, offset, length, null);
   }
 
-  @Override
   public Block getBlock() {
-    return new Block(blockId, length, genStamp);
+    return pair.getKey();
   }
 
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof FileRegion)) {
-      return false;
-    }
-    FileRegion o = (FileRegion) other;
-    return blockId == o.blockId
-      && offset == o.offset
-      && length == o.length
-      && genStamp == o.genStamp
-      && path.equals(o.path);
-  }
-
-  @Override
-  public int hashCode() {
-    return (int)(blockId & Integer.MIN_VALUE);
+  public ProvidedStorageLocation getProvidedStorageLocation() {
+    return pair.getValue();
   }
 
-  public Path getPath() {
-    return path;
+  public String getBlockPoolId() {
+    return this.bpid;
   }
 
-  public long getOffset() {
-    return offset;
-  }
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
 
-  public long getLength() {
-    return length;
-  }
+    FileRegion that = (FileRegion) o;
 
-  public long getGenerationStamp() {
-    return genStamp;
+    return pair.equals(that.pair);
   }
 
   @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("{ block=\"").append(getBlock()).append("\"");
-    sb.append(", path=\"").append(getPath()).append("\"");
-    sb.append(", off=\"").append(getOffset()).append("\"");
-    sb.append(", len=\"").append(getBlock().getNumBytes()).append("\"");
-    sb.append(", genStamp=\"").append(getBlock()
-        .getGenerationStamp()).append("\"");
-    sb.append(", bpid=\"").append(bpid).append("\"");
-    sb.append(" }");
-    return sb.toString();
-  }
-
-  public String getBlockPoolId() {
-    return this.bpid;
+  public int hashCode() {
+    return pair.hashCode();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
index d276fb5..e3b6cb5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.common.blockaliasmap;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Iterator;
+import java.util.Optional;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.BlockAlias;
@@ -29,6 +31,19 @@ import org.apache.hadoop.hdfs.server.common.BlockAlias;
 public abstract class BlockAliasMap<T extends BlockAlias> {
 
   /**
+   * ImmutableIterator is an Iterator that does not support the remove
+   * operation. This could inherit {@link java.util.Enumeration} but Iterator
+   * is supported by more APIs and Enumeration's javadoc even suggests using
+   * Iterator instead.
+   */
+  public abstract class ImmutableIterator implements Iterator<T> {
+    public void remove() {
+      throw new UnsupportedOperationException(
+          "Remove is not supported for provided storage");
+    }
+  }
+
+  /**
    * An abstract class that is used to read {@link BlockAlias}es
    * for provided blocks.
    */
@@ -45,7 +60,7 @@ public abstract class BlockAliasMap<T extends BlockAlias> {
      * @return BlockAlias correspoding to the provided block.
      * @throws IOException
      */
-    public abstract U resolve(Block ident) throws IOException;
+    public abstract Optional<U> resolve(Block ident) throws IOException;
 
   }
 
@@ -85,4 +100,6 @@ public abstract class BlockAliasMap<T extends BlockAlias> {
    */
   public abstract void refresh() throws IOException;
 
+  public abstract void close() throws IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
new file mode 100644
index 0000000..7b0b789
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
@@ -0,0 +1,156 @@
+/*
+ * 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.common.blockaliasmap.impl;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocolPB.InMemoryAliasMapProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+
+/**
+ * InMemoryLevelDBAliasMapClient is the client for the InMemoryAliasMapServer.
+ * This is used by the Datanode and fs2img to store and retrieve FileRegions
+ * based on the given Block.
+ */
+public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
+    implements Configurable {
+
+  private Configuration conf;
+  private InMemoryAliasMapProtocolClientSideTranslatorPB aliasMap;
+
+  @Override
+  public void close() {
+    aliasMap.stop();
+  }
+
+  class LevelDbReader extends BlockAliasMap.Reader<FileRegion> {
+
+    @Override
+    public Optional<FileRegion> resolve(Block block) throws IOException {
+      Optional<ProvidedStorageLocation> read = aliasMap.read(block);
+      return read.map(psl -> new FileRegion(block, psl, null));
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    private class LevelDbIterator
+        extends BlockAliasMap<FileRegion>.ImmutableIterator {
+
+      private Iterator<FileRegion> iterator;
+      private Optional<Block> nextMarker;
+
+      LevelDbIterator()  {
+        batch(Optional.empty());
+      }
+
+      private void batch(Optional<Block> newNextMarker) {
+        try {
+          InMemoryAliasMap.IterationResult iterationResult =
+              aliasMap.list(newNextMarker);
+          List<FileRegion> fileRegions = iterationResult.getFileRegions();
+          this.iterator = fileRegions.iterator();
+          this.nextMarker = iterationResult.getNextBlock();
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public boolean hasNext() {
+        return iterator.hasNext() || nextMarker.isPresent();
+      }
+
+      @Override
+      public FileRegion next() {
+        if (iterator.hasNext()) {
+          return iterator.next();
+        } else {
+          if (nextMarker.isPresent()) {
+            batch(nextMarker);
+            return next();
+          } else {
+            throw new NoSuchElementException();
+          }
+        }
+      }
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      return new LevelDbIterator();
+    }
+  }
+
+  class LevelDbWriter extends BlockAliasMap.Writer<FileRegion> {
+    @Override
+    public void store(FileRegion fileRegion) throws IOException {
+      aliasMap.write(fileRegion.getBlock(),
+          fileRegion.getProvidedStorageLocation());
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+
+  InMemoryLevelDBAliasMapClient() {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      throw new UnsupportedOperationException("Unable to start "
+          + "InMemoryLevelDBAliasMapClient as security is enabled");
+    }
+  }
+
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+    return new LevelDbReader();
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    return new LevelDbWriter();
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    this.aliasMap = new InMemoryAliasMapProtocolClientSideTranslatorPB(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void refresh() throws IOException {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 bd04d60..b86b280 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
@@ -32,6 +32,7 @@ import java.util.Map;
 import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.NoSuchElementException;
+import java.util.Optional;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -40,6 +41,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.io.MultipleIOException;
@@ -160,7 +162,7 @@ public class TextFileRegionAliasMap
       file = new Path(tmpfile);
       delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
           DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
-      LOG.info("TextFileRegionAliasMap: read path " + tmpfile.toString());
+      LOG.info("TextFileRegionAliasMap: read path {}", tmpfile);
     }
 
     @Override
@@ -190,7 +192,7 @@ public class TextFileRegionAliasMap
     private Configuration conf;
     private String codec = null;
     private Path file =
-        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);;
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
     private String delim =
         DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
 
@@ -252,7 +254,7 @@ public class TextFileRegionAliasMap
       Options delimiter(String delim);
     }
 
-    static ReaderOptions defaults() {
+    public static ReaderOptions defaults() {
       return new ReaderOptions();
     }
 
@@ -278,14 +280,14 @@ public class TextFileRegionAliasMap
     }
 
     @Override
-    public FileRegion resolve(Block ident) throws IOException {
+    public Optional<FileRegion> resolve(Block ident) throws IOException {
       // consider layering index w/ composable format
       Iterator<FileRegion> i = iterator();
       try {
         while (i.hasNext()) {
           FileRegion f = i.next();
           if (f.getBlock().equals(ident)) {
-            return f;
+            return Optional.of(f);
           }
         }
       } finally {
@@ -295,7 +297,7 @@ public class TextFileRegionAliasMap
           r.close();
         }
       }
-      return null;
+      return Optional.empty();
     }
 
     class FRIterator implements Iterator<FileRegion> {
@@ -342,8 +344,8 @@ public class TextFileRegionAliasMap
         throw new IOException("Invalid line: " + line);
       }
       return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
-          Long.parseLong(f[2]), Long.parseLong(f[3]), f[5],
-          Long.parseLong(f[4]));
+          Long.parseLong(f[2]), Long.parseLong(f[3]), f[4],
+          Long.parseLong(f[5]));
     }
 
     public InputStream createStream() throws IOException {
@@ -390,7 +392,6 @@ public class TextFileRegionAliasMap
         throw MultipleIOException.createIOException(ex);
       }
     }
-
   }
 
   /**
@@ -422,12 +423,16 @@ public class TextFileRegionAliasMap
 
     @Override
     public void store(FileRegion token) throws IOException {
-      out.append(String.valueOf(token.getBlock().getBlockId())).append(delim);
-      out.append(token.getPath().toString()).append(delim);
-      out.append(Long.toString(token.getOffset())).append(delim);
-      out.append(Long.toString(token.getLength())).append(delim);
-      out.append(Long.toString(token.getGenerationStamp())).append(delim);
-      out.append(token.getBlockPoolId()).append("\n");
+      final Block block = token.getBlock();
+      final ProvidedStorageLocation psl = token.getProvidedStorageLocation();
+
+      out.append(String.valueOf(block.getBlockId())).append(delim);
+      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(token.getBlockPoolId()).append(delim);
+      out.append(Long.toString(block.getGenerationStamp())).append(delim);
+      out.append("\n");
     }
 
     @Override
@@ -443,4 +448,9 @@ public class TextFileRegionAliasMap
         "Refresh not supported by " + getClass());
   }
 
+  @Override
+  public void close() throws IOException {
+    //nothing to do;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 bcc9a38..0fbfc15 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
@@ -22,6 +22,7 @@ import java.net.URI;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+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;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -38,6 +39,16 @@ public class FinalizedProvidedReplica extends ProvidedReplica {
         remoteFS);
   }
 
+  public FinalizedProvidedReplica(FileRegion fileRegion, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
+    super(fileRegion.getBlock().getBlockId(),
+        fileRegion.getProvidedStorageLocation().getPath().toUri(),
+        fileRegion.getProvidedStorageLocation().getOffset(),
+        fileRegion.getBlock().getNumBytes(),
+        fileRegion.getBlock().getGenerationStamp(),
+        volume, conf, remoteFS);
+  }
+
   public FinalizedProvidedReplica(long blockId, Path pathPrefix,
       String pathSuffix, long fileOffset, long blockLen, long genStamp,
       FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 de68e2d..8748918 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
@@ -315,12 +315,7 @@ public class ReplicaBuilder {
             offset, length, genStamp, volume, conf, remoteFS);
       }
     } else {
-      info = new FinalizedProvidedReplica(fileRegion.getBlock().getBlockId(),
-          fileRegion.getPath().toUri(),
-          fileRegion.getOffset(),
-          fileRegion.getBlock().getNumBytes(),
-          fileRegion.getBlock().getGenerationStamp(),
-          volume, conf, remoteFS);
+      info = new FinalizedProvidedReplica(fileRegion, volume, conf, remoteFS);
     }
     return info;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 ab59fa5..6bbfa91 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
@@ -148,7 +148,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
       this.aliasMap = blockAliasMap;
     }
 
-    public void getVolumeMap(ReplicaMap volumeMap,
+    void fetchVolumeMap(ReplicaMap volumeMap,
         RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
         throws IOException {
       BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null);
@@ -157,21 +157,19 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
             + "; no blocks will be populated");
         return;
       }
-      Iterator<FileRegion> iter = reader.iterator();
       Path blockPrefixPath = new Path(providedVolume.getBaseURI());
-      while (iter.hasNext()) {
-        FileRegion region = iter.next();
+      for (FileRegion region : reader) {
         if (region.getBlockPoolId() != null
             && region.getBlockPoolId().equals(bpid)
             && containsBlock(providedVolume.baseURI,
-                region.getPath().toUri())) {
-          String blockSuffix =
-              getSuffix(blockPrefixPath, new Path(region.getPath().toUri()));
+                region.getProvidedStorageLocation().getPath().toUri())) {
+          String blockSuffix = getSuffix(blockPrefixPath,
+              new Path(region.getProvidedStorageLocation().getPath().toUri()));
           ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
               .setBlockId(region.getBlock().getBlockId())
               .setPathPrefix(blockPrefixPath)
               .setPathSuffix(blockSuffix)
-              .setOffset(region.getOffset())
+              .setOffset(region.getProvidedStorageLocation().getOffset())
               .setLength(region.getBlock().getNumBytes())
               .setGenerationStamp(region.getBlock().getGenerationStamp())
               .setFsVolume(providedVolume)
@@ -216,18 +214,12 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
        */
       aliasMap.refresh();
       BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null);
-      if (reader == null) {
-        LOG.warn("Got null reader from BlockAliasMap " + aliasMap
-            + "; no blocks will be populated in scan report");
-        return;
-      }
-      Iterator<FileRegion> iter = reader.iterator();
-      while(iter.hasNext()) {
+      for (FileRegion region : reader) {
         reportCompiler.throttle();
-        FileRegion region = iter.next();
         if (region.getBlockPoolId().equals(bpid)) {
           report.add(new ScanInfo(region.getBlock().getBlockId(),
-              providedVolume, region, region.getLength()));
+              providedVolume, region,
+              region.getProvidedStorageLocation().getLength()));
         }
       }
     }
@@ -522,7 +514,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
           throws IOException {
     LOG.info("Creating volumemap for provided volume " + this);
     for(ProvidedBlockPoolSlice s : bpSlices.values()) {
-      s.getVolumeMap(volumeMap, ramDiskReplicaMap, remoteFS);
+      s.fetchVolumeMap(volumeMap, ramDiskReplicaMap, remoteFS);
     }
   }
 
@@ -539,7 +531,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   void getVolumeMap(String bpid, ReplicaMap volumeMap,
       final RamDiskReplicaTracker ramDiskReplicaMap)
           throws IOException {
-    getProvidedBlockPoolSlice(bpid).getVolumeMap(volumeMap, ramDiskReplicaMap,
+    getProvidedBlockPoolSlice(bpid).fetchVolumeMap(volumeMap, ramDiskReplicaMap,
         remoteFS);
   }
 
@@ -601,7 +593,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   @Override
   public LinkedList<ScanInfo> compileReport(String bpid,
       LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
-          throws InterruptedException, IOException {
+      throws InterruptedException, IOException {
     LOG.info("Compiling report for volume: " + this + " bpid " + bpid);
     //get the report from the appropriate block pool.
     if(bpSlices.containsKey(bpid)) {
@@ -690,6 +682,12 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   }
 
   @VisibleForTesting
+  BlockAliasMap<FileRegion> getFileRegionProvider(String bpid) throws
+      IOException {
+    return getProvidedBlockPoolSlice(bpid).getBlockAliasMap();
+  }
+
+  @VisibleForTesting
   void setFileRegionProvider(String bpid,
       BlockAliasMap<FileRegion> blockAliasMap) throws IOException {
     ProvidedBlockPoolSlice bp = bpSlices.get(bpid);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 32b873b..993716a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -208,6 +210,8 @@ public class NameNode extends ReconfigurableBase implements
     HdfsConfiguration.init();
   }
 
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+
   /**
    * Categories of operations supported by the namenode.
    */
@@ -745,6 +749,20 @@ public class NameNode extends ReconfigurableBase implements
 
     startCommonServices(conf);
     startMetricsLogger(conf);
+    startAliasMapServerIfNecessary(conf);
+  }
+
+  private void startAliasMapServerIfNecessary(Configuration conf)
+      throws IOException {
+    if (conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED,
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT)
+        && conf.getBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED,
+            DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED_DEFAULT)) {
+      levelDBAliasMapServer =
+          new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+      levelDBAliasMapServer.setConf(conf);
+      levelDBAliasMapServer.start();
+    }
   }
 
   private void initReconfigurableBackoffKey() {
@@ -1027,6 +1045,9 @@ public class NameNode extends ReconfigurableBase implements
         MBeans.unregister(nameNodeStatusBeanName);
         nameNodeStatusBeanName = null;
       }
+      if (levelDBAliasMapServer != null) {
+        levelDBAliasMapServer.close();
+      }
     }
     tracer.close();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
new file mode 100644
index 0000000..08f10bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
@@ -0,0 +1,60 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "AliasMapProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message KeyValueProto {
+  optional BlockProto key = 1;
+  optional ProvidedStorageLocationProto value = 2;
+}
+
+message WriteRequestProto {
+  required KeyValueProto keyValuePair = 1;
+}
+
+message WriteResponseProto {
+}
+
+message ReadRequestProto {
+  required BlockProto key = 1;
+}
+
+message ReadResponseProto {
+  optional ProvidedStorageLocationProto value = 1;
+}
+
+message ListRequestProto {
+  optional BlockProto marker = 1;
+}
+
+message ListResponseProto {
+  repeated KeyValueProto fileRegions = 1;
+  optional BlockProto nextMarker = 2;
+}
+
+service AliasMapProtocolService {
+  rpc write(WriteRequestProto) returns(WriteResponseProto);
+  rpc read(ReadRequestProto) returns(ReadResponseProto);
+  rpc list(ListRequestProto) returns(ListResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 655f9cb..ddc07ac 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
@@ -4653,6 +4653,40 @@
   </property>
 
   <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>0.0.0.0:50200</value>
+    <description>
+      The address where the aliasmap server will be running
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.leveldb.dir</name>
+    <value>/tmp</value>
+    <description>
+      The directory where the leveldb files will be kept
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.enabled</name>
+    <value>false</value>
+    <description>
+      Don't use the aliasmap by default. Some tests will fail
+      because they try to start the namenode twice with the
+      same parameters if you turn it on.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.provided.aliasmap.text.delimiter</name>
     <value>,</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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
new file mode 100644
index 0000000..6f1ff3e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.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.hdfs.server.aliasmap;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Optional;
+
+/**
+ * ITestInMemoryAliasMap is an integration test that writes and reads to
+ * an AliasMap. This is an integration test because it can't be run in parallel
+ * like normal unit tests since there is conflict over the port being in use.
+ */
+public class ITestInMemoryAliasMap {
+  private InMemoryAliasMap aliasMap;
+  private File tempDirectory;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    tempDirectory = Files.createTempDirectory("seagull").toFile();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDirectory.getAbsolutePath());
+    aliasMap = InMemoryAliasMap.init(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    aliasMap.close();
+    FileUtils.deleteDirectory(tempDirectory);
+  }
+
+  @Test
+  public void readNotFoundReturnsNothing() throws IOException {
+    Block block = new Block(42, 43, 44);
+
+    Optional<ProvidedStorageLocation> actualProvidedStorageLocationOpt
+        = aliasMap.read(block);
+
+    assertFalse(actualProvidedStorageLocationOpt.isPresent());
+  }
+
+  @Test
+  public void readWrite() throws Exception {
+    Block block = new Block(42, 43, 44);
+
+    Path path = new Path("eagle", "mouse");
+    long offset = 47;
+    long length = 48;
+    int nonceSize = 4;
+    byte[] nonce = new byte[nonceSize];
+    Arrays.fill(nonce, 0, (nonceSize - 1), Byte.parseByte("0011", 2));
+
+    ProvidedStorageLocation expectedProvidedStorageLocation =
+        new ProvidedStorageLocation(path, offset, length, nonce);
+
+    aliasMap.write(block, expectedProvidedStorageLocation);
+
+    Optional<ProvidedStorageLocation> actualProvidedStorageLocationOpt
+        = aliasMap.read(block);
+
+    assertTrue(actualProvidedStorageLocationOpt.isPresent());
+    assertEquals(expectedProvidedStorageLocation,
+        actualProvidedStorageLocationOpt.get());
+
+  }
+
+  @Test
+  public void list() throws IOException {
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    Block block3 = new Block(44, 45, 46);
+
+    Path path = new Path("eagle", "mouse");
+    int nonceSize = 4;
+    byte[] nonce = new byte[nonceSize];
+    Arrays.fill(nonce, 0, (nonceSize - 1), Byte.parseByte("0011", 2));
+    ProvidedStorageLocation expectedProvidedStorageLocation1 =
+        new ProvidedStorageLocation(path, 47, 48, nonce);
+    ProvidedStorageLocation expectedProvidedStorageLocation2 =
+        new ProvidedStorageLocation(path, 48, 49, nonce);
+    ProvidedStorageLocation expectedProvidedStorageLocation3 =
+        new ProvidedStorageLocation(path, 49, 50, nonce);
+
+    aliasMap.write(block1, expectedProvidedStorageLocation1);
+    aliasMap.write(block2, expectedProvidedStorageLocation2);
+    aliasMap.write(block3, expectedProvidedStorageLocation3);
+
+    InMemoryAliasMap.IterationResult list = aliasMap.list(Optional.empty());
+    // we should have 3 results
+    assertEquals(3, list.getFileRegions().size());
+    // no more results expected
+    assertFalse(list.getNextBlock().isPresent());
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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
new file mode 100644
index 0000000..f699055
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
@@ -0,0 +1,45 @@
+/*
+ * 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.aliasmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+
+/**
+ * TestInMemoryAliasMap tests the initialization of an AliasMap. Most of the
+ * rest of the tests are in ITestInMemoryAliasMap since the tests are not
+ * thread safe (there is competition for the port).
+ */
+public class TestInMemoryAliasMap {
+
+  @Test
+  public void testInit() {
+    String nonExistingDirectory = "non-existing-directory";
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        nonExistingDirectory);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() -> InMemoryAliasMap.init(conf)).withMessage(
+            InMemoryAliasMap.createPathErrorMessage(nonExistingDirectory));
+  }
+}
\ No newline at end of file


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


[20/50] [abbrv] hadoop git commit: HDFS-12665. [AliasMap] Create a version of the AliasMap that runs in memory in the Namenode (leveldb). Contributed by Ewan Higgs.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 1ef2f2b..faf1f83 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.junit.Before;
 import org.junit.Test;
-
 import java.io.IOException;
 
 import static org.junit.Assert.assertNotNull;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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
new file mode 100644
index 0000000..4a9661b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
@@ -0,0 +1,341 @@
+/*
+ * 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.common.blockaliasmap.impl;
+
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+/**
+ * Tests the {@link InMemoryLevelDBAliasMapClient}.
+ */
+public class TestInMemoryLevelDBAliasMapClient {
+
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+  private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
+  private File tempDir;
+  private Configuration conf;
+
+  @Before
+  public void setUp() throws IOException {
+    levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+    conf = new Configuration();
+    int port = 9876;
+
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:" + port);
+    tempDir = Files.createTempDir();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDir.getAbsolutePath());
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    levelDBAliasMapServer.close();
+    inMemoryLevelDBAliasMapClient.close();
+    FileUtils.deleteDirectory(tempDir);
+  }
+
+  @Test
+  public void writeRead() throws Exception {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block = new Block(42, 43, 44);
+    byte[] nonce = "blackbird".getBytes();
+    ProvidedStorageLocation providedStorageLocation
+        = new ProvidedStorageLocation(new Path("cuckoo"),
+        45, 46, nonce);
+    BlockAliasMap.Writer<FileRegion> writer =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+    writer.store(new FileRegion(block, providedStorageLocation));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    Optional<FileRegion> fileRegion = reader.resolve(block);
+    assertEquals(new FileRegion(block, providedStorageLocation),
+        fileRegion.get());
+  }
+
+  @Test
+  public void iterateSingleBatch() throws Exception {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    byte[] nonce1 = "blackbird".getBytes();
+    byte[] nonce2 = "cuckoo".getBytes();
+    ProvidedStorageLocation providedStorageLocation1 =
+        new ProvidedStorageLocation(new Path("eagle"),
+        46, 47, nonce1);
+    ProvidedStorageLocation providedStorageLocation2 =
+        new ProvidedStorageLocation(new Path("falcon"),
+            46, 47, nonce2);
+    BlockAliasMap.Writer<FileRegion> writer1 =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+    writer1.store(new FileRegion(block1, providedStorageLocation1));
+    BlockAliasMap.Writer<FileRegion> writer2 =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+    writer2.store(new FileRegion(block2, providedStorageLocation2));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    List<FileRegion> actualFileRegions =
+        Lists.newArrayListWithCapacity(2);
+    for (FileRegion fileRegion : reader) {
+      actualFileRegions.add(fileRegion);
+    }
+
+    assertArrayEquals(
+        new FileRegion[] {new FileRegion(block1, providedStorageLocation1),
+            new FileRegion(block2, providedStorageLocation2)},
+        actualFileRegions.toArray());
+  }
+
+  @Test
+  public void iterateThreeBatches() throws Exception {
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE, "2");
+    levelDBAliasMapServer.setConf(conf);
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    Block block3 = new Block(44, 45, 46);
+    Block block4 = new Block(47, 48, 49);
+    Block block5 = new Block(50, 51, 52);
+    Block block6 = new Block(53, 54, 55);
+    byte[] nonce1 = "blackbird".getBytes();
+    byte[] nonce2 = "cuckoo".getBytes();
+    byte[] nonce3 = "sparrow".getBytes();
+    byte[] nonce4 = "magpie".getBytes();
+    byte[] nonce5 = "seagull".getBytes();
+    byte[] nonce6 = "finch".getBytes();
+    ProvidedStorageLocation providedStorageLocation1 =
+        new ProvidedStorageLocation(new Path("eagle"),
+            46, 47, nonce1);
+    ProvidedStorageLocation providedStorageLocation2 =
+        new ProvidedStorageLocation(new Path("falcon"),
+            48, 49, nonce2);
+    ProvidedStorageLocation providedStorageLocation3 =
+        new ProvidedStorageLocation(new Path("robin"),
+            50, 51, nonce3);
+    ProvidedStorageLocation providedStorageLocation4 =
+        new ProvidedStorageLocation(new Path("parakeet"),
+            52, 53, nonce4);
+    ProvidedStorageLocation providedStorageLocation5 =
+        new ProvidedStorageLocation(new Path("heron"),
+            54, 55, nonce5);
+    ProvidedStorageLocation providedStorageLocation6 =
+        new ProvidedStorageLocation(new Path("duck"),
+            56, 57, nonce6);
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block1, providedStorageLocation1));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block2, providedStorageLocation2));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block3, providedStorageLocation3));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block4, providedStorageLocation4));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block5, providedStorageLocation5));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block6, providedStorageLocation6));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    List<FileRegion> actualFileRegions =
+        Lists.newArrayListWithCapacity(6);
+    for (FileRegion fileRegion : reader) {
+      actualFileRegions.add(fileRegion);
+    }
+
+    FileRegion[] expectedFileRegions =
+        new FileRegion[] {new FileRegion(block1, providedStorageLocation1),
+            new FileRegion(block2, providedStorageLocation2),
+            new FileRegion(block3, providedStorageLocation3),
+            new FileRegion(block4, providedStorageLocation4),
+            new FileRegion(block5, providedStorageLocation5),
+            new FileRegion(block6, providedStorageLocation6)};
+    assertArrayEquals(expectedFileRegions, actualFileRegions.toArray());
+  }
+
+
+  class ReadThread implements Runnable {
+    private final Block block;
+    private final BlockAliasMap.Reader<FileRegion> reader;
+    private int delay;
+    private Optional<FileRegion> fileRegionOpt;
+
+    ReadThread(Block block, BlockAliasMap.Reader<FileRegion> reader,
+        int delay) {
+      this.block = block;
+      this.reader = reader;
+      this.delay = delay;
+    }
+
+    public Optional<FileRegion> getFileRegion() {
+      return fileRegionOpt;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(delay);
+        fileRegionOpt = reader.resolve(block);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  class WriteThread implements Runnable {
+    private final Block block;
+    private final BlockAliasMap.Writer<FileRegion> writer;
+    private final ProvidedStorageLocation providedStorageLocation;
+    private int delay;
+
+    WriteThread(Block block, ProvidedStorageLocation providedStorageLocation,
+        BlockAliasMap.Writer<FileRegion> writer, int delay) {
+      this.block = block;
+      this.writer = writer;
+      this.providedStorageLocation = providedStorageLocation;
+      this.delay = delay;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(delay);
+        writer.store(new FileRegion(block, providedStorageLocation));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public FileRegion generateRandomFileRegion(int seed) {
+    Block block = new Block(seed, seed + 1, seed + 2);
+    Path path = new Path("koekoek");
+    byte[] nonce = new byte[0];
+    ProvidedStorageLocation providedStorageLocation =
+        new ProvidedStorageLocation(path, seed + 3, seed + 4, nonce);
+    return new FileRegion(block, providedStorageLocation);
+  }
+
+  @Test
+  public void multipleReads() throws IOException {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+
+    Random r = new Random();
+    List<FileRegion> expectedFileRegions = r.ints(0, 200)
+        .limit(50)
+        .boxed()
+        .map(i -> generateRandomFileRegion(i))
+        .collect(Collectors.toList());
+
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    BlockAliasMap.Writer<FileRegion> writer =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+
+    ExecutorService executor = Executors.newCachedThreadPool();
+
+    List<ReadThread> readThreads = expectedFileRegions
+        .stream()
+        .map(fileRegion -> new ReadThread(fileRegion.getBlock(),
+            reader,
+            4000))
+        .collect(Collectors.toList());
+
+
+    List<? extends Future<?>> readFutures =
+        readThreads.stream()
+            .map(readThread -> executor.submit(readThread))
+            .collect(Collectors.toList());
+
+    List<? extends Future<?>> writeFutures = expectedFileRegions
+        .stream()
+        .map(fileRegion -> new WriteThread(fileRegion.getBlock(),
+            fileRegion.getProvidedStorageLocation(),
+            writer,
+            1000))
+        .map(writeThread -> executor.submit(writeThread))
+        .collect(Collectors.toList());
+
+    readFutures.stream()
+        .map(readFuture -> {
+          try {
+            return readFuture.get();
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          } catch (ExecutionException e) {
+            throw new RuntimeException(e);
+          }
+        })
+        .collect(Collectors.toList());
+
+    List<FileRegion> actualFileRegions = readThreads.stream()
+        .map(readThread -> readThread.getFileRegion().get())
+        .collect(Collectors.toList());
+
+    assertThat(actualFileRegions).containsExactlyInAnyOrder(
+        expectedFileRegions.toArray(new FileRegion[0]));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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
new file mode 100644
index 0000000..43fc68c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
@@ -0,0 +1,116 @@
+/*
+ * 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.common.blockaliasmap.impl;
+
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.iq80.leveldb.DBException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.File;
+import java.io.IOException;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests the in-memory alias map with a mock level-db implementation.
+ */
+public class TestLevelDbMockAliasMapClient {
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+  private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
+  private File tempDir;
+  private Configuration conf;
+  private InMemoryAliasMap aliasMapMock;
+
+  @Before
+  public void setUp() throws IOException {
+    aliasMapMock = mock(InMemoryAliasMap.class);
+    levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
+        config -> aliasMapMock);
+    conf = new Configuration();
+    int port = 9877;
+
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:" + port);
+    tempDir = Files.createTempDir();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDir.getAbsolutePath());
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    levelDBAliasMapServer.close();
+    inMemoryLevelDBAliasMapClient.close();
+    FileUtils.deleteDirectory(tempDir);
+  }
+
+  @Test
+  public void readFailure() throws Exception {
+    Block block = new Block(42, 43, 44);
+    doThrow(new IOException())
+        .doThrow(new DBException())
+        .when(aliasMapMock)
+        .read(block);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getReader(null).resolve(block));
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getReader(null).resolve(block));
+  }
+
+  @Test
+  public void writeFailure() throws IOException {
+    Block block = new Block(42, 43, 44);
+    byte[] nonce = new byte[0];
+    Path path = new Path("koekoek");
+    ProvidedStorageLocation providedStorageLocation =
+        new ProvidedStorageLocation(path, 45, 46, nonce);
+
+    doThrow(new IOException())
+        .when(aliasMapMock)
+        .write(block, providedStorageLocation);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getWriter(null)
+                .store(new FileRegion(block, providedStorageLocation)));
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getWriter(null)
+                .store(new FileRegion(block, providedStorageLocation)));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 4190730..8bdbaa4 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
@@ -43,6 +43,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
@@ -214,7 +215,8 @@ public class TestProvidedImpl {
             }
 
             @Override
-            public FileRegion resolve(Block ident) throws IOException {
+            public Optional<FileRegion> resolve(Block ident)
+                throws IOException {
               return null;
             }
           };
@@ -232,6 +234,11 @@ public class TestProvidedImpl {
     public void refresh() throws IOException {
       // do nothing!
     }
+
+    @Override
+    public void close() throws IOException {
+      // do nothing
+    }
   }
 
   private static Storage.StorageDirectory createLocalStorageDirectory(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 0866f3e..efc8c2d 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1336,7 +1336,6 @@
           <artifactId>mssql-jdbc</artifactId>
           <version>${mssql.version}</version>
         </dependency>
-
         <dependency>
           <groupId>io.swagger</groupId>
           <artifactId>swagger-annotations</artifactId>
@@ -1352,7 +1351,12 @@
           <artifactId>snakeyaml</artifactId>
           <version>${snakeyaml.version}</version>
         </dependency>
-
+        <dependency>
+          <groupId>org.assertj</groupId>
+          <artifactId>assertj-core</artifactId>
+          <version>3.8.0</version>
+          <scope>test</scope>
+        </dependency>
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-tools/hadoop-fs2img/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/pom.xml b/hadoop-tools/hadoop-fs2img/pom.xml
index e1411f8..8661c82 100644
--- a/hadoop-tools/hadoop-fs2img/pom.xml
+++ b/hadoop-tools/hadoop-fs2img/pom.xml
@@ -66,6 +66,12 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <version>3.8.0</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
index 4cdf473..63d1f27 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
+import java.util.Optional;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
@@ -57,14 +58,14 @@ public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
       }
 
       @Override
-      public FileRegion resolve(Block ident) throws IOException {
+      public Optional<FileRegion> resolve(Block ident) throws IOException {
         throw new UnsupportedOperationException();
       }
     };
   }
 
   @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+  public Writer getWriter(Writer.Options opts) throws IOException {
     return new Writer<FileRegion>() {
       @Override
       public void store(FileRegion token) throws IOException {
@@ -83,4 +84,8 @@ public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
     // do nothing
   }
 
+  @Override
+  public void close() throws IOException {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/352f994b/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 09e8f97..70e4c33 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
@@ -27,11 +27,13 @@ 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;
@@ -39,6 +41,7 @@ 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.BlockMissingException;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -48,6 +51,8 @@ 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;
@@ -56,6 +61,7 @@ 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;
 
@@ -172,16 +178,16 @@ public class TestNameNodeProvidedImplementation {
 
   void createImage(TreeWalk t, Path out,
       Class<? extends BlockResolver> blockIdsClass) throws Exception {
-    createImage(t, out, blockIdsClass, "");
+    createImage(t, out, blockIdsClass, "", TextFileRegionAliasMap.class);
   }
 
   void createImage(TreeWalk t, Path out,
-      Class<? extends BlockResolver> blockIdsClass, String clusterID)
-      throws Exception {
+      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(TextFileRegionAliasMap.class)
+        .blocks(aliasMapClass)
         .blockIds(blockIdsClass)
         .clusterID(clusterID);
     try (ImageWriter w = new ImageWriter(opts)) {
@@ -389,17 +395,8 @@ public class TestNameNodeProvidedImplementation {
     return ret;
   }
 
-  @Test(timeout=30000)
-  public void testBlockRead() throws Exception {
-    conf.setClass(ImageWriter.Options.UGI_CLASS,
-        FsUGIResolver.class, UGIResolver.class);
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3,
-        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
-        false);
+  private void verifyFileSystemContents() throws Exception {
     FileSystem fs = cluster.getFileSystem();
-    Thread.sleep(2000);
     int count = 0;
     // read NN metadata, verify contents match
     for (TreePath e : new FSTreeWalk(NAMEPATH, conf)) {
@@ -683,7 +680,7 @@ public class TestNameNodeProvidedImplementation {
   public void testSetClusterID() throws Exception {
     String clusterID = "PROVIDED-CLUSTER";
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class, clusterID);
+        FixedBlockResolver.class, clusterID, TextFileRegionAliasMap.class);
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
@@ -744,4 +741,42 @@ public class TestNameNodeProvidedImplementation {
       verifyFileLocation(i, expectedLocations);
     }
   }
+
+
+  // This test will fail until there is a refactoring of the FileRegion
+  // (HDFS-12713).
+  @Test(expected=BlockMissingException.class)
+  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();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDirectory.getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
+
+    InMemoryLevelDBAliasMapServer levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+    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);
+  }
+
 }


---------------------------------------------------------------------
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-11791. [READ] Test for increasing replication of provided files.

Posted by kk...@apache.org.
HDFS-11791. [READ] Test for increasing replication of provided files.


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

Branch: refs/heads/YARN-6592
Commit: 4851f06bc2df9d2cfc69fc7c4cecf7babcaa7728
Parents: 89b9faf
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed May 31 10:29:53 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../TestNameNodeProvidedImplementation.java     | 55 ++++++++++++++++++++
 1 file changed, 55 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4851f06b/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 5062439..e171557 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
@@ -23,6 +23,7 @@ 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;
@@ -34,10 +35,15 @@ 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.blockmanagement.BlockFormatProvider;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
 import org.apache.hadoop.hdfs.server.common.BlockFormat;
@@ -378,4 +384,53 @@ public class TestNameNodeProvidedImplementation {
     assertEquals(1, locations.length);
     assertEquals(2, locations[0].getHosts().length);
   }
+
+  private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
+      String filename, int expectedLocations) throws IOException {
+    LocatedBlocks locatedBlocks = client.getLocatedBlocks(
+        filename, 0, baseFileLen);
+    //given the start and length in the above call,
+    //only one LocatedBlock in LocatedBlocks
+    assertEquals(1, locatedBlocks.getLocatedBlocks().size());
+    LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
+    assertEquals(expectedLocations, locatedBlock.getLocations().length);
+    return locatedBlock.getLocations();
+  }
+
+  /**
+   * Tests setting replication of provided files.
+   * @throws Exception
+   */
+  @Test
+  public void testSetReplicationForProvidedFiles() throws Exception {
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    startCluster(NNDIRPATH, 2, null,
+        new StorageType[][] {
+                {StorageType.PROVIDED},
+                {StorageType.DISK}},
+        false);
+
+    String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
+    Path file = new Path(filename);
+    FileSystem fs = cluster.getFileSystem();
+
+    //set the replication to 2, and test that the file has
+    //the required replication.
+    fs.setReplication(file, (short) 2);
+    DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
+        file, (short) 2, 10000);
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+        cluster.getNameNodePort()), cluster.getConfiguration(0));
+    getAndCheckBlockLocations(client, filename, 2);
+
+    //set the replication back to 1
+    fs.setReplication(file, (short) 1);
+    DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
+        file, (short) 1, 10000);
+    //the only replica left should be the PROVIDED datanode
+    DatanodeInfo[] infos = getAndCheckBlockLocations(client, filename, 1);
+    assertEquals(cluster.getDataNodes().get(0).getDatanodeUuid(),
+        infos[0].getDatanodeUuid());
+  }
 }


---------------------------------------------------------------------
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: HDFS-12713. [READ] Refactor FileRegion and BlockAliasMap to separate out HDFS metadata and PROVIDED storage metadata. Contributed by Ewan Higgs

Posted by kk...@apache.org.
HDFS-12713. [READ] Refactor FileRegion and BlockAliasMap to separate out HDFS metadata and PROVIDED storage metadata. Contributed by Ewan Higgs


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

Branch: refs/heads/YARN-6592
Commit: 9c35be86e17021202823bfd3c2067ff3b312ce5c
Parents: a027055
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Dec 5 13:46:30 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 10 +--
 .../AliasMapProtocolServerSideTranslatorPB.java | 10 +++
 ...yAliasMapProtocolClientSideTranslatorPB.java | 17 ++++-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  2 +-
 .../hdfs/server/aliasmap/InMemoryAliasMap.java  |  7 +-
 .../aliasmap/InMemoryAliasMapProtocol.java      |  7 ++
 .../aliasmap/InMemoryLevelDBAliasMapServer.java | 13 +++-
 .../blockmanagement/ProvidedStorageMap.java     |  8 ++-
 .../hadoop/hdfs/server/common/FileRegion.java   | 30 ++------
 .../common/blockaliasmap/BlockAliasMap.java     | 14 ++--
 .../impl/InMemoryLevelDBAliasMapClient.java     | 24 ++++++-
 .../impl/LevelDBFileRegionAliasMap.java         | 22 ++++--
 .../impl/TextFileRegionAliasMap.java            | 76 ++++++++++++--------
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 41 ++++++-----
 .../hadoop/hdfs/server/namenode/NameNode.java   |  6 +-
 .../hdfs/server/protocol/NamespaceInfo.java     |  4 ++
 .../src/main/proto/AliasMapProtocol.proto       |  8 +++
 .../src/main/resources/hdfs-default.xml         | 23 +++++-
 .../blockmanagement/TestProvidedStorageMap.java |  4 +-
 .../impl/TestInMemoryLevelDBAliasMapClient.java | 41 +++++------
 .../impl/TestLevelDBFileRegionAliasMap.java     | 10 +--
 .../impl/TestLevelDbMockAliasMapClient.java     | 19 +++--
 .../impl/TestTextBlockAliasMap.java             | 55 +++++++++++---
 .../fsdataset/impl/TestProvidedImpl.java        |  9 ++-
 .../hdfs/server/namenode/FileSystemImage.java   |  4 ++
 .../hdfs/server/namenode/ImageWriter.java       | 14 +++-
 .../hdfs/server/namenode/NullBlockAliasMap.java |  6 +-
 .../hadoop/hdfs/server/namenode/TreePath.java   |  3 +-
 .../TestNameNodeProvidedImplementation.java     | 24 +++----
 29 files changed, 346 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 7db0a8d..2ef2bf0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -342,17 +342,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";
   public static final String DFS_PROVIDER_STORAGEUUID_DEFAULT =  "DS-PROVIDED";
   public static final String DFS_PROVIDED_ALIASMAP_CLASS = "dfs.provided.aliasmap.class";
+  public static final String DFS_PROVIDED_ALIASMAP_LOAD_RETRIES = "dfs.provided.aliasmap.load.retries";
 
   public static final String DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER = "dfs.provided.aliasmap.text.delimiter";
   public static final String DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT = ",";
 
-  public static final String DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH = "dfs.provided.aliasmap.text.read.path";
-  public static final String DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT = "file:///tmp/blocks.csv";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE = "dfs.provided.aliasmap.text.read.file";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE_DEFAULT = "file:///tmp/blocks.csv";
 
   public static final String DFS_PROVIDED_ALIASMAP_TEXT_CODEC = "dfs.provided.aliasmap.text.codec";
-  public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH = "dfs.provided.aliasmap.text.write.path";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR = "dfs.provided.aliasmap.text.write.dir";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT = "file:///tmp/";
 
-  public static final String DFS_PROVIDED_ALIASMAP_LEVELDB_PATH = "dfs.provided.aliasmap.leveldb.read.path";
+  public static final String DFS_PROVIDED_ALIASMAP_LEVELDB_PATH = "dfs.provided.aliasmap.leveldb.path";
 
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
index 15a1124..8d89c40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
@@ -121,4 +121,14 @@ public class AliasMapProtocolServerSideTranslatorPB
       throw new ServiceException(e);
     }
   }
+
+  public BlockPoolResponseProto getBlockPoolId(RpcController controller,
+      BlockPoolRequestProto req) throws ServiceException {
+    try {
+      String bpid = aliasMap.getBlockPoolId();
+      return BlockPoolResponseProto.newBuilder().setBlockPoolId(bpid).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
index 7f6cc91..fc23c88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
@@ -73,7 +73,8 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB
           RPC.getProtocolVersion(AliasMapProtocolPB.class), aliasMapAddr, null,
           conf, NetUtils.getDefaultSocketFactory(conf), 0);
     } catch (IOException e) {
-      e.printStackTrace();
+      throw new RuntimeException(
+          "Error in connecting to " + addr + " Got: " + e);
     }
   }
 
@@ -93,8 +94,7 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB
           .stream()
           .map(kv -> new FileRegion(
               PBHelperClient.convert(kv.getKey()),
-              PBHelperClient.convert(kv.getValue()),
-              null
+              PBHelperClient.convert(kv.getValue())
           ))
           .collect(Collectors.toList());
       BlockProto nextMarker = response.getNextMarker();
@@ -157,6 +157,17 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB
     }
   }
 
+  @Override
+  public String getBlockPoolId() throws IOException {
+    try {
+      BlockPoolResponseProto response = rpcProxy.getBlockPoolId(null,
+          BlockPoolRequestProto.newBuilder().build());
+      return response.getBlockPoolId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   public void stop() {
     RPC.stopProxy(rpcProxy);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 2952a5b..ac01348 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -1122,6 +1122,6 @@ public class PBHelper {
     ProvidedStorageLocation providedStorageLocation =
         PBHelperClient.convert(providedStorageLocationProto);
 
-    return new FileRegion(block, providedStorageLocation, null);
+    return new FileRegion(block, providedStorageLocation);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 f7be111..3d9eeea 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
@@ -121,7 +121,7 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
         Block block = fromBlockBytes(entry.getKey());
         ProvidedStorageLocation providedStorageLocation =
             fromProvidedStorageLocationBytes(entry.getValue());
-        batch.add(new FileRegion(block, providedStorageLocation, null));
+        batch.add(new FileRegion(block, providedStorageLocation));
         ++i;
       }
       if (iterator.hasNext()) {
@@ -157,6 +157,11 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     levelDb.put(extendedBlockDbFormat, providedStorageLocationDbFormat);
   }
 
+  @Override
+  public String getBlockPoolId() {
+    return null;
+  }
+
   public void close() throws IOException {
     levelDb.close();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
index c89c6b6..89f590c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
@@ -93,4 +93,11 @@ public interface InMemoryAliasMapProtocol {
   void write(@Nonnull Block block,
       @Nonnull ProvidedStorageLocation providedStorageLocation)
       throws IOException;
+
+  /**
+   * Get the associated block pool id.
+   * @return the block pool id associated with the Namenode running
+   * the in-memory alias map.
+   */
+  String getBlockPoolId() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 197e663..a229ae7 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
@@ -55,11 +55,13 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   private RPC.Server aliasMapServer;
   private Configuration conf;
   private InMemoryAliasMap aliasMap;
+  private String blockPoolId;
 
   public InMemoryLevelDBAliasMapServer(
-      CheckedFunction<Configuration, InMemoryAliasMap> initFun) {
+      CheckedFunction<Configuration, InMemoryAliasMap> initFun,
+      String blockPoolId) {
     this.initFun = initFun;
-
+    this.blockPoolId = blockPoolId;
   }
 
   public void start() throws IOException {
@@ -92,7 +94,7 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
         .setVerbose(true)
         .build();
 
-    LOG.info("Starting InMemoryLevelDBAliasMapServer on ", rpcAddress);
+    LOG.info("Starting InMemoryLevelDBAliasMapServer on {}", rpcAddress);
     aliasMapServer.start();
   }
 
@@ -117,6 +119,11 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   }
 
   @Override
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+
+  @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 f6b0c0b..7fbc71a 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
@@ -152,8 +152,12 @@ public class ProvidedStorageMap {
       LOG.info("Calling process first blk report from storage: "
           + providedStorageInfo);
       // first pass; periodic refresh should call bm.processReport
-      bm.processFirstBlockReport(providedStorageInfo,
-          new ProvidedBlockList(aliasMap.getReader(null).iterator()));
+      BlockAliasMap.Reader<BlockAlias> reader =
+          aliasMap.getReader(null, bm.getBlockPoolId());
+      if (reader != null) {
+        bm.processFirstBlockReport(providedStorageInfo,
+                new ProvidedBlockList(reader.iterator()));
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 c2697c8..e6f0d0a 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
@@ -34,39 +34,21 @@ import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 public class FileRegion implements BlockAlias {
 
   private final Pair<Block, ProvidedStorageLocation> pair;
-  private final String bpid;
 
   public FileRegion(long blockId, Path path, long offset,
-      long length, String bpid, long genStamp) {
+      long length, long genStamp) {
     this(new Block(blockId, length, genStamp),
-        new ProvidedStorageLocation(path, offset, length, new byte[0]), bpid);
+        new ProvidedStorageLocation(path, offset, length, new byte[0]));
   }
 
-  public FileRegion(long blockId, Path path, long offset,
-      long length, String bpid) {
-    this(blockId, path, offset, length, bpid,
+  public FileRegion(long blockId, Path path, long offset, long length) {
+    this(blockId, path, offset, length,
         HdfsConstants.GRANDFATHER_GENERATION_STAMP);
   }
 
-  public FileRegion(long blockId, Path path, long offset,
-      long length, long genStamp) {
-    this(blockId, path, offset, length, null, genStamp);
-  }
-
   public FileRegion(Block block,
       ProvidedStorageLocation providedStorageLocation) {
     this.pair  = Pair.of(block, providedStorageLocation);
-    this.bpid = null;
-  }
-
-  public FileRegion(Block block,
-      ProvidedStorageLocation providedStorageLocation, String bpid) {
-    this.pair  = Pair.of(block, providedStorageLocation);
-    this.bpid = bpid;
-  }
-
-  public FileRegion(long blockId, Path path, long offset, long length) {
-    this(blockId, path, offset, length, null);
   }
 
   public Block getBlock() {
@@ -77,10 +59,6 @@ public class FileRegion implements BlockAlias {
     return pair.getValue();
   }
 
-  public String getBlockPoolId() {
-    return this.bpid;
-  }
-
   @Override
   public boolean equals(Object o) {
     if (this == o) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
index 8233fbe..897aefd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
@@ -61,20 +61,22 @@ public abstract class BlockAliasMap<T extends BlockAlias> {
 
     /**
      * @param ident block to resolve
-     * @return BlockAlias correspoding to the provided block.
+     * @return BlockAlias corresponding to the provided block.
      * @throws IOException
      */
     public abstract Optional<U> resolve(Block ident) throws IOException;
-
   }
 
   /**
    * Returns a reader to the alias map.
    * @param opts reader options
-   * @return {@link Reader} to the alias map.
+   * @param blockPoolID block pool id to use
+   * @return {@link Reader} to the alias map. If a Reader for the blockPoolID
+   * cannot be created, this will return null.
    * @throws IOException
    */
-  public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
+  public abstract Reader<T> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException;
 
   /**
    * An abstract class used as a writer for the provided block map.
@@ -93,10 +95,12 @@ public abstract class BlockAliasMap<T extends BlockAlias> {
   /**
    * Returns the writer for the alias map.
    * @param opts writer options.
+   * @param blockPoolID block pool id to use
    * @return {@link Writer} to the alias map.
    * @throws IOException
    */
-  public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
+  public abstract Writer<T> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException;
 
   /**
    * Refresh the alias map.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
index bd6ebd1..d389184 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
@@ -46,6 +46,7 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
 
   private Configuration conf;
   private InMemoryAliasMapProtocolClientSideTranslatorPB aliasMap;
+  private String blockPoolID;
 
   @Override
   public void close() {
@@ -57,7 +58,7 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
     @Override
     public Optional<FileRegion> resolve(Block block) throws IOException {
       Optional<ProvidedStorageLocation> read = aliasMap.read(block);
-      return read.map(psl -> new FileRegion(block, psl, null));
+      return read.map(psl -> new FileRegion(block, psl));
     }
 
     @Override
@@ -133,12 +134,29 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
 
 
   @Override
-  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException {
+    if (this.blockPoolID == null) {
+      this.blockPoolID = aliasMap.getBlockPoolId();
+    }
+    // if a block pool id has been supplied, and doesn't match the associated
+    // block pool id, return null.
+    if (blockPoolID != null && this.blockPoolID != null
+        && !this.blockPoolID.equals(blockPoolID)) {
+      return null;
+    }
     return new LevelDbReader();
   }
 
   @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+  public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
+    if (this.blockPoolID == null) {
+      this.blockPoolID = aliasMap.getBlockPoolId();
+    }
+    if (blockPoolID != null && !this.blockPoolID.equals(blockPoolID)) {
+      return null;
+    }
     return new LevelDbWriter();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 6a3d55c..adad79a 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
@@ -70,7 +70,8 @@ public class LevelDBFileRegionAliasMap
   }
 
   @Override
-  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException {
     if (null == opts) {
       opts = this.opts;
     }
@@ -79,11 +80,12 @@ public class LevelDBFileRegionAliasMap
     }
     LevelDBOptions o = (LevelDBOptions) opts;
     return new LevelDBFileRegionAliasMap.LevelDBReader(
-        createDB(o.levelDBPath, false));
+        createDB(o.levelDBPath, false, blockPoolID));
   }
 
   @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+  public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
     if (null == opts) {
       opts = this.opts;
     }
@@ -92,11 +94,11 @@ public class LevelDBFileRegionAliasMap
     }
     LevelDBOptions o = (LevelDBOptions) opts;
     return new LevelDBFileRegionAliasMap.LevelDBWriter(
-        createDB(o.levelDBPath, true));
+        createDB(o.levelDBPath, true, blockPoolID));
   }
 
-  private static DB createDB(String levelDBPath, boolean createIfMissing)
-      throws IOException {
+  private static DB createDB(String levelDBPath, boolean createIfMissing,
+      String blockPoolID) throws IOException {
     if (levelDBPath == null || levelDBPath.length() == 0) {
       throw new IllegalArgumentException(
           "A valid path needs to be specified for "
@@ -105,7 +107,13 @@ public class LevelDBFileRegionAliasMap
     }
     org.iq80.leveldb.Options options = new org.iq80.leveldb.Options();
     options.createIfMissing(createIfMissing);
-    return factory.open(new File(levelDBPath), options);
+    File dbFile;
+    if (blockPoolID != null) {
+      dbFile = new File(levelDBPath, blockPoolID);
+    } else {
+      dbFile = new File(levelDBPath);
+    }
+    return factory.open(dbFile, options);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 69d9cc1..878a208 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
@@ -82,7 +82,7 @@ public class TextFileRegionAliasMap
   }
 
   @Override
-  public Reader<FileRegion> getReader(Reader.Options opts)
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
       throws IOException {
     if (null == opts) {
       opts = readerOpts;
@@ -94,23 +94,29 @@ public class TextFileRegionAliasMap
     Configuration readerConf = (null == o.getConf())
         ? new Configuration()
             : o.getConf();
-    return createReader(o.file, o.delim, readerConf);
+    return createReader(o.file, o.delim, readerConf, blockPoolID);
   }
 
   @VisibleForTesting
-  TextReader createReader(Path file, String delim, Configuration cfg)
-      throws IOException {
+  TextReader createReader(Path file, String delim, Configuration cfg,
+      String blockPoolID) throws IOException {
     FileSystem fs = file.getFileSystem(cfg);
     if (fs instanceof LocalFileSystem) {
       fs = ((LocalFileSystem)fs).getRaw();
     }
     CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
     CompressionCodec codec = factory.getCodec(file);
-    return new TextReader(fs, file, codec, delim);
+    String filename = fileNameFromBlockPoolID(blockPoolID);
+    if (codec != null) {
+      filename = filename + codec.getDefaultExtension();
+    }
+    Path bpidFilePath = new Path(file.getParent(), filename);
+    return new TextReader(fs, bpidFilePath, codec, delim);
   }
 
   @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+  public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
     if (null == opts) {
       opts = writerOpts;
     }
@@ -121,14 +127,15 @@ public class TextFileRegionAliasMap
     Configuration cfg = (null == o.getConf())
         ? new Configuration()
             : o.getConf();
+    String baseName = fileNameFromBlockPoolID(blockPoolID);
+    Path blocksFile = new Path(o.dir, baseName);
     if (o.codec != null) {
       CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
       CompressionCodec codec = factory.getCodecByName(o.codec);
-      String name = o.file.getName() + codec.getDefaultExtension();
-      o.filename(new Path(o.file.getParent(), name));
-      return createWriter(o.file, codec, o.delim, cfg);
+      blocksFile = new Path(o.dir, baseName + codec.getDefaultExtension());
+      return createWriter(blocksFile, codec, o.delim, cfg);
     }
-    return createWriter(o.file, null, o.delim, conf);
+    return createWriter(blocksFile, null, o.delim, conf);
   }
 
   @VisibleForTesting
@@ -154,15 +161,15 @@ public class TextFileRegionAliasMap
     private String delim =
         DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
     private Path file = new Path(
-        new File(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT).toURI()
-            .toString());
+        new File(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE_DEFAULT)
+            .toURI().toString());
 
     @Override
     public void setConf(Configuration conf) {
       this.conf = conf;
       String tmpfile =
-          conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
-              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
+          conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE_DEFAULT);
       file = new Path(tmpfile);
       delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
           DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
@@ -195,17 +202,17 @@ public class TextFileRegionAliasMap
 
     private Configuration conf;
     private String codec = null;
-    private Path file =
-        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
+    private Path dir =
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT);
     private String delim =
         DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
 
     @Override
     public void setConf(Configuration conf) {
       this.conf = conf;
-      String tmpfile = conf.get(
-          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH, file.toString());
-      file = new Path(tmpfile);
+      String tmpDir = conf.get(
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR, dir.toString());
+      dir = new Path(tmpDir);
       codec = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_CODEC);
       delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
           DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
@@ -217,8 +224,8 @@ public class TextFileRegionAliasMap
     }
 
     @Override
-    public WriterOptions filename(Path file) {
-      this.file = file;
+    public WriterOptions dirName(Path dir) {
+      this.dir = dir;
       return this;
     }
 
@@ -226,8 +233,8 @@ public class TextFileRegionAliasMap
       return codec;
     }
 
-    public Path getFile() {
-      return file;
+    public Path getDir() {
+      return dir;
     }
 
     @Override
@@ -267,6 +274,7 @@ public class TextFileRegionAliasMap
     private final FileSystem fs;
     private final CompressionCodec codec;
     private final Map<FRIterator, BufferedReader> iterators;
+    private final String blockPoolID;
 
     protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
         String delim) {
@@ -281,6 +289,7 @@ public class TextFileRegionAliasMap
       this.codec = codec;
       this.delim = delim;
       this.iterators = Collections.synchronizedMap(iterators);
+      this.blockPoolID = blockPoolIDFromFileName(file);
     }
 
     @Override
@@ -344,12 +353,11 @@ public class TextFileRegionAliasMap
         return null;
       }
       String[] f = line.split(delim);
-      if (f.length != 6) {
+      if (f.length != 5) {
         throw new IOException("Invalid line: " + line);
       }
       return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
-          Long.parseLong(f[2]), Long.parseLong(f[3]), f[4],
-          Long.parseLong(f[5]));
+          Long.parseLong(f[2]), Long.parseLong(f[3]), Long.parseLong(f[4]));
     }
 
     public InputStream createStream() throws IOException {
@@ -409,7 +417,7 @@ public class TextFileRegionAliasMap
      */
     public interface Options extends Writer.Options {
       Options codec(String codec);
-      Options filename(Path file);
+      Options dirName(Path dir);
       Options delimiter(String delim);
     }
 
@@ -434,7 +442,6 @@ 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(token.getBlockPoolId()).append(delim);
       out.append(Long.toString(block.getGenerationStamp())).append(delim);
       out.append("\n");
     }
@@ -457,4 +464,17 @@ public class TextFileRegionAliasMap
     //nothing to do;
   }
 
+  @VisibleForTesting
+  public static String blockPoolIDFromFileName(Path file) {
+    if (file == null) {
+      return "";
+    }
+    String fileName = file.getName();
+    return fileName.substring("blocks_".length()).split("\\.")[0];
+  }
+
+  @VisibleForTesting
+  public static String fileNameFromBlockPoolID(String blockPoolID) {
+    return "blocks_" + blockPoolID + ".csv";
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 445b455..bab788b 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
@@ -63,6 +63,8 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES;
+
 /**
  * This class is used to create provided volumes.
  */
@@ -125,6 +127,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
     private ReplicaMap bpVolumeMap;
     private ProvidedVolumeDF df;
     private AtomicLong numOfBlocks = new AtomicLong();
+    private int numRetries;
 
     ProvidedBlockPoolSlice(String bpid, ProvidedVolumeImpl volume,
         Configuration conf) {
@@ -138,6 +141,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
       this.bpid = bpid;
       this.df = new ProvidedVolumeDF();
       bpVolumeMap.initBlockPool(bpid);
+      this.numRetries = conf.getInt(DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 0);
       LOG.info("Created alias map using class: " + aliasMap.getClass());
     }
 
@@ -153,18 +157,27 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
     void fetchVolumeMap(ReplicaMap volumeMap,
         RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
         throws IOException {
-      BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null);
+      BlockAliasMap.Reader<FileRegion> reader = null;
+      int tries = 1;
+      do {
+        try {
+          reader = aliasMap.getReader(null, bpid);
+          break;
+        } catch (IOException e) {
+          tries++;
+          reader = null;
+        }
+      } while (tries <= numRetries);
+
       if (reader == null) {
-        LOG.warn("Got null reader from BlockAliasMap " + aliasMap
+        LOG.error("Got null reader from BlockAliasMap " + aliasMap
             + "; no blocks will be populated");
         return;
       }
       Path blockPrefixPath = new Path(providedVolume.getBaseURI());
       for (FileRegion region : reader) {
-        if (region.getBlockPoolId() != null
-            && region.getBlockPoolId().equals(bpid)
-            && containsBlock(providedVolume.baseURI,
-                region.getProvidedStorageLocation().getPath().toUri())) {
+        if (containsBlock(providedVolume.baseURI,
+            region.getProvidedStorageLocation().getPath().toUri())) {
           String blockSuffix = getSuffix(blockPrefixPath,
               new Path(region.getProvidedStorageLocation().getPath().toUri()));
           ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
@@ -215,14 +228,12 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
        * the ids remain the same.
        */
       aliasMap.refresh();
-      BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null);
+      BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null, bpid);
       for (FileRegion region : reader) {
         reportCompiler.throttle();
-        if (region.getBlockPoolId().equals(bpid)) {
-          report.add(new ScanInfo(region.getBlock().getBlockId(),
-              providedVolume, region,
-              region.getProvidedStorageLocation().getLength()));
-        }
+        report.add(new ScanInfo(region.getBlock().getBlockId(),
+            providedVolume, region,
+            region.getProvidedStorageLocation().getLength()));
       }
     }
 
@@ -415,9 +426,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
         if (temp.getBlock().getBlockId() < state.lastBlockId) {
           continue;
         }
-        if (temp.getBlockPoolId().equals(bpid)) {
-          nextRegion = temp;
-        }
+        nextRegion = temp;
       }
       if (null == nextRegion) {
         return null;
@@ -435,7 +444,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
     public void rewind() {
       BlockAliasMap.Reader<FileRegion> reader = null;
       try {
-        reader = blockAliasMap.getReader(null);
+        reader = blockAliasMap.getReader(null, bpid);
       } catch (IOException e) {
         LOG.warn("Exception in getting reader from provided alias map");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 993716a..a6f5790 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -729,6 +729,7 @@ public class NameNode extends ReconfigurableBase implements
     }
 
     loadNamesystem(conf);
+    startAliasMapServerIfNecessary(conf);
 
     rpcServer = createRpcServer(conf);
 
@@ -749,7 +750,6 @@ public class NameNode extends ReconfigurableBase implements
 
     startCommonServices(conf);
     startMetricsLogger(conf);
-    startAliasMapServerIfNecessary(conf);
   }
 
   private void startAliasMapServerIfNecessary(Configuration conf)
@@ -758,8 +758,8 @@ public class NameNode extends ReconfigurableBase implements
         DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT)
         && conf.getBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED,
             DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED_DEFAULT)) {
-      levelDBAliasMapServer =
-          new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+      levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
+          InMemoryAliasMap::init, namesystem.getBlockPoolId());
       levelDBAliasMapServer.setConf(conf);
       levelDBAliasMapServer.start();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index 433d9b7..10650da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -164,6 +164,10 @@ public class NamespaceInfo extends StorageInfo {
     this.clusterID = clusterID;
   }
 
+  public void setBlockPoolID(String blockPoolID) {
+    this.blockPoolID = blockPoolID;
+  }
+
   @Override
   public String toString(){
     return super.toString() + ";bpid=" + blockPoolID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
index 08f10bb..01dd952 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
@@ -53,8 +53,16 @@ message ListResponseProto {
   optional BlockProto nextMarker = 2;
 }
 
+message BlockPoolRequestProto {
+}
+
+message BlockPoolResponseProto {
+  required string blockPoolId = 1;
+}
+
 service AliasMapProtocolService {
   rpc write(WriteRequestProto) returns(WriteResponseProto);
   rpc read(ReadRequestProto) returns(ReadResponseProto);
   rpc list(ListRequestProto) returns(ListResponseProto);
+  rpc getBlockPoolId(BlockPoolRequestProto) returns(BlockPoolResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 ddc07ac..3dc583c 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
@@ -4696,7 +4696,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.aliasmap.text.read.path</name>
+    <name>dfs.provided.aliasmap.text.read.file</name>
     <value></value>
     <description>
         The path specifying the provided block map as a text file, specified as
@@ -4713,7 +4713,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.aliasmap.text.write.path</name>
+    <name>dfs.provided.aliasmap.text.write.dir</name>
     <value></value>
     <description>
         The path to which the provided block map should be written as a text
@@ -4722,6 +4722,25 @@
   </property>
 
   <property>
+    <name>dfs.provided.aliasmap.leveldb.path</name>
+    <value></value>
+    <description>
+      The read/write path for the leveldb-based alias map
+      (org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap).
+      The path has to be explicitly configured when this alias map is used.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.load.retries</name>
+    <value>0</value>
+    <description>
+      The number of retries on the Datanode to load the provided aliasmap;
+      defaults to 0.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.lock.suppress.warning.interval</name>
     <value>10s</value>
     <description>Instrumentation reporting long critical sections will suppress

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 faf1f83..b419c38 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
@@ -44,6 +44,7 @@ public class TestProvidedStorageMap {
   private BlockManager bm;
   private RwLock nameSystemLock;
   private String providedStorageID;
+  private String blockPoolID;
 
   @Before
   public void setup() {
@@ -55,8 +56,9 @@ public class TestProvidedStorageMap {
     conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
         TestProvidedImpl.TestFileRegionBlockAliasMap.class,
         BlockAliasMap.class);
-
+    blockPoolID = "BP-12344-10.1.1.2-12344";
     bm = mock(BlockManager.class);
+    when(bm.getBlockPoolId()).thenReturn(blockPoolID);
     nameSystemLock = mock(RwLock.class);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 4a9661b..a388199 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
@@ -56,11 +56,10 @@ public class TestInMemoryLevelDBAliasMapClient {
   private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
   private File tempDir;
   private Configuration conf;
+  private final static String BPID = "BPID-0";
 
   @Before
   public void setUp() throws IOException {
-    levelDBAliasMapServer =
-        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
     conf = new Configuration();
     int port = 9876;
 
@@ -69,6 +68,8 @@ public class TestInMemoryLevelDBAliasMapClient {
     tempDir = Files.createTempDir();
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
         tempDir.getAbsolutePath());
+    levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, BPID);
     inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
   }
 
@@ -81,20 +82,20 @@ public class TestInMemoryLevelDBAliasMapClient {
 
   @Test
   public void writeRead() throws Exception {
-    inMemoryLevelDBAliasMapClient.setConf(conf);
     levelDBAliasMapServer.setConf(conf);
     levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
     Block block = new Block(42, 43, 44);
     byte[] nonce = "blackbird".getBytes();
     ProvidedStorageLocation providedStorageLocation
         = new ProvidedStorageLocation(new Path("cuckoo"),
         45, 46, nonce);
     BlockAliasMap.Writer<FileRegion> writer =
-        inMemoryLevelDBAliasMapClient.getWriter(null);
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
     writer.store(new FileRegion(block, providedStorageLocation));
 
     BlockAliasMap.Reader<FileRegion> reader =
-        inMemoryLevelDBAliasMapClient.getReader(null);
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
     Optional<FileRegion> fileRegion = reader.resolve(block);
     assertEquals(new FileRegion(block, providedStorageLocation),
         fileRegion.get());
@@ -102,9 +103,9 @@ public class TestInMemoryLevelDBAliasMapClient {
 
   @Test
   public void iterateSingleBatch() throws Exception {
-    inMemoryLevelDBAliasMapClient.setConf(conf);
     levelDBAliasMapServer.setConf(conf);
     levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
     Block block1 = new Block(42, 43, 44);
     Block block2 = new Block(43, 44, 45);
     byte[] nonce1 = "blackbird".getBytes();
@@ -116,14 +117,14 @@ public class TestInMemoryLevelDBAliasMapClient {
         new ProvidedStorageLocation(new Path("falcon"),
             46, 47, nonce2);
     BlockAliasMap.Writer<FileRegion> writer1 =
-        inMemoryLevelDBAliasMapClient.getWriter(null);
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
     writer1.store(new FileRegion(block1, providedStorageLocation1));
     BlockAliasMap.Writer<FileRegion> writer2 =
-        inMemoryLevelDBAliasMapClient.getWriter(null);
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
     writer2.store(new FileRegion(block2, providedStorageLocation2));
 
     BlockAliasMap.Reader<FileRegion> reader =
-        inMemoryLevelDBAliasMapClient.getReader(null);
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
     List<FileRegion> actualFileRegions =
         Lists.newArrayListWithCapacity(2);
     for (FileRegion fileRegion : reader) {
@@ -140,8 +141,8 @@ public class TestInMemoryLevelDBAliasMapClient {
   public void iterateThreeBatches() throws Exception {
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE, "2");
     levelDBAliasMapServer.setConf(conf);
-    inMemoryLevelDBAliasMapClient.setConf(conf);
     levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
     Block block1 = new Block(42, 43, 44);
     Block block2 = new Block(43, 44, 45);
     Block block3 = new Block(44, 45, 46);
@@ -173,26 +174,26 @@ public class TestInMemoryLevelDBAliasMapClient {
         new ProvidedStorageLocation(new Path("duck"),
             56, 57, nonce6);
     inMemoryLevelDBAliasMapClient
-        .getWriter(null)
+        .getWriter(null, BPID)
         .store(new FileRegion(block1, providedStorageLocation1));
     inMemoryLevelDBAliasMapClient
-        .getWriter(null)
+        .getWriter(null, BPID)
         .store(new FileRegion(block2, providedStorageLocation2));
     inMemoryLevelDBAliasMapClient
-        .getWriter(null)
+        .getWriter(null, BPID)
         .store(new FileRegion(block3, providedStorageLocation3));
     inMemoryLevelDBAliasMapClient
-        .getWriter(null)
+        .getWriter(null, BPID)
         .store(new FileRegion(block4, providedStorageLocation4));
     inMemoryLevelDBAliasMapClient
-        .getWriter(null)
+        .getWriter(null, BPID)
         .store(new FileRegion(block5, providedStorageLocation5));
     inMemoryLevelDBAliasMapClient
-        .getWriter(null)
+        .getWriter(null, BPID)
         .store(new FileRegion(block6, providedStorageLocation6));
 
     BlockAliasMap.Reader<FileRegion> reader =
-        inMemoryLevelDBAliasMapClient.getReader(null);
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
     List<FileRegion> actualFileRegions =
         Lists.newArrayListWithCapacity(6);
     for (FileRegion fileRegion : reader) {
@@ -278,9 +279,9 @@ public class TestInMemoryLevelDBAliasMapClient {
 
   @Test
   public void multipleReads() throws IOException {
-    inMemoryLevelDBAliasMapClient.setConf(conf);
     levelDBAliasMapServer.setConf(conf);
     levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
 
     Random r = new Random();
     List<FileRegion> expectedFileRegions = r.ints(0, 200)
@@ -291,9 +292,9 @@ public class TestInMemoryLevelDBAliasMapClient {
 
 
     BlockAliasMap.Reader<FileRegion> reader =
-        inMemoryLevelDBAliasMapClient.getReader(null);
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
     BlockAliasMap.Writer<FileRegion> writer =
-        inMemoryLevelDBAliasMapClient.getWriter(null);
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
 
     ExecutorService executor = Executors.newCachedThreadPool();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
index 21199e1..a3c13e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
@@ -35,6 +35,8 @@ import static org.junit.Assert.assertNotEquals;
  */
 public class TestLevelDBFileRegionAliasMap {
 
+  private static final String BPID = "BPID-0";
+
   /**
    * A basic test to verify that we can write data and read it back again.
    * @throws Exception
@@ -48,13 +50,13 @@ public class TestLevelDBFileRegionAliasMap {
       LevelDBFileRegionAliasMap.LevelDBOptions opts =
           new LevelDBFileRegionAliasMap.LevelDBOptions()
               .filename(dbFile.getAbsolutePath());
-      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts);
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts, BPID);
 
       FileRegion fr = new FileRegion(1, new Path("/file"), 1, 1, 1);
       writer.store(fr);
       writer.close();
 
-      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts);
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts, BPID);
       FileRegion fr2 = reader.resolve(new Block(1, 1, 1)).get();
       assertEquals(fr, fr2);
       reader.close();
@@ -86,14 +88,14 @@ public class TestLevelDBFileRegionAliasMap {
       LevelDBFileRegionAliasMap.LevelDBOptions opts =
           new LevelDBFileRegionAliasMap.LevelDBOptions()
               .filename(dbFile.getAbsolutePath());
-      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts);
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts, BPID);
 
       for (FileRegion fr : regions) {
         writer.store(fr);
       }
       writer.close();
 
-      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts);
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts, BPID);
       Iterator<FileRegion> it = reader.iterator();
       int last = -1;
       int count = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 43fc68c..8212b28 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
@@ -36,6 +36,7 @@ import java.io.IOException;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests the in-memory alias map with a mock level-db implementation.
@@ -46,12 +47,14 @@ public class TestLevelDbMockAliasMapClient {
   private File tempDir;
   private Configuration conf;
   private InMemoryAliasMap aliasMapMock;
+  private final String bpid = "BPID-0";
 
   @Before
   public void setUp() throws IOException {
     aliasMapMock = mock(InMemoryAliasMap.class);
+    when(aliasMapMock.getBlockPoolId()).thenReturn(bpid);
     levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
-        config -> aliasMapMock);
+        config -> aliasMapMock, bpid);
     conf = new Configuration();
     int port = 9877;
 
@@ -60,10 +63,10 @@ public class TestLevelDbMockAliasMapClient {
     tempDir = Files.createTempDir();
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
         tempDir.getAbsolutePath());
-    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
-    inMemoryLevelDBAliasMapClient.setConf(conf);
     levelDBAliasMapServer.setConf(conf);
     levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
   }
 
   @After
@@ -83,11 +86,13 @@ public class TestLevelDbMockAliasMapClient {
 
     assertThatExceptionOfType(IOException.class)
         .isThrownBy(() ->
-            inMemoryLevelDBAliasMapClient.getReader(null).resolve(block));
+            inMemoryLevelDBAliasMapClient.getReader(null, bpid)
+                .resolve(block));
 
     assertThatExceptionOfType(IOException.class)
         .isThrownBy(() ->
-            inMemoryLevelDBAliasMapClient.getReader(null).resolve(block));
+            inMemoryLevelDBAliasMapClient.getReader(null, bpid)
+                .resolve(block));
   }
 
   @Test
@@ -104,12 +109,12 @@ public class TestLevelDbMockAliasMapClient {
 
     assertThatExceptionOfType(IOException.class)
         .isThrownBy(() ->
-            inMemoryLevelDBAliasMapClient.getWriter(null)
+            inMemoryLevelDBAliasMapClient.getWriter(null, bpid)
                 .store(new FileRegion(block, providedStorageLocation)));
 
     assertThatExceptionOfType(IOException.class)
         .isThrownBy(() ->
-            inMemoryLevelDBAliasMapClient.getWriter(null)
+            inMemoryLevelDBAliasMapClient.getWriter(null, bpid)
                 .store(new FileRegion(block, providedStorageLocation)));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
index 79308a3..29c53e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
@@ -31,7 +31,10 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.compress.CompressionCodec;
 
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.junit.Test;
+
+import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
 import static org.junit.Assert.*;
 
 /**
@@ -39,7 +42,10 @@ import static org.junit.Assert.*;
  */
 public class TestTextBlockAliasMap {
 
-  static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
+  static final String OUTFILE_PATH = "hdfs://dummyServer:0000/";
+  static final String OUTFILE_BASENAME = "dummyFile";
+  static final Path OUTFILE = new Path(OUTFILE_PATH, OUTFILE_BASENAME + "txt");
+  static final String BPID = "BPID-0";
 
   void check(TextWriter.Options opts, final Path vp,
       final Class<? extends CompressionCodec> vc) throws IOException {
@@ -56,7 +62,25 @@ public class TestTextBlockAliasMap {
         return null; // ignored
       }
     };
-    mFmt.getWriter(opts);
+    mFmt.getWriter(opts, BPID);
+  }
+
+  void check(TextReader.Options opts, final Path vp,
+      final Class<? extends CompressionCodec> vc) throws IOException {
+    TextFileRegionAliasMap aliasMap = new TextFileRegionAliasMap() {
+      @Override
+      public TextReader createReader(Path file, String delim, Configuration cfg,
+          String blockPoolID) throws IOException {
+        assertEquals(vp, file);
+        if (null != vc) {
+          CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+          CompressionCodec codec = factory.getCodec(file);
+          assertEquals(vc, codec.getClass());
+        }
+        return null; // ignored
+      }
+    };
+    aliasMap.getReader(opts, BPID);
   }
 
   @Test
@@ -64,18 +88,33 @@ public class TestTextBlockAliasMap {
     TextWriter.Options opts = TextWriter.defaults();
     assertTrue(opts instanceof WriterOptions);
     WriterOptions wopts = (WriterOptions) opts;
-    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
-    assertEquals(def, wopts.getFile());
+    Path def =
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT);
+    assertEquals(def, wopts.getDir());
     assertNull(wopts.getCodec());
 
-    opts.filename(OUTFILE);
-    check(opts, OUTFILE, null);
+    Path cp = new Path(OUTFILE_PATH, "blocks_" + BPID + ".csv");
+    opts.dirName(new Path(OUTFILE_PATH));
+    check(opts, cp, null);
 
-    opts.filename(OUTFILE);
     opts.codec("gzip");
-    Path cp = new Path(OUTFILE.getParent(), OUTFILE.getName() + ".gz");
+    cp = new Path(OUTFILE_PATH, "blocks_" + BPID + ".csv.gz");
     check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
+  }
 
+  @Test
+  public void testReaderOptions() throws Exception {
+    TextReader.Options opts = TextReader.defaults();
+    assertTrue(opts instanceof ReaderOptions);
+    ReaderOptions ropts = (ReaderOptions) opts;
+
+    Path cp = new Path(OUTFILE_PATH, fileNameFromBlockPoolID(BPID));
+    opts.filename(cp);
+    check(opts, cp, null);
+
+    cp = new Path(OUTFILE_PATH, "blocks_" + BPID + ".csv.gz");
+    opts.filename(cp);
+    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 8bdbaa4..1a89f76 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
@@ -150,7 +150,7 @@ public class TestProvidedImpl {
           }
         }
         region = new FileRegion(currentCount, new Path(newFile.toString()),
-            0, BLK_LEN, BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+            0, BLK_LEN);
         currentCount++;
       }
       return region;
@@ -194,9 +194,12 @@ public class TestProvidedImpl {
     }
 
     @Override
-    public Reader<FileRegion> getReader(Reader.Options opts)
+    public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolId)
         throws IOException {
 
+      if (!blockPoolId.equals(BLOCK_POOL_IDS[CHOSEN_BP_ID])) {
+        return null;
+      }
       BlockAliasMap.Reader<FileRegion> reader =
           new BlockAliasMap.Reader<FileRegion>() {
             @Override
@@ -224,7 +227,7 @@ public class TestProvidedImpl {
     }
 
     @Override
-    public Writer<FileRegion> getWriter(Writer.Options opts)
+    public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolId)
         throws IOException {
       // not implemented
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 4598e9c..80bbaf9 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
@@ -73,6 +73,7 @@ public class FileSystemImage implements Tool {
     options.addOption("i", "blockidclass", true, "Block resolver class");
     options.addOption("c", "cachedirs", true, "Max active dirents");
     options.addOption("cid", "clusterID", true, "Cluster ID");
+    options.addOption("bpid", "blockPoolID", true, "Block Pool ID");
     options.addOption("h", "help", false, "Print usage");
     return options;
   }
@@ -120,6 +121,9 @@ public class FileSystemImage implements Tool {
       case "cid":
         opts.clusterID(o.getValue());
         break;
+      case "bpid":
+        opts.blockPoolID(o.getValue());
+        break;
       default:
         throw new UnsupportedOperationException("Internal error");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 370c683..282429a 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
@@ -134,6 +134,11 @@ public class ImageWriter implements Closeable {
         if (opts.clusterID.length() > 0) {
           info.setClusterID(opts.clusterID);
         }
+        // if block pool id is given
+        if (opts.blockPoolID.length() > 0) {
+          info.setBlockPoolID(opts.blockPoolID);
+        }
+
         stor.format(info);
         blockPoolID = info.getBlockPoolID();
       }
@@ -165,7 +170,7 @@ public class ImageWriter implements Closeable {
     BlockAliasMap<FileRegion> fmt = null == opts.blocks
         ? ReflectionUtils.newInstance(opts.aliasMap, opts.getConf())
         : opts.blocks;
-    blocks = fmt.getWriter(null);
+    blocks = fmt.getWriter(null, blockPoolID);
     blockIds = null == opts.blockIds
         ? ReflectionUtils.newInstance(opts.blockIdsClass, opts.getConf())
         : opts.blockIds;
@@ -525,6 +530,7 @@ public class ImageWriter implements Closeable {
     private Class<? extends UGIResolver> ugisClass;
     private BlockAliasMap<FileRegion> blocks;
     private String clusterID;
+    private String blockPoolID;
 
     @SuppressWarnings("rawtypes")
     private Class<? extends BlockAliasMap> aliasMap;
@@ -552,6 +558,7 @@ public class ImageWriter implements Closeable {
       blockIdsClass = conf.getClass(BLOCK_RESOLVER_CLASS,
           FixedBlockResolver.class, BlockResolver.class);
       clusterID = "";
+      blockPoolID = "";
     }
 
     @Override
@@ -614,6 +621,11 @@ public class ImageWriter implements Closeable {
       this.clusterID = clusterID;
       return this;
     }
+
+    public Options blockPoolID(String blockPoolID) {
+      this.blockPoolID = blockPoolID;
+      return this;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
index fcaaf44..41e202d 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
@@ -36,7 +36,8 @@ import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
 
   @Override
-  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException {
     return new Reader<FileRegion>() {
       @Override
       public Iterator<FileRegion> iterator() {
@@ -69,7 +70,8 @@ public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
   }
 
   @Override
-  public Writer getWriter(Writer.Options opts) throws IOException {
+  public Writer getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
     return new Writer<FileRegion>() {
       @Override
       public void store(FileRegion token) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 8f1382a..aca1220 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
@@ -107,8 +107,7 @@ public class TreePath {
       long genStamp, String blockPoolID,
       BlockAliasMap.Writer<FileRegion> out) throws IOException {
     FileStatus s = getFileStatus();
-    out.store(new FileRegion(blockId, s.getPath(), offset, length,
-        blockPoolID, genStamp));
+    out.store(new FileRegion(blockId, s.getPath(), offset, length, genStamp));
   }
 
   INode toFile(UGIResolver ugi, BlockResolver blk,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c35be86/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 70e4c33..deaf9d5 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
@@ -41,7 +41,6 @@ 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.BlockMissingException;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -79,6 +78,7 @@ 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.*;
 
@@ -93,7 +93,6 @@ public class TestNameNodeProvidedImplementation {
   final Path BASE = new Path(fBASE.toURI().toString());
   final Path NAMEPATH = new Path(BASE, "providedDir");
   final Path NNDIRPATH = new Path(BASE, "nnDir");
-  final Path BLOCKFILE = new Path(NNDIRPATH, "blocks.csv");
   final String SINGLEUSER = "usr1";
   final String SINGLEGROUP = "grp1";
   private final int numFiles = 10;
@@ -101,6 +100,7 @@ public class TestNameNodeProvidedImplementation {
   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;
@@ -123,10 +123,10 @@ public class TestNameNodeProvidedImplementation {
 
     conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
         TextFileRegionAliasMap.class, BlockAliasMap.class);
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH,
-        BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
-        BLOCKFILE.toString());
+    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,
@@ -189,14 +189,14 @@ public class TestNameNodeProvidedImplementation {
     opts.output(out.toString())
         .blocks(aliasMapClass)
         .blockIds(blockIdsClass)
-        .clusterID(clusterID);
+        .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,
@@ -743,9 +743,7 @@ public class TestNameNodeProvidedImplementation {
   }
 
 
-  // This test will fail until there is a refactoring of the FileRegion
-  // (HDFS-12713).
-  @Test(expected=BlockMissingException.class)
+  @Test
   public void testInMemoryAliasMap() throws Exception {
     conf.setClass(ImageWriter.Options.UGI_CLASS,
         FsUGIResolver.class, UGIResolver.class);
@@ -758,9 +756,9 @@ public class TestNameNodeProvidedImplementation {
     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);
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, bpid);
     levelDBAliasMapServer.setConf(conf);
     levelDBAliasMapServer.start();
 


---------------------------------------------------------------------
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: YARN-7448. [API] Add SchedulingRequest to the AllocateRequest. (Panagiotis Garefalakis via asuresh)

Posted by kk...@apache.org.
YARN-7448. [API] Add SchedulingRequest to the AllocateRequest. (Panagiotis Garefalakis via asuresh)


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

Branch: refs/heads/YARN-6592
Commit: ca28a795ce9738471a20c81bf9a245bc70b7cffc
Parents: 3b9faf5
Author: Arun Suresh <as...@apache.org>
Authored: Fri Nov 17 10:42:43 2017 -0800
Committer: Konstantinos Karanasos <kk...@apache.org>
Committed: Mon Dec 18 16:07:00 2017 -0800

----------------------------------------------------------------------
 .../api/protocolrecords/AllocateRequest.java    | 42 ++++++++++
 .../hadoop/yarn/api/records/ResourceSizing.java | 27 +++++++
 .../yarn/api/records/SchedulingRequest.java     |  1 +
 .../src/main/proto/yarn_service_protos.proto    |  1 +
 .../impl/pb/AllocateRequestPBImpl.java          | 83 ++++++++++++++++++++
 .../records/impl/pb/ResourceSizingPBImpl.java   |  2 +-
 .../impl/pb/SchedulingRequestPBImpl.java        | 16 ++++
 .../hadoop/yarn/api/TestPBImplRecords.java      | 19 +++++
 8 files changed, 190 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
index ae0891e..d8d2347 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -28,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -212,6 +214,32 @@ public abstract class AllocateRequest {
   public abstract void setUpdateRequests(
       List<UpdateContainerRequest> updateRequests);
 
+  /**
+   * Get the list of Scheduling requests being sent by the
+   * <code>ApplicationMaster</code>.
+   * @return list of {@link SchedulingRequest} being sent by the
+   *         <code>ApplicationMaster</code>.
+   */
+  @Public
+  @Unstable
+  public List<SchedulingRequest> getSchedulingRequests() {
+    return Collections.EMPTY_LIST;
+  }
+
+  /**
+   * Set the list of Scheduling requests to inform the
+   * <code>ResourceManager</code> about the application's resource requirements
+   * (potentially including allocation tags & placement constraints).
+   * @param schedulingRequests list of <code>SchedulingRequest</code> to update
+   *          the <code>ResourceManager</code> about the application's resource
+   *          requirements.
+   */
+  @Public
+  @Unstable
+  public void setSchedulingRequests(
+      List<SchedulingRequest> schedulingRequests) {
+  }
+
   @Public
   @Unstable
   public static AllocateRequestBuilder newBuilder() {
@@ -314,6 +342,20 @@ public abstract class AllocateRequest {
     }
 
     /**
+     * Set the <code>schedulingRequests</code> of the request.
+     * @see AllocateRequest#setSchedulingRequests(List)
+     * @param schedulingRequests <code>SchedulingRequest</code> of the request
+     * @return {@link AllocateRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public AllocateRequestBuilder schedulingRequests(
+        List<SchedulingRequest> schedulingRequests) {
+      allocateRequest.setSchedulingRequests(schedulingRequests);
+      return this;
+    }
+
+    /**
      * Return generated {@link AllocateRequest} object.
      * @return {@link AllocateRequest}
      */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
index d82be11..8cdc63f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceSizing.java
@@ -61,4 +61,31 @@ public abstract class ResourceSizing {
   @Public
   @Unstable
   public abstract void setResources(Resource resources);
+
+  @Override
+  public int hashCode() {
+    int result = getResources().hashCode();
+    result = 31 * result + getNumAllocations();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if(obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+
+    ResourceSizing that = (ResourceSizing) obj;
+
+    if(getNumAllocations() != that.getNumAllocations()) {
+      return  false;
+    }
+    if(!getResources().equals(that.getResources())) {
+      return false;
+    }
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
index 47a0697..e32dd24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SchedulingRequest.java
@@ -49,6 +49,7 @@ public abstract class SchedulingRequest {
     return SchedulingRequest.newBuilder()
         .allocationRequestId(allocationRequestId).priority(priority)
         .executionType(executionType).allocationTags(allocationTags)
+        .resourceSizing(resourceSizing)
         .placementConstraintExpression(placementConstraintExpression).build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 68e585d..e49c4e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -91,6 +91,7 @@ message AllocateRequestProto {
   optional int32 response_id = 4;
   optional float progress = 5;
   repeated UpdateContainerRequestProto update_requests = 7;
+  repeated SchedulingRequestProto scheduling_requests = 10;
 }
 
 message NMTokenProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
index 0f0f571..b460044 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
@@ -29,14 +29,17 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.UpdateContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProtoOrBuilder;
@@ -53,6 +56,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private List<ResourceRequest> ask = null;
   private List<ContainerId> release = null;
   private List<UpdateContainerRequest> updateRequests = null;
+  private List<SchedulingRequest> schedulingRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
   
   public AllocateRequestPBImpl() {
@@ -101,6 +105,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.updateRequests != null) {
       addUpdateRequestsToProto();
     }
+    if (this.schedulingRequests != null) {
+      addSchedulingRequestsToProto();
+    }
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
@@ -178,6 +185,23 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   }
 
   @Override
+  public List<SchedulingRequest> getSchedulingRequests() {
+    initSchedulingRequests();
+    return this.schedulingRequests;
+  }
+
+  @Override
+  public void setSchedulingRequests(
+      List<SchedulingRequest> schedulingRequests) {
+    if (schedulingRequests == null) {
+      return;
+    }
+    initSchedulingRequests();
+    this.schedulingRequests.clear();
+    this.schedulingRequests.addAll(schedulingRequests);
+  }
+
+  @Override
   public ResourceBlacklistRequest getResourceBlacklistRequest() {
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.blacklistRequest != null) {
@@ -261,6 +285,20 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     }
   }
 
+  private void initSchedulingRequests() {
+    if (this.schedulingRequests != null) {
+      return;
+    }
+    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<SchedulingRequestProto> list =
+        p.getSchedulingRequestsList();
+    this.schedulingRequests = new ArrayList<>();
+
+    for (SchedulingRequestProto c : list) {
+      this.schedulingRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
   private void addUpdateRequestsToProto() {
     maybeInitBuilder();
     builder.clearUpdateRequests();
@@ -297,6 +335,41 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     builder.addAllUpdateRequests(iterable);
   }
 
+  private void addSchedulingRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearSchedulingRequests();
+    if (schedulingRequests == null) {
+      return;
+    }
+    Iterable<SchedulingRequestProto> iterable =
+        new Iterable<SchedulingRequestProto>() {
+          @Override
+          public Iterator<SchedulingRequestProto> iterator() {
+            return new Iterator<SchedulingRequestProto>() {
+
+              private Iterator<SchedulingRequest> iter =
+                  schedulingRequests.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public SchedulingRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+
+          }
+        };
+    builder.addAllSchedulingRequests(iterable);
+  }
   @Override
   public List<ContainerId> getReleaseList() {
     initReleases();
@@ -377,6 +450,16 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     return ((UpdateContainerRequestPBImpl) t).getProto();
   }
 
+  private SchedulingRequestPBImpl convertFromProtoFormat(
+      SchedulingRequestProto p) {
+    return new SchedulingRequestPBImpl(p);
+  }
+
+  private SchedulingRequestProto convertToProtoFormat(
+      SchedulingRequest t) {
+    return ((SchedulingRequestPBImpl) t).getProto();
+  }
+
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
index 05bb3bd..f98e488 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceSizingPBImpl.java
@@ -112,6 +112,6 @@ public class ResourceSizingPBImpl extends ResourceSizing {
   }
 
   private ResourceProto convertToProtoFormat(Resource r) {
-    return ((ResourcePBImpl) r).getProto();
+    return ProtoUtils.convertToProtoFormat(r);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
index 7826b36..305856a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SchedulingRequestPBImpl.java
@@ -263,4 +263,20 @@ public class SchedulingRequestPBImpl extends SchedulingRequest {
     this.allocationTags = new HashSet<>();
     this.allocationTags.addAll(p.getAllocationTagsList());
   }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca28a795/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..a0b907d 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
@@ -149,8 +149,10 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.ResourceTypeInfo;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -189,7 +191,9 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceOptionPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourceSizingPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceTypeInfoPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
@@ -225,6 +229,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceOptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceSizingProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SchedulingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
@@ -428,6 +434,8 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(QueueConfigurations.class);
     generateByNewInstance(CollectorInfo.class);
     generateByNewInstance(ResourceTypeInfo.class);
+    generateByNewInstance(ResourceSizing.class);
+    generateByNewInstance(SchedulingRequest.class);
   }
 
   @Test
@@ -907,6 +915,17 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
   }
 
   @Test
+  public void testResourceSizingPBImpl() throws Exception {
+    validatePBImplRecord(ResourceSizingPBImpl.class, ResourceSizingProto.class);
+  }
+
+  @Test
+  public void testSchedulingRequestPBImpl() throws Exception {
+    validatePBImplRecord(SchedulingRequestPBImpl.class,
+        SchedulingRequestProto.class);
+  }
+
+  @Test
   public void testSerializedExceptionPBImpl() throws Exception {
     validatePBImplRecord(SerializedExceptionPBImpl.class,
         SerializedExceptionProto.class);


---------------------------------------------------------------------
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-11902. [READ] Merge BlockFormatProvider and FileRegionProvider.

Posted by kk...@apache.org.
HDFS-11902. [READ] Merge BlockFormatProvider and FileRegionProvider.


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

Branch: refs/heads/YARN-6592
Commit: 98f5ed5aa377ddd3f35b763b20c499d2ccac2ed5
Parents: d6a9a89
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Nov 3 13:45:56 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  17 +-
 .../blockmanagement/BlockFormatProvider.java    |  91 ----
 .../server/blockmanagement/BlockProvider.java   |  75 ----
 .../blockmanagement/ProvidedStorageMap.java     |  63 ++-
 .../hadoop/hdfs/server/common/BlockFormat.java  |  82 ----
 .../hdfs/server/common/FileRegionProvider.java  |  37 --
 .../server/common/TextFileRegionFormat.java     | 442 ------------------
 .../server/common/TextFileRegionProvider.java   |  88 ----
 .../common/blockaliasmap/BlockAliasMap.java     |  88 ++++
 .../impl/TextFileRegionAliasMap.java            | 445 +++++++++++++++++++
 .../common/blockaliasmap/package-info.java      |  27 ++
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  76 ++--
 .../src/main/resources/hdfs-default.xml         |  34 +-
 .../blockmanagement/TestProvidedStorageMap.java |  41 +-
 .../hdfs/server/common/TestTextBlockFormat.java | 160 -------
 .../impl/TestTextBlockAliasMap.java             | 161 +++++++
 .../fsdataset/impl/TestProvidedImpl.java        |  75 ++--
 .../hdfs/server/namenode/FileSystemImage.java   |   4 +-
 .../hdfs/server/namenode/ImageWriter.java       |  25 +-
 .../hdfs/server/namenode/NullBlockAliasMap.java |  86 ++++
 .../hdfs/server/namenode/NullBlockFormat.java   |  87 ----
 .../hadoop/hdfs/server/namenode/TreePath.java   |   8 +-
 .../TestNameNodeProvidedImplementation.java     |  25 +-
 23 files changed, 994 insertions(+), 1243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 7449987..cb57675 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -331,22 +331,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
   public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;
 
-  public static final String DFS_NAMENODE_BLOCK_PROVIDER_CLASS = "dfs.namenode.block.provider.class";
-
-  public static final String DFS_PROVIDER_CLASS = "dfs.provider.class";
   public static final String DFS_PROVIDER_DF_CLASS = "dfs.provided.df.class";
   public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";
   public static final String DFS_PROVIDER_STORAGEUUID_DEFAULT =  "DS-PROVIDED";
-  public static final String DFS_PROVIDER_BLK_FORMAT_CLASS = "dfs.provided.blockformat.class";
+  public static final String DFS_PROVIDED_ALIASMAP_CLASS = "dfs.provided.aliasmap.class";
 
-  public static final String DFS_PROVIDED_BLOCK_MAP_DELIMITER = "dfs.provided.textprovider.delimiter";
-  public static final String DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT = ",";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER = "dfs.provided.aliasmap.text.delimiter";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT = ",";
 
-  public static final String DFS_PROVIDED_BLOCK_MAP_READ_PATH = "dfs.provided.textprovider.read.path";
-  public static final String DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT = "file:///tmp/blocks.csv";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH = "dfs.provided.aliasmap.text.read.path";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT = "file:///tmp/blocks.csv";
 
-  public static final String DFS_PROVIDED_BLOCK_MAP_CODEC = "dfs.provided.textprovider.read.codec";
-  public static final String DFS_PROVIDED_BLOCK_MAP_WRITE_PATH  = "dfs.provided.textprovider.write.path";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_CODEC = "dfs.provided.aliasmap.text.codec";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH = "dfs.provided.aliasmap.text.write.path";
 
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
deleted file mode 100644
index 930263d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
+++ /dev/null
@@ -1,91 +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.blockmanagement;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.BlockAlias;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Loads provided blocks from a {@link BlockFormat}.
- */
-public class BlockFormatProvider extends BlockProvider
-    implements Configurable {
-
-  private Configuration conf;
-  private BlockFormat<? extends BlockAlias> blockFormat;
-  public static final Logger LOG =
-      LoggerFactory.getLogger(BlockFormatProvider.class);
-
-  @Override
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  public void setConf(Configuration conf) {
-    Class<? extends BlockFormat> c = conf.getClass(
-        DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-        TextFileRegionFormat.class, BlockFormat.class);
-    blockFormat = ReflectionUtils.newInstance(c, conf);
-    LOG.info("Loaded BlockFormat class : " + c.getClass().getName());
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public Iterator<Block> iterator() {
-    try {
-      final BlockFormat.Reader<? extends BlockAlias> reader =
-          blockFormat.getReader(null);
-
-      return new Iterator<Block>() {
-
-        private final Iterator<? extends BlockAlias> inner = reader.iterator();
-
-        @Override
-        public boolean hasNext() {
-          return inner.hasNext();
-        }
-
-        @Override
-        public Block next() {
-          return inner.next().getBlock();
-        }
-
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to read provided blocks", e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
deleted file mode 100644
index 2214868..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
+++ /dev/null
@@ -1,75 +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.blockmanagement;
-
-import java.io.IOException;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap.ProvidedBlockList;
-import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.util.RwLock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Used to load provided blocks in the {@link BlockManager}.
- */
-public abstract class BlockProvider implements Iterable<Block> {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ProvidedStorageMap.class);
-
-  private RwLock lock;
-  private BlockManager bm;
-  private DatanodeStorageInfo storage;
-  private boolean hasDNs = false;
-
-  /**
-   * @param lock the namesystem lock
-   * @param bm block manager
-   * @param storage storage for provided blocks
-   */
-  void init(RwLock lock, BlockManager bm, DatanodeStorageInfo storage) {
-    this.bm = bm;
-    this.lock = lock;
-    this.storage = storage;
-  }
-
-  /**
-   * start the processing of block report for provided blocks.
-   * @throws IOException
-   */
-  void start(BlockReportContext context) throws IOException {
-    assert lock.hasWriteLock() : "Not holding write lock";
-    if (hasDNs) {
-      return;
-    }
-    if (storage.getBlockReportCount() == 0) {
-      LOG.info("Calling process first blk report from storage: " + storage);
-      // first pass; periodic refresh should call bm.processReport
-      bm.processFirstBlockReport(storage, new ProvidedBlockList(iterator()));
-    } else {
-      bm.processReport(storage, new ProvidedBlockList(iterator()), context);
-    }
-    hasDNs = true;
-  }
-
-  void stop() {
-    assert lock.hasWriteLock() : "Not holding write lock";
-    hasDNs = false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 5717e0c..a848d50 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
@@ -40,7 +40,10 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.common.BlockAlias;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -61,7 +64,11 @@ public class ProvidedStorageMap {
       LoggerFactory.getLogger(ProvidedStorageMap.class);
 
   // limit to a single provider for now
-  private final BlockProvider blockProvider;
+  private RwLock lock;
+  private BlockManager bm;
+  private boolean hasDNs = false;
+  private BlockAliasMap aliasMap;
+
   private final String storageId;
   private final ProvidedDescriptor providedDescriptor;
   private final DatanodeStorageInfo providedStorageInfo;
@@ -79,7 +86,7 @@ public class ProvidedStorageMap {
 
     if (!providedEnabled) {
       // disable mapping
-      blockProvider = null;
+      aliasMap = null;
       providedDescriptor = null;
       providedStorageInfo = null;
       return;
@@ -90,15 +97,17 @@ public class ProvidedStorageMap {
     providedDescriptor = new ProvidedDescriptor();
     providedStorageInfo = providedDescriptor.createProvidedStorage(ds);
 
+    this.bm = bm;
+    this.lock = lock;
+
     // load block reader into storage
-    Class<? extends BlockProvider> fmt = conf.getClass(
-        DFSConfigKeys.DFS_NAMENODE_BLOCK_PROVIDER_CLASS,
-        BlockFormatProvider.class, BlockProvider.class);
-
-    blockProvider = ReflectionUtils.newInstance(fmt, conf);
-    blockProvider.init(lock, bm, providedStorageInfo);
-    LOG.info("Loaded block provider class: " +
-        blockProvider.getClass() + " storage: " + providedStorageInfo);
+    Class<? extends BlockAliasMap> aliasMapClass = conf.getClass(
+            DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+            TextFileRegionAliasMap.class, BlockAliasMap.class);
+    aliasMap = ReflectionUtils.newInstance(aliasMapClass, conf);
+
+    LOG.info("Loaded alias map class: " +
+        aliasMap.getClass() + " storage: " + providedStorageInfo);
   }
 
   /**
@@ -114,8 +123,7 @@ public class ProvidedStorageMap {
       BlockReportContext context) throws IOException {
     if (providedEnabled && storageId.equals(s.getStorageID())) {
       if (StorageType.PROVIDED.equals(s.getStorageType())) {
-        // poll service, initiate
-        blockProvider.start(context);
+        processProvidedStorageReport(context);
         dn.injectStorage(providedStorageInfo);
         return providedDescriptor.getProvidedStorage(dn, s);
       }
@@ -124,6 +132,26 @@ public class ProvidedStorageMap {
     return dn.getStorageInfo(s.getStorageID());
   }
 
+  private void processProvidedStorageReport(BlockReportContext context)
+      throws IOException {
+    assert lock.hasWriteLock() : "Not holding write lock";
+    if (hasDNs) {
+      return;
+    }
+    if (providedStorageInfo.getBlockReportCount() == 0) {
+      LOG.info("Calling process first blk report from storage: "
+          + providedStorageInfo);
+      // first pass; periodic refresh should call bm.processReport
+      bm.processFirstBlockReport(providedStorageInfo,
+          new ProvidedBlockList(aliasMap.getReader(null).iterator()));
+    } else {
+      bm.processReport(providedStorageInfo,
+          new ProvidedBlockList(aliasMap.getReader(null).iterator()),
+          context);
+    }
+    hasDNs = true;
+  }
+
   @VisibleForTesting
   public DatanodeStorageInfo getProvidedStorageInfo() {
     return providedStorageInfo;
@@ -137,10 +165,11 @@ public class ProvidedStorageMap {
   }
 
   public void removeDatanode(DatanodeDescriptor dnToRemove) {
-    if (providedDescriptor != null) {
+    if (providedEnabled) {
+      assert lock.hasWriteLock() : "Not holding write lock";
       int remainingDatanodes = providedDescriptor.remove(dnToRemove);
       if (remainingDatanodes == 0) {
-        blockProvider.stop();
+        hasDNs = false;
       }
     }
   }
@@ -443,9 +472,9 @@ public class ProvidedStorageMap {
    */
   static class ProvidedBlockList extends BlockListAsLongs {
 
-    private final Iterator<Block> inner;
+    private final Iterator<BlockAlias> inner;
 
-    ProvidedBlockList(Iterator<Block> inner) {
+    ProvidedBlockList(Iterator<BlockAlias> inner) {
       this.inner = inner;
     }
 
@@ -454,7 +483,7 @@ public class ProvidedStorageMap {
       return new Iterator<BlockReportReplica>() {
         @Override
         public BlockReportReplica next() {
-          return new BlockReportReplica(inner.next());
+          return new BlockReportReplica(inner.next().getBlock());
         }
         @Override
         public boolean hasNext() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
deleted file mode 100644
index 66e7fdf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
+++ /dev/null
@@ -1,82 +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.common;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-
-/**
- * An abstract class used to read and write block maps for provided blocks.
- */
-public abstract class BlockFormat<T extends BlockAlias>  {
-
-  /**
-   * An abstract class that is used to read {@link BlockAlias}es
-   * for provided blocks.
-   */
-  public static abstract class Reader<U extends BlockAlias>
-      implements Iterable<U>, Closeable {
-
-    /**
-     * reader options.
-     */
-    public interface Options { }
-
-    public abstract U resolve(Block ident) throws IOException;
-
-  }
-
-  /**
-   * Returns the reader for the provided block map.
-   * @param opts reader options
-   * @return {@link Reader} to the block map.
-   * @throws IOException
-   */
-  public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
-
-  /**
-   * An abstract class used as a writer for the provided block map.
-   */
-  public static abstract class Writer<U extends BlockAlias>
-      implements Closeable {
-    /**
-     * writer options.
-     */
-    public interface Options { }
-
-    public abstract void store(U token) throws IOException;
-
-  }
-
-  /**
-   * Returns the writer for the provided block map.
-   * @param opts writer options.
-   * @return {@link Writer} to the block map.
-   * @throws IOException
-   */
-  public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
-
-  /**
-   * Refresh based on the underlying block map.
-   * @throws IOException
-   */
-  public abstract void refresh() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
deleted file mode 100644
index 2e94239..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
+++ /dev/null
@@ -1,37 +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.common;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-
-/**
- * This class is a stub for reading file regions from the block map.
- */
-public class FileRegionProvider implements Iterable<FileRegion> {
-  @Override
-  public Iterator<FileRegion> iterator() {
-    return Collections.emptyListIterator();
-  }
-
-  public void refresh() throws IOException {
-    return;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
deleted file mode 100644
index eacd08f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
+++ /dev/null
@@ -1,442 +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.common;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Collections;
-import java.util.IdentityHashMap;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * This class is used for block maps stored as text files,
- * with a specified delimiter.
- */
-public class TextFileRegionFormat
-    extends BlockFormat<FileRegion> implements Configurable {
-
-  private Configuration conf;
-  private ReaderOptions readerOpts = TextReader.defaults();
-  private WriterOptions writerOpts = TextWriter.defaults();
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(TextFileRegionFormat.class);
-  @Override
-  public void setConf(Configuration conf) {
-    readerOpts.setConf(conf);
-    writerOpts.setConf(conf);
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public Reader<FileRegion> getReader(Reader.Options opts)
-      throws IOException {
-    if (null == opts) {
-      opts = readerOpts;
-    }
-    if (!(opts instanceof ReaderOptions)) {
-      throw new IllegalArgumentException("Invalid options " + opts.getClass());
-    }
-    ReaderOptions o = (ReaderOptions) opts;
-    Configuration readerConf = (null == o.getConf())
-        ? new Configuration()
-            : o.getConf();
-    return createReader(o.file, o.delim, readerConf);
-  }
-
-  @VisibleForTesting
-  TextReader createReader(Path file, String delim, Configuration cfg)
-      throws IOException {
-    FileSystem fs = file.getFileSystem(cfg);
-    if (fs instanceof LocalFileSystem) {
-      fs = ((LocalFileSystem)fs).getRaw();
-    }
-    CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
-    CompressionCodec codec = factory.getCodec(file);
-    return new TextReader(fs, file, codec, delim);
-  }
-
-  @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
-    if (null == opts) {
-      opts = writerOpts;
-    }
-    if (!(opts instanceof WriterOptions)) {
-      throw new IllegalArgumentException("Invalid options " + opts.getClass());
-    }
-    WriterOptions o = (WriterOptions) opts;
-    Configuration cfg = (null == o.getConf())
-        ? new Configuration()
-            : o.getConf();
-    if (o.codec != null) {
-      CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
-      CompressionCodec codec = factory.getCodecByName(o.codec);
-      String name = o.file.getName() + codec.getDefaultExtension();
-      o.filename(new Path(o.file.getParent(), name));
-      return createWriter(o.file, codec, o.delim, cfg);
-    }
-    return createWriter(o.file, null, o.delim, conf);
-  }
-
-  @VisibleForTesting
-  TextWriter createWriter(Path file, CompressionCodec codec, String delim,
-      Configuration cfg) throws IOException {
-    FileSystem fs = file.getFileSystem(cfg);
-    if (fs instanceof LocalFileSystem) {
-      fs = ((LocalFileSystem)fs).getRaw();
-    }
-    OutputStream tmp = fs.create(file);
-    java.io.Writer out = new BufferedWriter(new OutputStreamWriter(
-          (null == codec) ? tmp : codec.createOutputStream(tmp), "UTF-8"));
-    return new TextWriter(out, delim);
-  }
-
-  /**
-   * Class specifying reader options for the {@link TextFileRegionFormat}.
-   */
-  public static class ReaderOptions
-      implements TextReader.Options, Configurable {
-
-    private Configuration conf;
-    private String delim =
-        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
-    private Path file = new Path(
-        new File(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT)
-        .toURI().toString());
-
-    @Override
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-      String tmpfile = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
-      file = new Path(tmpfile);
-      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT);
-      LOG.info("TextFileRegionFormat: read path " + tmpfile.toString());
-    }
-
-    @Override
-    public Configuration getConf() {
-      return conf;
-    }
-
-    @Override
-    public ReaderOptions filename(Path file) {
-      this.file = file;
-      return this;
-    }
-
-    @Override
-    public ReaderOptions delimiter(String delim) {
-      this.delim = delim;
-      return this;
-    }
-  }
-
-  /**
-   * Class specifying writer options for the {@link TextFileRegionFormat}.
-   */
-  public static class WriterOptions
-      implements TextWriter.Options, Configurable {
-
-    private Configuration conf;
-    private String codec = null;
-    private Path file =
-        new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
-    private String delim =
-        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
-
-    @Override
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-      String tmpfile = conf.get(
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_WRITE_PATH, file.toString());
-      file = new Path(tmpfile);
-      codec = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_CODEC);
-      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT);
-    }
-
-    @Override
-    public Configuration getConf() {
-      return conf;
-    }
-
-    @Override
-    public WriterOptions filename(Path file) {
-      this.file = file;
-      return this;
-    }
-
-    public String getCodec() {
-      return codec;
-    }
-
-    public Path getFile() {
-      return file;
-    }
-
-    @Override
-    public WriterOptions codec(String codec) {
-      this.codec = codec;
-      return this;
-    }
-
-    @Override
-    public WriterOptions delimiter(String delim) {
-      this.delim = delim;
-      return this;
-    }
-
-  }
-
-  /**
-   * This class is used as a reader for block maps which
-   * are stored as delimited text files.
-   */
-  public static class TextReader extends Reader<FileRegion> {
-
-    /**
-     * Options for {@link TextReader}.
-     */
-    public interface Options extends Reader.Options {
-      Options filename(Path file);
-      Options delimiter(String delim);
-    }
-
-    static ReaderOptions defaults() {
-      return new ReaderOptions();
-    }
-
-    private final Path file;
-    private final String delim;
-    private final FileSystem fs;
-    private final CompressionCodec codec;
-    private final Map<FRIterator, BufferedReader> iterators;
-
-    protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
-        String delim) {
-      this(fs, file, codec, delim,
-          new IdentityHashMap<FRIterator, BufferedReader>());
-    }
-
-    TextReader(FileSystem fs, Path file, CompressionCodec codec, String delim,
-        Map<FRIterator, BufferedReader> iterators) {
-      this.fs = fs;
-      this.file = file;
-      this.codec = codec;
-      this.delim = delim;
-      this.iterators = Collections.synchronizedMap(iterators);
-    }
-
-    @Override
-    public FileRegion resolve(Block ident) throws IOException {
-      // consider layering index w/ composable format
-      Iterator<FileRegion> i = iterator();
-      try {
-        while (i.hasNext()) {
-          FileRegion f = i.next();
-          if (f.getBlock().equals(ident)) {
-            return f;
-          }
-        }
-      } finally {
-        BufferedReader r = iterators.remove(i);
-        if (r != null) {
-          // null on last element
-          r.close();
-        }
-      }
-      return null;
-    }
-
-    class FRIterator implements Iterator<FileRegion> {
-
-      private FileRegion pending;
-
-      @Override
-      public boolean hasNext() {
-        return pending != null;
-      }
-
-      @Override
-      public FileRegion next() {
-        if (null == pending) {
-          throw new NoSuchElementException();
-        }
-        FileRegion ret = pending;
-        try {
-          pending = nextInternal(this);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-        return ret;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    }
-
-    private FileRegion nextInternal(Iterator<FileRegion> i) throws IOException {
-      BufferedReader r = iterators.get(i);
-      if (null == r) {
-        throw new IllegalStateException();
-      }
-      String line = r.readLine();
-      if (null == line) {
-        iterators.remove(i);
-        return null;
-      }
-      String[] f = line.split(delim);
-      if (f.length != 6) {
-        throw new IOException("Invalid line: " + line);
-      }
-      return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
-          Long.parseLong(f[2]), Long.parseLong(f[3]), f[5],
-          Long.parseLong(f[4]));
-    }
-
-    public InputStream createStream() throws IOException {
-      InputStream i = fs.open(file);
-      if (codec != null) {
-        i = codec.createInputStream(i);
-      }
-      return i;
-    }
-
-    @Override
-    public Iterator<FileRegion> iterator() {
-      FRIterator i = new FRIterator();
-      try {
-        BufferedReader r =
-            new BufferedReader(new InputStreamReader(createStream(), "UTF-8"));
-        iterators.put(i, r);
-        i.pending = nextInternal(i);
-      } catch (IOException e) {
-        iterators.remove(i);
-        throw new RuntimeException(e);
-      }
-      return i;
-    }
-
-    @Override
-    public void close() throws IOException {
-      ArrayList<IOException> ex = new ArrayList<>();
-      synchronized (iterators) {
-        for (Iterator<BufferedReader> i = iterators.values().iterator();
-             i.hasNext();) {
-          try {
-            BufferedReader r = i.next();
-            r.close();
-          } catch (IOException e) {
-            ex.add(e);
-          } finally {
-            i.remove();
-          }
-        }
-        iterators.clear();
-      }
-      if (!ex.isEmpty()) {
-        throw MultipleIOException.createIOException(ex);
-      }
-    }
-
-  }
-
-  /**
-   * This class is used as a writer for block maps which
-   * are stored as delimited text files.
-   */
-  public static class TextWriter extends Writer<FileRegion> {
-
-    /**
-     * Interface for Writer options.
-     */
-    public interface Options extends Writer.Options {
-      Options codec(String codec);
-      Options filename(Path file);
-      Options delimiter(String delim);
-    }
-
-    public static WriterOptions defaults() {
-      return new WriterOptions();
-    }
-
-    private final String delim;
-    private final java.io.Writer out;
-
-    public TextWriter(java.io.Writer out, String delim) {
-      this.out = out;
-      this.delim = delim;
-    }
-
-    @Override
-    public void store(FileRegion token) throws IOException {
-      out.append(String.valueOf(token.getBlock().getBlockId())).append(delim);
-      out.append(token.getPath().toString()).append(delim);
-      out.append(Long.toString(token.getOffset())).append(delim);
-      out.append(Long.toString(token.getLength())).append(delim);
-      out.append(Long.toString(token.getGenerationStamp())).append(delim);
-      out.append(token.getBlockPoolId()).append("\n");
-    }
-
-    @Override
-    public void close() throws IOException {
-      out.close();
-    }
-
-  }
-
-  @Override
-  public void refresh() throws IOException {
-    //nothing to do;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
deleted file mode 100644
index 0fa667e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
+++ /dev/null
@@ -1,88 +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.common;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * This class is used to read file regions from block maps
- * specified using delimited text.
- */
-public class TextFileRegionProvider
-    extends FileRegionProvider implements Configurable {
-
-  private Configuration conf;
-  private BlockFormat<FileRegion> fmt;
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void setConf(Configuration conf) {
-    fmt = ReflectionUtils.newInstance(
-        conf.getClass(DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-            TextFileRegionFormat.class,
-            BlockFormat.class),
-        conf);
-    ((Configurable)fmt).setConf(conf); //redundant?
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public Iterator<FileRegion> iterator() {
-    try {
-      final BlockFormat.Reader<FileRegion> r = fmt.getReader(null);
-      return new Iterator<FileRegion>() {
-
-        private final Iterator<FileRegion> inner = r.iterator();
-
-        @Override
-        public boolean hasNext() {
-          return inner.hasNext();
-        }
-
-        @Override
-        public FileRegion next() {
-          return inner.next();
-        }
-
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to read provided blocks", e);
-    }
-  }
-
-  @Override
-  public void refresh() throws IOException {
-    fmt.refresh();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
new file mode 100644
index 0000000..d276fb5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
@@ -0,0 +1,88 @@
+/**
+ * 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.common.blockaliasmap;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.BlockAlias;
+
+/**
+ * An abstract class used to read and write block maps for provided blocks.
+ */
+public abstract class BlockAliasMap<T extends BlockAlias> {
+
+  /**
+   * An abstract class that is used to read {@link BlockAlias}es
+   * for provided blocks.
+   */
+  public static abstract class Reader<U extends BlockAlias>
+      implements Iterable<U>, Closeable {
+
+    /**
+     * reader options.
+     */
+    public interface Options { }
+
+    /**
+     * @param ident block to resolve
+     * @return BlockAlias correspoding to the provided block.
+     * @throws IOException
+     */
+    public abstract U resolve(Block ident) throws IOException;
+
+  }
+
+  /**
+   * Returns a reader to the alias map.
+   * @param opts reader options
+   * @return {@link Reader} to the alias map.
+   * @throws IOException
+   */
+  public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
+
+  /**
+   * An abstract class used as a writer for the provided block map.
+   */
+  public static abstract class Writer<U extends BlockAlias>
+      implements Closeable {
+    /**
+     * writer options.
+     */
+    public interface Options { }
+
+    public abstract void store(U token) throws IOException;
+
+  }
+
+  /**
+   * Returns the writer for the alias map.
+   * @param opts writer options.
+   * @return {@link Writer} to the alias map.
+   * @throws IOException
+   */
+  public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
+
+  /**
+   * Refresh the alias map.
+   * @throws IOException
+   */
+  public abstract void refresh() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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
new file mode 100644
index 0000000..80f48c1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
@@ -0,0 +1,445 @@
+/**
+ * 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.common.blockaliasmap.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is used for block maps stored as text files,
+ * with a specified delimiter.
+ */
+public class TextFileRegionAliasMap
+    extends BlockAliasMap<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private ReaderOptions readerOpts = TextReader.defaults();
+  private WriterOptions writerOpts = TextWriter.defaults();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TextFileRegionAliasMap.class);
+  @Override
+  public void setConf(Configuration conf) {
+    readerOpts.setConf(conf);
+    writerOpts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts)
+      throws IOException {
+    if (null == opts) {
+      opts = readerOpts;
+    }
+    if (!(opts instanceof ReaderOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    ReaderOptions o = (ReaderOptions) opts;
+    Configuration readerConf = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    return createReader(o.file, o.delim, readerConf);
+  }
+
+  @VisibleForTesting
+  TextReader createReader(Path file, String delim, Configuration cfg)
+      throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+    CompressionCodec codec = factory.getCodec(file);
+    return new TextReader(fs, file, codec, delim);
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    if (null == opts) {
+      opts = writerOpts;
+    }
+    if (!(opts instanceof WriterOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    WriterOptions o = (WriterOptions) opts;
+    Configuration cfg = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    if (o.codec != null) {
+      CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+      CompressionCodec codec = factory.getCodecByName(o.codec);
+      String name = o.file.getName() + codec.getDefaultExtension();
+      o.filename(new Path(o.file.getParent(), name));
+      return createWriter(o.file, codec, o.delim, cfg);
+    }
+    return createWriter(o.file, null, o.delim, conf);
+  }
+
+  @VisibleForTesting
+  TextWriter createWriter(Path file, CompressionCodec codec, String delim,
+      Configuration cfg) throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    OutputStream tmp = fs.create(file);
+    java.io.Writer out = new BufferedWriter(new OutputStreamWriter(
+          (null == codec) ? tmp : codec.createOutputStream(tmp), "UTF-8"));
+    return new TextWriter(out, delim);
+  }
+
+  /**
+   * Class specifying reader options for the {@link TextFileRegionAliasMap}.
+   */
+  public static class ReaderOptions
+      implements TextReader.Options, Configurable {
+
+    private Configuration conf;
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
+    private Path file = new Path(
+        new File(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT).toURI()
+            .toString());
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile =
+          conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
+      file = new Path(tmpfile);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+      LOG.info("TextFileRegionAliasMap: read path " + tmpfile.toString());
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public ReaderOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    @Override
+    public ReaderOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+  }
+
+  /**
+   * Class specifying writer options for the {@link TextFileRegionAliasMap}.
+   */
+  public static class WriterOptions
+      implements TextWriter.Options, Configurable {
+
+    private Configuration conf;
+    private String codec = null;
+    private Path file =
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);;
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile = conf.get(
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH, file.toString());
+      file = new Path(tmpfile);
+      codec = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_CODEC);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public WriterOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    public String getCodec() {
+      return codec;
+    }
+
+    public Path getFile() {
+      return file;
+    }
+
+    @Override
+    public WriterOptions codec(String codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    @Override
+    public WriterOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link TextReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    static ReaderOptions defaults() {
+      return new ReaderOptions();
+    }
+
+    private final Path file;
+    private final String delim;
+    private final FileSystem fs;
+    private final CompressionCodec codec;
+    private final Map<FRIterator, BufferedReader> iterators;
+
+    protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
+        String delim) {
+      this(fs, file, codec, delim,
+          new IdentityHashMap<FRIterator, BufferedReader>());
+    }
+
+    TextReader(FileSystem fs, Path file, CompressionCodec codec, String delim,
+        Map<FRIterator, BufferedReader> iterators) {
+      this.fs = fs;
+      this.file = file;
+      this.codec = codec;
+      this.delim = delim;
+      this.iterators = Collections.synchronizedMap(iterators);
+    }
+
+    @Override
+    public FileRegion resolve(Block ident) throws IOException {
+      // consider layering index w/ composable format
+      Iterator<FileRegion> i = iterator();
+      try {
+        while (i.hasNext()) {
+          FileRegion f = i.next();
+          if (f.getBlock().equals(ident)) {
+            return f;
+          }
+        }
+      } finally {
+        BufferedReader r = iterators.remove(i);
+        if (r != null) {
+          // null on last element
+          r.close();
+        }
+      }
+      return null;
+    }
+
+    class FRIterator implements Iterator<FileRegion> {
+
+      private FileRegion pending;
+
+      @Override
+      public boolean hasNext() {
+        return pending != null;
+      }
+
+      @Override
+      public FileRegion next() {
+        if (null == pending) {
+          throw new NoSuchElementException();
+        }
+        FileRegion ret = pending;
+        try {
+          pending = nextInternal(this);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return ret;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    private FileRegion nextInternal(Iterator<FileRegion> i) throws IOException {
+      BufferedReader r = iterators.get(i);
+      if (null == r) {
+        throw new IllegalStateException();
+      }
+      String line = r.readLine();
+      if (null == line) {
+        iterators.remove(i);
+        return null;
+      }
+      String[] f = line.split(delim);
+      if (f.length != 6) {
+        throw new IOException("Invalid line: " + line);
+      }
+      return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
+          Long.parseLong(f[2]), Long.parseLong(f[3]), f[5],
+          Long.parseLong(f[4]));
+    }
+
+    public InputStream createStream() throws IOException {
+      InputStream i = fs.open(file);
+      if (codec != null) {
+        i = codec.createInputStream(i);
+      }
+      return i;
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      FRIterator i = new FRIterator();
+      try {
+        BufferedReader r =
+            new BufferedReader(new InputStreamReader(createStream(), "UTF-8"));
+        iterators.put(i, r);
+        i.pending = nextInternal(i);
+      } catch (IOException e) {
+        iterators.remove(i);
+        throw new RuntimeException(e);
+      }
+      return i;
+    }
+
+    @Override
+    public void close() throws IOException {
+      ArrayList<IOException> ex = new ArrayList<>();
+      synchronized (iterators) {
+        for (Iterator<BufferedReader> i = iterators.values().iterator();
+             i.hasNext();) {
+          try {
+            BufferedReader r = i.next();
+            r.close();
+          } catch (IOException e) {
+            ex.add(e);
+          } finally {
+            i.remove();
+          }
+        }
+        iterators.clear();
+      }
+      if (!ex.isEmpty()) {
+        throw MultipleIOException.createIOException(ex);
+      }
+    }
+
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options codec(String codec);
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    public static WriterOptions defaults() {
+      return new WriterOptions();
+    }
+
+    private final String delim;
+    private final java.io.Writer out;
+
+    public TextWriter(java.io.Writer out, String delim) {
+      this.out = out;
+      this.delim = delim;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      out.append(String.valueOf(token.getBlock().getBlockId())).append(delim);
+      out.append(token.getPath().toString()).append(delim);
+      out.append(Long.toString(token.getOffset())).append(delim);
+      out.append(Long.toString(token.getLength())).append(delim);
+      out.append(Long.toString(token.getGenerationStamp())).append(delim);
+      out.append(token.getBlockPoolId()).append("\n");
+    }
+
+    @Override
+    public void close() throws IOException {
+      out.close();
+    }
+
+  }
+
+  @Override
+  public void refresh() throws IOException {
+    //nothing to do;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
new file mode 100644
index 0000000..b906791
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.common.blockaliasmap;
+
+/**
+ * The AliasMap defines mapping of PROVIDED HDFS blocks to data in remote
+ * storage systems.
+ */
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 d1a7015..092672d 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
@@ -35,9 +35,9 @@ 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.common.FileRegion;
-import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionProvider;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
@@ -68,7 +68,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   static class ProvidedBlockPoolSlice {
     private ProvidedVolumeImpl providedVolume;
 
-    private FileRegionProvider provider;
+    private BlockAliasMap<FileRegion> aliasMap;
     private Configuration conf;
     private String bpid;
     private ReplicaMap bpVolumeMap;
@@ -77,29 +77,35 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
         Configuration conf) {
       this.providedVolume = volume;
       bpVolumeMap = new ReplicaMap(new AutoCloseableLock());
-      Class<? extends FileRegionProvider> fmt =
-          conf.getClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
-              TextFileRegionProvider.class, FileRegionProvider.class);
-      provider = ReflectionUtils.newInstance(fmt, conf);
+      Class<? extends BlockAliasMap> fmt =
+          conf.getClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+              TextFileRegionAliasMap.class, BlockAliasMap.class);
+      aliasMap = ReflectionUtils.newInstance(fmt, conf);
       this.conf = conf;
       this.bpid = bpid;
       bpVolumeMap.initBlockPool(bpid);
-      LOG.info("Created provider: " + provider.getClass());
+      LOG.info("Created alias map using class: " + aliasMap.getClass());
     }
 
-    FileRegionProvider getFileRegionProvider() {
-      return provider;
+    BlockAliasMap<FileRegion> getBlockAliasMap() {
+      return aliasMap;
     }
 
     @VisibleForTesting
-    void setFileRegionProvider(FileRegionProvider newProvider) {
-      this.provider = newProvider;
+    void setFileRegionProvider(BlockAliasMap<FileRegion> blockAliasMap) {
+      this.aliasMap = blockAliasMap;
     }
 
     public void getVolumeMap(ReplicaMap volumeMap,
         RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
         throws IOException {
-      Iterator<FileRegion> iter = provider.iterator();
+      BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null);
+      if (reader == null) {
+        LOG.warn("Got null reader from BlockAliasMap " + aliasMap
+            + "; no blocks will be populated");
+        return;
+      }
+      Iterator<FileRegion> iter = reader.iterator();
       while (iter.hasNext()) {
         FileRegion region = iter.next();
         if (region.getBlockPoolId() != null
@@ -140,14 +146,20 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
     public void compileReport(LinkedList<ScanInfo> report,
         ReportCompiler reportCompiler)
             throws IOException, InterruptedException {
-      /* refresh the provider and return the list of blocks found.
+      /* refresh the aliasMap and return the list of blocks found.
        * the assumption here is that the block ids in the external
        * block map, after the refresh, are consistent with those
        * from before the refresh, i.e., for blocks which did not change,
        * the ids remain the same.
        */
-      provider.refresh();
-      Iterator<FileRegion> iter = provider.iterator();
+      aliasMap.refresh();
+      BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null);
+      if (reader == null) {
+        LOG.warn("Got null reader from BlockAliasMap " + aliasMap
+            + "; no blocks will be populated in scan report");
+        return;
+      }
+      Iterator<FileRegion> iter = reader.iterator();
       while(iter.hasNext()) {
         reportCompiler.throttle();
         FileRegion region = iter.next();
@@ -284,15 +296,15 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
     private String bpid;
     private String name;
-    private FileRegionProvider provider;
+    private BlockAliasMap<FileRegion> blockAliasMap;
     private Iterator<FileRegion> blockIterator;
     private ProvidedBlockIteratorState state;
 
     ProviderBlockIteratorImpl(String bpid, String name,
-        FileRegionProvider provider) {
+        BlockAliasMap<FileRegion> blockAliasMap) {
       this.bpid = bpid;
       this.name = name;
-      this.provider = provider;
+      this.blockAliasMap = blockAliasMap;
       rewind();
     }
 
@@ -330,7 +342,17 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
     @Override
     public void rewind() {
-      blockIterator = provider.iterator();
+      BlockAliasMap.Reader<FileRegion> reader = null;
+      try {
+        reader = blockAliasMap.getReader(null);
+      } catch (IOException e) {
+        LOG.warn("Exception in getting reader from provided alias map");
+      }
+      if (reader != null) {
+        blockIterator = reader.iterator();
+      } else {
+        blockIterator = null;
+      }
       state = new ProvidedBlockIteratorState();
     }
 
@@ -372,14 +394,14 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   @Override
   public BlockIterator newBlockIterator(String bpid, String name) {
     return new ProviderBlockIteratorImpl(bpid, name,
-        bpSlices.get(bpid).getFileRegionProvider());
+        bpSlices.get(bpid).getBlockAliasMap());
   }
 
   @Override
   public BlockIterator loadBlockIterator(String bpid, String name)
       throws IOException {
     ProviderBlockIteratorImpl iter = new ProviderBlockIteratorImpl(bpid, name,
-        bpSlices.get(bpid).getFileRegionProvider());
+        bpSlices.get(bpid).getBlockAliasMap());
     iter.load();
     return iter;
   }
@@ -425,8 +447,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   }
 
   @VisibleForTesting
-  FileRegionProvider getFileRegionProvider(String bpid) throws IOException {
-    return getProvidedBlockPoolSlice(bpid).getFileRegionProvider();
+  BlockAliasMap<FileRegion> getBlockFormat(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).getBlockAliasMap();
   }
 
   @Override
@@ -571,12 +593,12 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   }
 
   @VisibleForTesting
-  void setFileRegionProvider(String bpid, FileRegionProvider provider)
-      throws IOException {
+  void setFileRegionProvider(String bpid,
+      BlockAliasMap<FileRegion> blockAliasMap) throws IOException {
     ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
     if (bp == null) {
       throw new IOException("block pool " + bpid + " is not found");
     }
-    bp.setFileRegionProvider(provider);
+    bp.setFileRegionProvider(blockAliasMap);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 0f1407a..835d8c4 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
@@ -4630,26 +4630,6 @@
   </property>
 
   <property>
-    <name>dfs.namenode.block.provider.class</name>
-    <value>org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider</value>
-    <description>
-      The class that is used to load provided blocks in the Namenode.
-    </description>
-  </property>
-
-  <property>
-    <name>dfs.provider.class</name>
-    <value>org.apache.hadoop.hdfs.server.common.TextFileRegionProvider</value>
-    <description>
-      The class that is used to load information about blocks stored in
-      provided storages.
-      org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TextFileRegionProvider
-      is used as the default, which expects the blocks to be specified
-      using a delimited text file.
-    </description>
-  </property>
-
-  <property>
     <name>dfs.provided.df.class</name>
     <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
     <description>
@@ -4666,12 +4646,12 @@
   </property>
 
   <property>
-    <name>dfs.provided.blockformat.class</name>
-    <value>org.apache.hadoop.hdfs.server.common.TextFileRegionFormat</value>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap</value>
     <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.TextFileRegionFormat which uses
+      org.apache.hadoop.hdfs.server.common.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
@@ -4681,7 +4661,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.delimiter</name>
+    <name>dfs.provided.aliasmap.text.delimiter</name>
     <value>,</value>
     <description>
         The delimiter used when the provided block map is specified as
@@ -4690,7 +4670,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.read.path</name>
+    <name>dfs.provided.aliasmap.text.read.path</name>
     <value></value>
     <description>
         The path specifying the provided block map as a text file, specified as
@@ -4699,7 +4679,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.read.codec</name>
+    <name>dfs.provided.aliasmap.text.codec</name>
     <value></value>
     <description>
         The codec used to de-compress the provided block map.
@@ -4707,7 +4687,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.write.path</name>
+    <name>dfs.provided.aliasmap.text.write.path</name>
     <value></value>
     <description>
         The path to which the provided block map should be written as a text

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/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 2296c82..89741b5 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
@@ -17,20 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestProvidedImpl;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Iterator;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -47,37 +46,6 @@ public class TestProvidedStorageMap {
   private RwLock nameSystemLock;
   private String providedStorageID;
 
-  static class TestBlockProvider extends BlockProvider
-          implements Configurable {
-
-    @Override
-    public void setConf(Configuration conf) {
-    }
-
-    @Override
-    public Configuration getConf() {
-      return null;
-    }
-
-    @Override
-    public Iterator<Block> iterator() {
-      return new Iterator<Block>() {
-        @Override
-        public boolean hasNext() {
-          return false;
-        }
-        @Override
-        public Block next() {
-          return null;
-        }
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    }
-  }
-
   @Before
   public void setup() {
     providedStorageID = DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT;
@@ -85,8 +53,9 @@ public class TestProvidedStorageMap {
     conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
             providedStorageID);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
-    conf.setClass(DFSConfigKeys.DFS_NAMENODE_BLOCK_PROVIDER_CLASS,
-            TestBlockProvider.class, BlockProvider.class);
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TestProvidedImpl.TestFileRegionBlockAliasMap.class,
+        BlockAliasMap.class);
 
     bm = mock(BlockManager.class);
     nameSystemLock = mock(RwLock.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
deleted file mode 100644
index eaaac22..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
+++ /dev/null
@@ -1,160 +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.common;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStreamWriter;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat.*;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.compress.CompressionCodec;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-/**
- * Test for the text based block format for provided block maps.
- */
-public class TestTextBlockFormat {
-
-  static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
-
-  void check(TextWriter.Options opts, final Path vp,
-      final Class<? extends CompressionCodec> vc) throws IOException {
-    TextFileRegionFormat mFmt = new TextFileRegionFormat() {
-      @Override
-      public TextWriter createWriter(Path file, CompressionCodec codec,
-          String delim, Configuration conf) throws IOException {
-        assertEquals(vp, file);
-        if (null == vc) {
-          assertNull(codec);
-        } else {
-          assertEquals(vc, codec.getClass());
-        }
-        return null; // ignored
-      }
-    };
-    mFmt.getWriter(opts);
-  }
-
-  @Test
-  public void testWriterOptions() throws Exception {
-    TextWriter.Options opts = TextWriter.defaults();
-    assertTrue(opts instanceof WriterOptions);
-    WriterOptions wopts = (WriterOptions) opts;
-    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
-    assertEquals(def, wopts.getFile());
-    assertNull(wopts.getCodec());
-
-    opts.filename(OUTFILE);
-    check(opts, OUTFILE, null);
-
-    opts.filename(OUTFILE);
-    opts.codec("gzip");
-    Path cp = new Path(OUTFILE.getParent(), OUTFILE.getName() + ".gz");
-    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
-
-  }
-
-  @Test
-  public void testCSVReadWrite() throws Exception {
-    final DataOutputBuffer out = new DataOutputBuffer();
-    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
-    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
-    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
-    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), ",")) {
-      csv.store(r1);
-      csv.store(r2);
-      csv.store(r3);
-    }
-    Iterator<FileRegion> i3;
-    try (TextReader csv = new TextReader(null, null, null, ",") {
-      @Override
-      public InputStream createStream() {
-        DataInputBuffer in = new DataInputBuffer();
-        in.reset(out.getData(), 0, out.getLength());
-        return in;
-        }}) {
-      Iterator<FileRegion> i1 = csv.iterator();
-      assertEquals(r1, i1.next());
-      Iterator<FileRegion> i2 = csv.iterator();
-      assertEquals(r1, i2.next());
-      assertEquals(r2, i2.next());
-      assertEquals(r3, i2.next());
-      assertEquals(r2, i1.next());
-      assertEquals(r3, i1.next());
-
-      assertFalse(i1.hasNext());
-      assertFalse(i2.hasNext());
-      i3 = csv.iterator();
-    }
-    try {
-      i3.next();
-    } catch (IllegalStateException e) {
-      return;
-    }
-    fail("Invalid iterator");
-  }
-
-  @Test
-  public void testCSVReadWriteTsv() throws Exception {
-    final DataOutputBuffer out = new DataOutputBuffer();
-    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
-    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
-    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
-    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), "\t")) {
-      csv.store(r1);
-      csv.store(r2);
-      csv.store(r3);
-    }
-    Iterator<FileRegion> i3;
-    try (TextReader csv = new TextReader(null, null, null, "\t") {
-      @Override
-      public InputStream createStream() {
-        DataInputBuffer in = new DataInputBuffer();
-        in.reset(out.getData(), 0, out.getLength());
-        return in;
-      }}) {
-      Iterator<FileRegion> i1 = csv.iterator();
-      assertEquals(r1, i1.next());
-      Iterator<FileRegion> i2 = csv.iterator();
-      assertEquals(r1, i2.next());
-      assertEquals(r2, i2.next());
-      assertEquals(r3, i2.next());
-      assertEquals(r2, i1.next());
-      assertEquals(r3, i1.next());
-
-      assertFalse(i1.hasNext());
-      assertFalse(i2.hasNext());
-      i3 = csv.iterator();
-    }
-    try {
-      i3.next();
-    } catch (IllegalStateException e) {
-      return;
-    }
-    fail("Invalid iterator");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98f5ed5a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
new file mode 100644
index 0000000..79308a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.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.hdfs.server.common.blockaliasmap.impl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.*;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test for the text based block format for provided block maps.
+ */
+public class TestTextBlockAliasMap {
+
+  static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
+
+  void check(TextWriter.Options opts, final Path vp,
+      final Class<? extends CompressionCodec> vc) throws IOException {
+    TextFileRegionAliasMap mFmt = new TextFileRegionAliasMap() {
+      @Override
+      public TextWriter createWriter(Path file, CompressionCodec codec,
+          String delim, Configuration conf) throws IOException {
+        assertEquals(vp, file);
+        if (null == vc) {
+          assertNull(codec);
+        } else {
+          assertEquals(vc, codec.getClass());
+        }
+        return null; // ignored
+      }
+    };
+    mFmt.getWriter(opts);
+  }
+
+  @Test
+  public void testWriterOptions() throws Exception {
+    TextWriter.Options opts = TextWriter.defaults();
+    assertTrue(opts instanceof WriterOptions);
+    WriterOptions wopts = (WriterOptions) opts;
+    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
+    assertEquals(def, wopts.getFile());
+    assertNull(wopts.getCodec());
+
+    opts.filename(OUTFILE);
+    check(opts, OUTFILE, null);
+
+    opts.filename(OUTFILE);
+    opts.codec("gzip");
+    Path cp = new Path(OUTFILE.getParent(), OUTFILE.getName() + ".gz");
+    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
+
+  }
+
+  @Test
+  public void testCSVReadWrite() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), ",")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, ",") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+        }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+  @Test
+  public void testCSVReadWriteTsv() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), "\t")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, "\t") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+      }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+}


---------------------------------------------------------------------
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-12091. [READ] Check that the replicas served from a ProvidedVolumeImpl belong to the correct external storage

Posted by kk...@apache.org.
HDFS-12091. [READ] Check that the replicas served from a ProvidedVolumeImpl belong to the correct external storage


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

Branch: refs/heads/YARN-6592
Commit: 663b3c08b131ea2db693e1a5d2f5da98242fa854
Parents: 546b95f
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Aug 7 11:35:49 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/StorageLocation.java   |  26 +++--
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  67 ++++++++++--
 .../fsdataset/impl/TestProvidedImpl.java        | 105 ++++++++++++++++++-
 3 files changed, 173 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/663b3c08/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 fb7acfd..d72448d 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
@@ -64,21 +64,25 @@ public class StorageLocation
     this.storageType = storageType;
     if (uri.getScheme() == null || uri.getScheme().equals("file")) {
       // make sure all URIs that point to a file have the same scheme
-      try {
-        File uriFile = new File(uri.getPath());
-        String uriStr = uriFile.toURI().normalize().toString();
-        if (uriStr.endsWith("/")) {
-          uriStr = uriStr.substring(0, uriStr.length() - 1);
-        }
-        uri = new URI(uriStr);
-      } catch (URISyntaxException e) {
-        throw new IllegalArgumentException(
-            "URI: " + uri + " is not in the expected format");
-      }
+      uri = normalizeFileURI(uri);
     }
     baseURI = uri;
   }
 
+  public static URI normalizeFileURI(URI uri) {
+    try {
+      File uriFile = new File(uri.getPath());
+      String uriStr = uriFile.toURI().normalize().toString();
+      if (uriStr.endsWith("/")) {
+        uriStr = uriStr.substring(0, uriStr.length() - 1);
+      }
+      return new URI(uriStr);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(
+              "URI: " + uri + " is not in the expected format");
+    }
+  }
+
   public StorageType getStorageType() {
     return this.storageType;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/663b3c08/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 421b9cc..5cd28c7 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
@@ -64,7 +65,7 @@ import org.apache.hadoop.util.Time;
 public class ProvidedVolumeImpl extends FsVolumeImpl {
 
   static class ProvidedBlockPoolSlice {
-    private FsVolumeImpl providedVolume;
+    private ProvidedVolumeImpl providedVolume;
 
     private FileRegionProvider provider;
     private Configuration conf;
@@ -89,13 +90,20 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
       return provider;
     }
 
+    @VisibleForTesting
+    void setFileRegionProvider(FileRegionProvider newProvider) {
+      this.provider = newProvider;
+    }
+
     public void getVolumeMap(ReplicaMap volumeMap,
         RamDiskReplicaTracker ramDiskReplicaMap) throws IOException {
       Iterator<FileRegion> iter = provider.iterator();
-      while(iter.hasNext()) {
+      while (iter.hasNext()) {
         FileRegion region = iter.next();
-        if (region.getBlockPoolId() != null &&
-            region.getBlockPoolId().equals(bpid)) {
+        if (region.getBlockPoolId() != null
+            && region.getBlockPoolId().equals(bpid)
+            && containsBlock(providedVolume.baseURI,
+                region.getPath().toUri())) {
           ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
               .setBlockId(region.getBlock().getBlockId())
               .setURI(region.getPath().toUri())
@@ -103,17 +111,16 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
               .setLength(region.getBlock().getNumBytes())
               .setGenerationStamp(region.getBlock().getGenerationStamp())
               .setFsVolume(providedVolume)
-              .setConf(conf).build();
-
-          ReplicaInfo oldReplica =
-              volumeMap.get(bpid, newReplica.getBlockId());
+              .setConf(conf)
+              .build();
+          // check if the replica already exists
+          ReplicaInfo oldReplica = volumeMap.get(bpid, newReplica.getBlockId());
           if (oldReplica == null) {
             volumeMap.add(bpid, newReplica);
             bpVolumeMap.add(bpid, newReplica);
           } else {
-            throw new IOException(
-                "A block with id " + newReplica.getBlockId() +
-                " already exists in the volumeMap");
+            throw new IOException("A block with id " + newReplica.getBlockId()
+                + " already exists in the volumeMap");
           }
         }
       }
@@ -527,4 +534,42 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
     throw new UnsupportedOperationException(
         "ProvidedVolume does not yet support writes");
   }
+
+  private static URI getAbsoluteURI(URI uri) {
+    if (!uri.isAbsolute()) {
+      // URI is not absolute implies it is for a local file
+      // normalize the URI
+      return StorageLocation.normalizeFileURI(uri);
+    } else {
+      return uri;
+    }
+  }
+  /**
+   * @param volumeURI URI of the volume
+   * @param blockURI URI of the block
+   * @return true if the {@code blockURI} can belong to the volume or both URIs
+   * are null.
+   */
+  @VisibleForTesting
+  public static boolean containsBlock(URI volumeURI, URI blockURI) {
+    if (volumeURI == null && blockURI == null){
+      return true;
+    }
+    if (volumeURI == null || blockURI == null) {
+      return false;
+    }
+    volumeURI = getAbsoluteURI(volumeURI);
+    blockURI = getAbsoluteURI(blockURI);
+    return !volumeURI.relativize(blockURI).equals(blockURI);
+  }
+
+  @VisibleForTesting
+  void setFileRegionProvider(String bpid, FileRegionProvider provider)
+      throws IOException {
+    ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp == null) {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
+    bp.setFileRegionProvider(provider);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/663b3c08/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 4753235..8782e71 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
@@ -31,6 +31,8 @@ import java.io.IOException;
 import java.io.InputStream;
 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;
@@ -174,15 +176,26 @@ public class TestProvidedImpl {
     private Configuration conf;
     private int minId;
     private int numBlocks;
+    private Iterator<FileRegion> suppliedIterator;
 
     TestFileRegionProvider() {
-      minId = MIN_BLK_ID;
-      numBlocks = NUM_PROVIDED_BLKS;
+      this(null, MIN_BLK_ID, NUM_PROVIDED_BLKS);
+    }
+
+    TestFileRegionProvider(Iterator<FileRegion> iterator, int minId,
+        int numBlocks) {
+      this.suppliedIterator = iterator;
+      this.minId = minId;
+      this.numBlocks = numBlocks;
     }
 
     @Override
     public Iterator<FileRegion> iterator() {
-      return new TestFileRegionIterator(providedBasePath, minId, numBlocks);
+      if (suppliedIterator == null) {
+        return new TestFileRegionIterator(providedBasePath, minId, numBlocks);
+      } else {
+        return suppliedIterator;
+      }
     }
 
     @Override
@@ -503,4 +516,90 @@ public class TestProvidedImpl {
       }
     }
   }
+
+  private int getBlocksInProvidedVolumes(String basePath, int numBlocks,
+      int minBlockId) throws IOException {
+    TestFileRegionIterator fileRegionIterator =
+        new TestFileRegionIterator(basePath, minBlockId, numBlocks);
+    int totalBlocks = 0;
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
+      vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID],
+          new TestFileRegionProvider(fileRegionIterator, minBlockId,
+              numBlocks));
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null);
+      totalBlocks += volumeMap.size(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+    }
+    return totalBlocks;
+  }
+
+  /**
+   * Tests if the FileRegions provided by the FileRegionProvider
+   * can belong to the Providevolume.
+   * @throws IOException
+   */
+  @Test
+  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
+    int blocksFound = getBlocksInProvidedVolumes(providedBasePath + "/test1/",
+        expectedBlocks, minId);
+    assertEquals(
+        "Number of blocks in provided volumes should be " + expectedBlocks,
+        expectedBlocks, blocksFound);
+    blocksFound = getBlocksInProvidedVolumes(
+        "file:/" + providedBasePath + "/test1/", expectedBlocks, minId);
+    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
+    blocksFound =
+        getBlocksInProvidedVolumes("randomtest1/", expectedBlocks, minId);
+    assertEquals("Number of blocks in provided volumes should be 0", 0,
+        blocksFound);
+  }
+
+  @Test
+  public void testProvidedVolumeContainsBlock() throws URISyntaxException {
+    assertEquals(true, ProvidedVolumeImpl.containsBlock(null, null));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a"), null));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/c/"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c/"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c/"),
+            new URI("/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/c/"),
+            new URI("/a/b/c/d/e.file")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/e"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/e"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket1/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket2/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket1/dir1/"),
+            new URI("s3a:/bucket1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("/bucket1/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+  }
 }


---------------------------------------------------------------------
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: HDFS-12903. [READ] Fix closing streams in ImageWriter. Contributed by Virajith Jalaparti

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


[17/50] [abbrv] hadoop git commit: HDFS-12779. [READ] Allow cluster id to be specified to the Image generation tool

Posted by kk...@apache.org.
HDFS-12779. [READ] Allow cluster id to be specified to the Image generation tool


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

Branch: refs/heads/YARN-6592
Commit: 6cd80b2521e6283036d8c7058d8e452a93ff8e4b
Parents: 90d1b47
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 9 14:09:14 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/protocol/NamespaceInfo.java     |  4 ++++
 .../hdfs/server/namenode/FileSystemImage.java   |  4 ++++
 .../hdfs/server/namenode/ImageWriter.java       | 11 ++++++++-
 .../TestNameNodeProvidedImplementation.java     | 24 +++++++++++++++++++-
 4 files changed, 41 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd80b25/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index 66ce9ee..433d9b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -160,6 +160,10 @@ public class NamespaceInfo extends StorageInfo {
     return state;
   }
 
+  public void setClusterID(String clusterID) {
+    this.clusterID = clusterID;
+  }
+
   @Override
   public String toString(){
     return super.toString() + ";bpid=" + blockPoolID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd80b25/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 2e57c9f..b66c830 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
@@ -68,6 +68,7 @@ public class FileSystemImage implements Tool {
     options.addOption("b", "blockclass", true, "Block output class");
     options.addOption("i", "blockidclass", true, "Block resolver class");
     options.addOption("c", "cachedirs", true, "Max active dirents");
+    options.addOption("cid", "clusterID", true, "Cluster ID");
     options.addOption("h", "help", false, "Print usage");
     return options;
   }
@@ -112,6 +113,9 @@ public class FileSystemImage implements Tool {
       case "c":
         opts.cache(Integer.parseInt(o.getValue()));
         break;
+      case "cid":
+        opts.clusterID(o.getValue());
+        break;
       default:
         throw new UnsupportedOperationException("Internal error");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd80b25/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 390bb39..9bd8852 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
@@ -126,13 +126,16 @@ public class ImageWriter implements Closeable {
           throw new IllegalStateException("Incompatible layout " +
               info.getLayoutVersion() + " (expected " + LAYOUT_VERSION);
         }
+        // set the cluster id, if given
+        if (opts.clusterID.length() > 0) {
+          info.setClusterID(opts.clusterID);
+        }
         stor.format(info);
         blockPoolID = info.getBlockPoolID();
       }
       outdir = new Path(tmp, "current");
       out = outfs.create(new Path(outdir, "fsimage_0000000000000000000"));
     } else {
-      // XXX necessary? writing a NNStorage now...
       outdir = null;
       outfs = null;
       out = opts.outStream;
@@ -517,6 +520,7 @@ public class ImageWriter implements Closeable {
     private UGIResolver ugis;
     private Class<? extends UGIResolver> ugisClass;
     private BlockAliasMap<FileRegion> blocks;
+    private String clusterID;
 
     @SuppressWarnings("rawtypes")
     private Class<? extends BlockAliasMap> aliasMap;
@@ -543,6 +547,7 @@ public class ImageWriter implements Closeable {
           NullBlockAliasMap.class, BlockAliasMap.class);
       blockIdsClass = conf.getClass(BLOCK_RESOLVER_CLASS,
           FixedBlockResolver.class, BlockResolver.class);
+      clusterID = "";
     }
 
     @Override
@@ -601,6 +606,10 @@ public class ImageWriter implements Closeable {
       return this;
     }
 
+    public Options clusterID(String clusterID) {
+      this.clusterID = clusterID;
+      return this;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd80b25/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 1f6aebb..22f00aa 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
@@ -155,11 +155,18 @@ public class TestNameNodeProvidedImplementation {
 
   void createImage(TreeWalk t, Path out,
       Class<? extends BlockResolver> blockIdsClass) throws Exception {
+    createImage(t, out, blockIdsClass, "");
+  }
+
+  void createImage(TreeWalk t, Path out,
+      Class<? extends BlockResolver> blockIdsClass, String clusterID)
+      throws Exception {
     ImageWriter.Options opts = ImageWriter.defaults();
     opts.setConf(conf);
     opts.output(out.toString())
         .blocks(TextFileRegionAliasMap.class)
-        .blockIds(blockIdsClass);
+        .blockIds(blockIdsClass)
+        .clusterID(clusterID);
     try (ImageWriter w = new ImageWriter(opts)) {
       for (TreePath e : t) {
         w.accept(e);
@@ -562,4 +569,19 @@ public class TestNameNodeProvidedImplementation {
           dnInfos[0].getDatanodeUuid());
     }
   }
+
+  @Test
+  public void testSetClusterID() throws Exception {
+    String clusterID = "PROVIDED-CLUSTER";
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class, clusterID);
+    // 2 Datanodes, 1 PROVIDED and other DISK
+    startCluster(NNDIRPATH, 2, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
+        false);
+    NameNode nn = cluster.getNameNode();
+    assertEquals(clusterID, nn.getNamesystem().getClusterId());
+  }
 }


---------------------------------------------------------------------
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: HDFS-12885. Add visibility/stability annotations. Contributed by Chris Douglas

Posted by kk...@apache.org.
HDFS-12885. Add visibility/stability annotations. Contributed by Chris Douglas


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

Branch: refs/heads/YARN-6592
Commit: a027055dd2bf5009fe272e9ceb08305bd0a8cc31
Parents: b634053
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Dec 5 09:51:09 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java    | 4 ++++
 .../org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java   | 2 ++
 .../hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java | 4 ++++
 .../InMemoryAliasMapProtocolClientSideTranslatorPB.java         | 4 ++++
 .../apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java    | 4 ++++
 .../hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java   | 4 ++++
 .../hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java     | 4 ++++
 .../hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java  | 4 ++++
 .../java/org/apache/hadoop/hdfs/server/common/BlockAlias.java   | 4 ++++
 .../java/org/apache/hadoop/hdfs/server/common/FileRegion.java   | 4 ++++
 .../hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java  | 4 ++++
 .../blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java       | 4 ++++
 .../common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java    | 4 ++++
 .../common/blockaliasmap/impl/TextFileRegionAliasMap.java       | 4 ++++
 .../hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java   | 4 ++++
 .../org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java | 4 ++++
 .../hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java | 4 +++-
 .../org/apache/hadoop/hdfs/server/namenode/BlockResolver.java   | 4 ++++
 .../java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java | 4 ++++
 .../org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java | 4 ++++
 .../hdfs/server/namenode/FixedBlockMultiReplicaResolver.java    | 4 ++++
 .../apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java  | 4 ++++
 .../org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java   | 5 +++++
 .../org/apache/hadoop/hdfs/server/namenode/ImageWriter.java     | 4 ++++
 .../apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java   | 4 ++++
 .../apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java   | 4 ++++
 .../java/org/apache/hadoop/hdfs/server/namenode/TreePath.java   | 4 ++++
 .../java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java   | 5 +++++
 .../org/apache/hadoop/hdfs/server/namenode/UGIResolver.java     | 4 ++++
 .../org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java  | 4 ++++
 30 files changed, 119 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
index eee58ba..861ef8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
@@ -16,6 +16,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 
 import javax.annotation.Nonnull;
@@ -25,6 +27,8 @@ import java.util.Arrays;
  * ProvidedStorageLocation is a location in an external storage system
  * containing the data for a block (~Replica).
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public class ProvidedStorageLocation {
   private final Path path;
   private final long offset;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
index 98b3ee1..4e14fad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
@@ -17,6 +17,7 @@
 package org.apache.hadoop.hdfs.protocolPB;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos;
 import org.apache.hadoop.ipc.ProtocolInfo;
 
@@ -30,6 +31,7 @@ import org.apache.hadoop.ipc.ProtocolInfo;
         "org.apache.hadoop.hdfs.server.aliasmap.AliasMapProtocol",
     protocolVersion = 1)
 @InterfaceAudience.Private
+@InterfaceStability.Unstable
 public interface AliasMapProtocolPB extends
     AliasMapProtocolProtos.AliasMapProtocolService.BlockingInterface {
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
index 808c43b..15a1124 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
@@ -18,6 +18,8 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
@@ -40,6 +42,8 @@ import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.*;
  * AliasMapProtocolServerSideTranslatorPB is responsible for translating RPC
  * calls and forwarding them to the internal InMemoryAliasMap.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class AliasMapProtocolServerSideTranslatorPB
     implements AliasMapProtocolPB {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
index a79360f..7f6cc91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
@@ -17,6 +17,8 @@
 package org.apache.hadoop.hdfs.protocolPB;
 
 import com.google.protobuf.ServiceException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
@@ -47,6 +49,8 @@ import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.*;
  * {@link InMemoryAliasMapProtocol} interface to the RPC server implementing
  * {@link AliasMapProtocolPB}.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class InMemoryAliasMapProtocolClientSideTranslatorPB
     implements InMemoryAliasMapProtocol {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 be891e5..f7be111 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
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.aliasmap;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -47,6 +49,8 @@ import java.util.Optional;
  * InMemoryAliasMap is an implementation of the InMemoryAliasMapProtocol for
  * use with LevelDB.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
index fb6e8b3..c89c6b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
@@ -16,6 +16,8 @@
  */
 package org.apache.hadoop.hdfs.server.aliasmap;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
@@ -30,6 +32,8 @@ import java.util.Optional;
  * provided blocks for an in-memory implementation of the
  * {@link org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap}.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public interface InMemoryAliasMapProtocol {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 91b1e83..197e663 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
@@ -17,6 +17,8 @@
 package org.apache.hadoop.hdfs.server.aliasmap;
 
 import com.google.protobuf.BlockingService;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
@@ -42,6 +44,8 @@ import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFun
  * InMemoryLevelDBAliasMapServer is the entry point from the Namenode into
  * the {@link InMemoryAliasMap}.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
     Configurable, Closeable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 c85eb2c..f6b0c0b 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
@@ -30,6 +30,8 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentSkipListMap;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -57,6 +59,8 @@ import com.google.protobuf.ByteString;
  * This class allows us to manage and multiplex between storages local to
  * datanodes, and provided storage.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class ProvidedStorageMap {
 
   private static final Logger LOG =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
index b2fac97..316953f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
@@ -17,11 +17,15 @@
  */
 package org.apache.hadoop.hdfs.server.common;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 
 /**
  * Interface used to load provided blocks.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public interface BlockAlias {
 
   Block getBlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 5d04640..c2697c8 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.common;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -27,6 +29,8 @@ import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
  * This class is used to represent provided blocks that are file regions,
  * i.e., can be described using (path, offset, length).
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FileRegion implements BlockAlias {
 
   private final Pair<Block, ProvidedStorageLocation> pair;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
index e3b6cb5..8233fbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
@@ -22,12 +22,16 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.Optional;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.BlockAlias;
 
 /**
  * An abstract class used to read and write block maps for provided blocks.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public abstract class BlockAliasMap<T extends BlockAlias> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
index 7b0b789..bd6ebd1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
@@ -16,6 +16,8 @@
  */
 package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -37,6 +39,8 @@ import java.util.Optional;
  * This is used by the Datanode and fs2img to store and retrieve FileRegions
  * based on the given Block.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
     implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 66971a3..6a3d55c 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
@@ -24,6 +24,8 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBIterator;
 import static org.fusesource.leveldbjni.JniDBFactory.factory;
@@ -45,6 +47,8 @@ import org.slf4j.LoggerFactory;
 /**
  * A LevelDB based implementation of {@link BlockAliasMap}.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class LevelDBFileRegionAliasMap
       extends BlockAliasMap<FileRegion> implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 b86b280..69d9cc1 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
@@ -34,6 +34,8 @@ import java.util.IdentityHashMap;
 import java.util.NoSuchElementException;
 import java.util.Optional;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -56,6 +58,8 @@ import com.google.common.annotations.VisibleForTesting;
  * This class is used for block maps stored as text files,
  * with a specified delimiter.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class TextFileRegionAliasMap
     extends BlockAliasMap<FileRegion> implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 0fbfc15..43fd234 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
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.net.URI;
 
+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;
@@ -30,6 +32,8 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 /**
  * This class is used for provided replicas that are finalized.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class FinalizedProvidedReplica extends ProvidedReplica {
 
   public FinalizedProvidedReplica(long blockId, URI fileURI,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 8681421..3d20b00 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
@@ -25,6 +25,8 @@ import java.net.URI;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.input.BoundedInputStream;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,6 +44,8 @@ import org.slf4j.LoggerFactory;
 /**
  * This abstract class is used as a base class for provided replicas.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public abstract class ProvidedReplica extends ReplicaInfo {
 
   public static final Logger LOG =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 6bbfa91..445b455 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
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -65,7 +66,8 @@ import org.apache.hadoop.util.Time;
 /**
  * This class is used to create provided volumes.
  */
-public class ProvidedVolumeImpl extends FsVolumeImpl {
+@InterfaceAudience.Private
+class ProvidedVolumeImpl extends FsVolumeImpl {
 
   /**
    * Get a suffix of the full path, excluding the given prefix.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
index 94b92b8..b18b910 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 
@@ -27,6 +29,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
  * Given an external reference, create a sequence of blocks and associated
  * metadata.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public abstract class BlockResolver {
 
   protected BlockProto buildBlock(long blockId, long bytes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 f736112..7d66c64 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
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,6 +33,8 @@ import org.apache.hadoop.fs.Path;
 /**
  * Traversal of an external FileSystem.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FSTreeWalk extends TreeWalk {
 
   private final Path root;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 b66c830..4598e9c 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
@@ -26,6 +26,8 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
+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;
@@ -37,6 +39,8 @@ import org.apache.hadoop.util.ToolRunner;
 /**
  * Create FSImage from an external namespace.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FileSystemImage implements Tool {
 
   private Configuration conf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
index 0c8ce6e..42d4848 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 
@@ -24,6 +26,8 @@ import org.apache.hadoop.fs.FileStatus;
  * Resolver mapping all files to a configurable, uniform blocksize
  * and replication.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FixedBlockMultiReplicaResolver extends FixedBlockResolver {
 
   public static final String REPLICATION =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
index 4b3a01f..d130f8d 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
@@ -21,6 +21,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -28,6 +30,8 @@ import org.apache.hadoop.fs.FileStatus;
 /**
  * Resolver mapping all files to a configurable, uniform blocksize.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FixedBlockResolver extends BlockResolver implements Configurable {
 
   public static final String BLOCKSIZE =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
index ca16d96..8c61bc2 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 import java.util.HashSet;
 import java.util.Set;
 
@@ -24,6 +27,8 @@ import java.util.Set;
  * Dynamically assign ids to users/groups as they appear in the external
  * filesystem.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FsUGIResolver extends UGIResolver {
 
   private int id;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 9bd8852..370c683 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
@@ -38,6 +38,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.google.common.base.Charsets;
 import com.google.protobuf.CodedOutputStream;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -72,6 +74,8 @@ import static org.apache.hadoop.hdfs.server.namenode.FSImageUtil.MAGIC_HEADER;
  * a valid FSImage/NN storage.
  */
 // TODO: generalize to types beyond FileRegion
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class ImageWriter implements Closeable {
 
   private static final int ONDISK_VERSION = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
index 63d1f27..fcaaf44 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
@@ -22,6 +22,8 @@ import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.Optional;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
@@ -29,6 +31,8 @@ import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 /**
  * Null sink for region information emitted from FSImage.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 0fd3f2b..d60806f 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
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -27,6 +29,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 /**
  * Map all owners/groups in external system to a single user in FSImage.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class SingleUGIResolver extends UGIResolver implements Configurable {
 
   public static final String UID   = "hdfs.image.writer.ugi.single.uid";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 d327363..8f1382a 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
@@ -21,6 +21,8 @@ import java.io.IOException;
 
 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.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -37,6 +39,8 @@ import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.D
  * Traversal cursor in external filesystem.
  * TODO: generalize, move FS/FileRegion to FSTreePath
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class TreePath {
   private long id = -1;
   private final long parentId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
index 7fd26f9..03675ec 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 import java.util.ArrayDeque;
 import java.util.Deque;
 import java.util.Iterator;
@@ -24,6 +27,8 @@ import java.util.Iterator;
 /**
  * Traversal yielding a hierarchical sequence of paths.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public abstract class TreeWalk implements Iterable<TreePath> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
index 2d50668..72d1fa8 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
@@ -21,6 +21,8 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 
@@ -28,6 +30,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
  * Pluggable class for mapping ownership and permissions from an external
  * store to an FSImage.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public abstract class UGIResolver {
 
   static final int USER_STRID_OFFSET = 40;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a027055d/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 d002e4a..27152fd 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
@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -32,6 +34,8 @@ import org.apache.hadoop.fs.Path;
 /**
  * Random, repeatable hierarchy generator.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class RandomTreeWalk extends TreeWalk {
 
   private final Path root;


---------------------------------------------------------------------
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-11792. [READ] Test cases for ProvidedVolumeDF and ProviderBlockIteratorImpl

Posted by kk...@apache.org.
HDFS-11792. [READ] Test cases for ProvidedVolumeDF and ProviderBlockIteratorImpl


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

Branch: refs/heads/YARN-6592
Commit: 55ade54b8ed36e18f028f478381a96e7b8c6be50
Parents: 4851f06
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed May 31 15:17:12 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  6 +-
 .../fsdataset/impl/TestProvidedImpl.java        | 94 ++++++++++++++++++--
 2 files changed, 92 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55ade54b/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 a48e117..421b9cc 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
@@ -191,7 +191,11 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
   @Override
   long getBlockPoolUsed(String bpid) throws IOException {
-    return df.getBlockPoolUsed(bpid);
+    if (bpSlices.containsKey(bpid)) {
+      return df.getBlockPoolUsed(bpid);
+    } else {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55ade54b/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 2c119fe..4753235 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
@@ -83,6 +83,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.
   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;
@@ -208,6 +209,39 @@ public class TestProvidedImpl {
     }
   }
 
+  public static class TestProvidedVolumeDF
+      implements ProvidedVolumeDF, Configurable {
+
+    @Override
+    public void setConf(Configuration conf) {
+    }
+
+    @Override
+    public Configuration getConf() {
+      return null;
+    }
+
+    @Override
+    public long getCapacity() {
+      return Long.MAX_VALUE;
+    }
+
+    @Override
+    public long getSpaceUsed() {
+      return -1;
+    }
+
+    @Override
+    public long getBlockPoolUsed(String bpid) {
+      return -1;
+    }
+
+    @Override
+    public long getAvailable() {
+      return Long.MAX_VALUE;
+    }
+  }
+
   private static Storage.StorageDirectory createLocalStorageDirectory(
       File root, Configuration conf)
       throws SecurityException, IOException {
@@ -299,8 +333,8 @@ public class TestProvidedImpl {
   public void setUp() throws IOException {
     datanode = mock(DataNode.class);
     storage = mock(DataStorage.class);
-    this.conf = new Configuration();
-    this.conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
+    conf = new Configuration();
+    conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
 
     when(datanode.getConf()).thenReturn(conf);
     final DNConf dnConf = new DNConf(datanode);
@@ -312,8 +346,10 @@ public class TestProvidedImpl {
         new ShortCircuitRegistry(conf);
     when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry);
 
-    this.conf.setClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
+    conf.setClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
         TestFileRegionProvider.class, FileRegionProvider.class);
+    conf.setClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
+        TestProvidedVolumeDF.class, ProvidedVolumeDF.class);
 
     blkToPathMap = new HashMap<Long, String>();
     providedVolumes = new LinkedList<FsVolumeImpl>();
@@ -333,17 +369,43 @@ public class TestProvidedImpl {
     for (String bpid : BLOCK_POOL_IDS) {
       dataset.addBlockPool(bpid, conf);
     }
+  }
+
+  @Test
+  public void testProvidedVolumeImpl() throws IOException {
 
     assertEquals(NUM_LOCAL_INIT_VOLUMES + NUM_PROVIDED_INIT_VOLUMES,
         getNumVolumes());
+    assertEquals(NUM_PROVIDED_INIT_VOLUMES, providedVolumes.size());
     assertEquals(0, dataset.getNumFailedVolumes());
-  }
 
-  @Test
-  public void testProvidedStorageID() throws IOException {
+    TestProvidedVolumeDF df = new TestProvidedVolumeDF();
+
     for (int i = 0; i < providedVolumes.size(); i++) {
+      //check basic information about provided volume
       assertEquals(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT,
           providedVolumes.get(i).getStorageID());
+      assertEquals(StorageType.PROVIDED,
+          providedVolumes.get(i).getStorageType());
+
+      //check the df stats of the volume
+      assertEquals(df.getAvailable(), providedVolumes.get(i).getAvailable());
+      assertEquals(df.getBlockPoolUsed(BLOCK_POOL_IDS[CHOSEN_BP_ID]),
+          providedVolumes.get(i).getBlockPoolUsed(
+              BLOCK_POOL_IDS[CHOSEN_BP_ID]));
+
+      providedVolumes.get(i).shutdownBlockPool(
+          BLOCK_POOL_IDS[1 - CHOSEN_BP_ID], null);
+      try {
+        assertEquals(df.getBlockPoolUsed(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]),
+            providedVolumes.get(i).getBlockPoolUsed(
+                BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]));
+        //should not be triggered
+        assertTrue(false);
+      } catch (IOException e) {
+        LOG.info("Expected exception: " + e);
+      }
+
     }
   }
 
@@ -385,6 +447,8 @@ public class TestProvidedImpl {
       BlockIterator iter =
           vol.newBlockIterator(BLOCK_POOL_IDS[CHOSEN_BP_ID], "temp");
       Set<Long> blockIdsUsed = new HashSet<Long>();
+
+      assertEquals(BLOCK_POOL_IDS[CHOSEN_BP_ID], iter.getBlockPoolId());
       while(!iter.atEnd()) {
         ExtendedBlock eb = iter.nextBlock();
         long blkId = eb.getBlockId();
@@ -394,10 +458,26 @@ public class TestProvidedImpl {
         blockIdsUsed.add(blkId);
       }
       assertEquals(NUM_PROVIDED_BLKS, blockIdsUsed.size());
+
+      // rewind the block iterator
+      iter.rewind();
+      while(!iter.atEnd()) {
+        ExtendedBlock eb = iter.nextBlock();
+        long blkId = eb.getBlockId();
+        //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
+      assertEquals(0, blockIdsUsed.size());
+
+      //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());
     }
   }
 
-
   @Test
   public void testRefresh() throws IOException {
     conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);


---------------------------------------------------------------------
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: HDFS-12671. [READ] Test NameNode restarts when PROVIDED is configured

Posted by kk...@apache.org.
HDFS-12671. [READ] Test NameNode restarts when PROVIDED is configured


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

Branch: refs/heads/YARN-6592
Commit: c293cc8e9b032d2c573340725ef8ecc15d49430d
Parents: 71d0a82
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Nov 7 12:54:27 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../TestNameNodeProvidedImplementation.java     | 52 +++++++++++++++-----
 1 file changed, 39 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c293cc8e/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 aae04be..f0303b5 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
@@ -507,16 +507,10 @@ public class TestNameNodeProvidedImplementation {
     DataNode providedDatanode = cluster.getDataNodes().get(0);
 
     DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-            cluster.getNameNodePort()), cluster.getConfiguration(0));
+        cluster.getNameNodePort()), cluster.getConfiguration(0));
 
     for (int i= 0; i < numFiles; i++) {
-      String filename = "/" + filePrefix + i + fileSuffix;
-
-      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
-      // location should be the provided DN.
-      assertTrue(dnInfos[0].getDatanodeUuid()
-          .equals(providedDatanode.getDatanodeUuid()));
-
+      verifyFileLocation(i);
       // NameNode thinks the datanode is down
       BlockManagerTestUtil.noticeDeadDatanode(
           cluster.getNameNode(),
@@ -524,12 +518,44 @@ public class TestNameNodeProvidedImplementation {
       cluster.waitActive();
       cluster.triggerHeartbeats();
       Thread.sleep(1000);
+      verifyFileLocation(i);
+    }
+  }
 
-      // should find the block on the 2nd provided datanode.
-      dnInfos = getAndCheckBlockLocations(client, filename, 1);
-      assertTrue(
-          dnInfos[0].getDatanodeUuid()
-              .equals(providedDatanode.getDatanodeUuid()));
+  @Test(timeout=30000)
+  public void testNamenodeRestart() throws Exception {
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    // 2 Datanodes, 1 PROVIDED and other DISK
+    startCluster(NNDIRPATH, 2, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
+        false);
+
+    verifyFileLocation(numFiles - 1);
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    verifyFileLocation(numFiles - 1);
+  }
+
+  /**
+   * 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)
+      throws Exception {
+    DataNode providedDatanode = cluster.getDataNodes().get(0);
+    DFSClient client = new DFSClient(
+        new InetSocketAddress("localhost", cluster.getNameNodePort()),
+        cluster.getConfiguration(0));
+    if (fileIndex <= numFiles && fileIndex >= 0) {
+      String filename = "/" + filePrefix + fileIndex + fileSuffix;
+      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      // location should be the provided DN
+      assertEquals(providedDatanode.getDatanodeUuid(),
+          dnInfos[0].getDatanodeUuid());
     }
   }
 }


---------------------------------------------------------------------
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: YARN-6595. [API] Add Placement Constraints at the application level. (Arun Suresh via kkaranasos)

Posted by kk...@apache.org.
YARN-6595. [API] Add Placement Constraints at the application level. (Arun Suresh via kkaranasos)


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

Branch: refs/heads/YARN-6592
Commit: 3b9faf585af3760fabd0bb31d22d87ed95b2ef23
Parents: 16cbed8
Author: Konstantinos Karanasos <kk...@apache.org>
Authored: Mon Nov 13 15:25:24 2017 -0800
Committer: Konstantinos Karanasos <kk...@apache.org>
Committed: Mon Dec 18 16:07:00 2017 -0800

----------------------------------------------------------------------
 .../RegisterApplicationMasterRequest.java       |  42 ++++-
 .../yarn/api/resource/PlacementConstraint.java  | 156 +++++++++++++++++++
 .../src/main/proto/yarn_protos.proto            |   6 +
 .../src/main/proto/yarn_service_protos.proto    |   1 +
 .../RegisterApplicationMasterRequestPBImpl.java | 106 ++++++++++++-
 .../hadoop/yarn/api/BasePBImplRecordsTest.java  |  11 ++
 6 files changed, 313 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b9faf58/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
index 395e190..f2d537a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
@@ -18,11 +18,16 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 import org.apache.hadoop.yarn.util.Records;
-
 /**
  * The request sent by the {@code ApplicationMaster} to {@code ResourceManager}
  * on registration.
@@ -132,4 +137,39 @@ public abstract class RegisterApplicationMasterRequest {
   @Public
   @Stable
   public abstract void setTrackingUrl(String trackingUrl);
+
+  /**
+   * Return all Placement Constraints specified at the Application level. The
+   * mapping is from a set of allocation tags to a
+   * <code>PlacementConstraint</code> associated with the tags, i.e., each
+   * {@link org.apache.hadoop.yarn.api.records.SchedulingRequest} that has those
+   * tags will be placed taking into account the corresponding constraint.
+   *
+   * @return A map of Placement Constraints.
+   */
+  @Public
+  @Unstable
+  public Map<Set<String>, PlacementConstraint> getPlacementConstraints() {
+    return new HashMap<>();
+  }
+
+  /**
+   * Set Placement Constraints applicable to the
+   * {@link org.apache.hadoop.yarn.api.records.SchedulingRequest}s
+   * of this application.
+   * The mapping is from a set of allocation tags to a
+   * <code>PlacementConstraint</code> associated with the tags.
+   * For example:
+   *  Map &lt;
+   *   &lt;hb_regionserver&gt; -&gt; node_anti_affinity,
+   *   &lt;hb_regionserver, hb_master&gt; -&gt; rack_affinity,
+   *   ...
+   *  &gt;
+   * @param placementConstraints Placement Constraint Mapping.
+   */
+  @Public
+  @Unstable
+  public void setPlacementConstraints(
+      Map<Set<String>, PlacementConstraint> placementConstraints) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b9faf58/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
index f0e3982..b6e851a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
@@ -54,6 +54,26 @@ public class PlacementConstraint {
     return constraintExpr;
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof PlacementConstraint)) {
+      return false;
+    }
+
+    PlacementConstraint that = (PlacementConstraint) o;
+
+    return getConstraintExpr() != null ? getConstraintExpr().equals(that
+        .getConstraintExpr()) : that.getConstraintExpr() == null;
+  }
+
+  @Override
+  public int hashCode() {
+    return getConstraintExpr() != null ? getConstraintExpr().hashCode() : 0;
+  }
+
   /**
    * Interface used to enable the elements of the constraint tree to be visited.
    */
@@ -174,6 +194,38 @@ public class PlacementConstraint {
     }
 
     @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof SingleConstraint)) {
+        return false;
+      }
+
+      SingleConstraint that = (SingleConstraint) o;
+
+      if (getMinCardinality() != that.getMinCardinality()) {
+        return false;
+      }
+      if (getMaxCardinality() != that.getMaxCardinality()) {
+        return false;
+      }
+      if (!getScope().equals(that.getScope())) {
+        return false;
+      }
+      return getTargetExpressions().equals(that.getTargetExpressions());
+    }
+
+    @Override
+    public int hashCode() {
+      int result = getScope().hashCode();
+      result = 31 * result + getMinCardinality();
+      result = 31 * result + getMaxCardinality();
+      result = 31 * result + getTargetExpressions().hashCode();
+      return result;
+    }
+
+    @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
@@ -332,6 +384,34 @@ public class PlacementConstraint {
     }
 
     @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof TargetConstraint)) {
+        return false;
+      }
+
+      TargetConstraint that = (TargetConstraint) o;
+
+      if (getOp() != that.getOp()) {
+        return false;
+      }
+      if (!getScope().equals(that.getScope())) {
+        return false;
+      }
+      return getTargetExpressions().equals(that.getTargetExpressions());
+    }
+
+    @Override
+    public int hashCode() {
+      int result = getOp().hashCode();
+      result = 31 * result + getScope().hashCode();
+      result = 31 * result + getTargetExpressions().hashCode();
+      return result;
+    }
+
+    @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
@@ -388,6 +468,34 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      CardinalityConstraint that = (CardinalityConstraint) o;
+
+      if (minCardinality != that.minCardinality) {
+        return false;
+      }
+      if (maxCardinality != that.maxCardinality) {
+        return false;
+      }
+      return scope != null ? scope.equals(that.scope) : that.scope == null;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = scope != null ? scope.hashCode() : 0;
+      result = 31 * result + minCardinality;
+      result = 31 * result + maxCardinality;
+      return result;
+    }
   }
 
   /**
@@ -406,6 +514,25 @@ public class PlacementConstraint {
      * @return the children of the composite constraint
      */
     public abstract List<R> getChildren();
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      return getChildren() != null ? getChildren().equals(
+          ((CompositeConstraint)o).getChildren()) :
+          ((CompositeConstraint)o).getChildren() == null;
+    }
+
+    @Override
+    public int hashCode() {
+      return getChildren() != null ? getChildren().hashCode() : 0;
+    }
   }
 
   /**
@@ -563,5 +690,34 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      TimedPlacementConstraint that = (TimedPlacementConstraint) o;
+
+      if (schedulingDelay != that.schedulingDelay) {
+        return false;
+      }
+      if (constraint != null ? !constraint.equals(that.constraint) :
+          that.constraint != null) {
+        return false;
+      }
+      return delayUnit == that.delayUnit;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = constraint != null ? constraint.hashCode() : 0;
+      result = 31 * result + (int) (schedulingDelay ^ (schedulingDelay >>> 32));
+      result = 31 * result + (delayUnit != null ? delayUnit.hashCode() : 0);
+      return result;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b9faf58/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 2dbdefb..ac43d1b 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
@@ -648,6 +648,12 @@ message CompositePlacementConstraintProto {
   repeated TimedPlacementConstraintProto timedChildConstraints = 3;
 }
 
+// This associates a set of allocation tags to a Placement Constraint.
+message PlacementConstraintMapEntryProto {
+  repeated string allocation_tags = 1;
+  optional PlacementConstraintProto placement_constraint = 2;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From reservation_protocol /////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b9faf58/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 4e97c74..68e585d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -38,6 +38,7 @@ message RegisterApplicationMasterRequestProto {
   optional string host = 1;
   optional int32 rpc_port = 2;
   optional string tracking_url = 3;
+  repeated PlacementConstraintMapEntryProto placement_constraints = 4;
 }
 
 message RegisterApplicationMasterResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b9faf58/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
index 037dfd9..64bee85 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterRequestPBImpl.java
@@ -21,24 +21,41 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProtoOrBuilder;
 
 import com.google.protobuf.TextFormat;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 @Private
 @Unstable
-public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationMasterRequest {
-  RegisterApplicationMasterRequestProto proto = RegisterApplicationMasterRequestProto.getDefaultInstance();
-  RegisterApplicationMasterRequestProto.Builder builder = null;
+public class RegisterApplicationMasterRequestPBImpl
+    extends RegisterApplicationMasterRequest {
+  private RegisterApplicationMasterRequestProto proto =
+      RegisterApplicationMasterRequestProto.getDefaultInstance();
+  private RegisterApplicationMasterRequestProto.Builder builder = null;
+  private Map<Set<String>, PlacementConstraint> placementConstraints = null;
   boolean viaProto = false;
   
   public RegisterApplicationMasterRequestPBImpl() {
     builder = RegisterApplicationMasterRequestProto.newBuilder();
   }
 
-  public RegisterApplicationMasterRequestPBImpl(RegisterApplicationMasterRequestProto proto) {
+  public RegisterApplicationMasterRequestPBImpl(
+      RegisterApplicationMasterRequestProto proto) {
     this.proto = proto;
     viaProto = true;
   }
@@ -71,6 +88,30 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
   }
 
   private void mergeLocalToBuilder() {
+    if (this.placementConstraints != null) {
+      addPlacementConstraintMap();
+    }
+  }
+
+  private void addPlacementConstraintMap() {
+    maybeInitBuilder();
+    builder.clearPlacementConstraints();
+    if (this.placementConstraints == null) {
+      return;
+    }
+    List<YarnProtos.PlacementConstraintMapEntryProto> protoList =
+        new ArrayList<>();
+    for (Map.Entry<Set<String>, PlacementConstraint> entry :
+        this.placementConstraints.entrySet()) {
+      protoList.add(
+          YarnProtos.PlacementConstraintMapEntryProto.newBuilder()
+              .addAllAllocationTags(entry.getKey())
+              .setPlacementConstraint(
+                  new PlacementConstraintToProtoConverter(
+                      entry.getValue()).convert())
+              .build());
+    }
+    builder.addAllPlacementConstraints(protoList);
   }
 
   private void mergeLocalToProto() {
@@ -90,7 +131,8 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
 
   @Override
   public String getHost() {
-    RegisterApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getHost();
   }
 
@@ -106,7 +148,8 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
 
   @Override
   public int getRpcPort() {
-    RegisterApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getRpcPort();
   }
 
@@ -118,7 +161,8 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
 
   @Override
   public String getTrackingUrl() {
-    RegisterApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getTrackingUrl();
   }
 
@@ -131,4 +175,50 @@ public class RegisterApplicationMasterRequestPBImpl extends RegisterApplicationM
     }
     builder.setTrackingUrl(url);
   }
-}  
+
+  private void initPlacementConstraintMap() {
+    if (this.placementConstraints != null) {
+      return;
+    }
+    RegisterApplicationMasterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<YarnProtos.PlacementConstraintMapEntryProto> pcmList =
+        p.getPlacementConstraintsList();
+    this.placementConstraints = new HashMap<>();
+    for (YarnProtos.PlacementConstraintMapEntryProto e : pcmList) {
+      this.placementConstraints.put(
+          new HashSet<>(e.getAllocationTagsList()),
+          new PlacementConstraintFromProtoConverter(
+              e.getPlacementConstraint()).convert());
+    }
+  }
+
+  @Override
+  public Map<Set<String>, PlacementConstraint> getPlacementConstraints() {
+    initPlacementConstraintMap();
+    return this.placementConstraints;
+  }
+
+  @Override
+  public void setPlacementConstraints(
+      Map<Set<String>, PlacementConstraint> constraints) {
+    maybeInitBuilder();
+    if (constraints == null) {
+      builder.clearPlacementConstraints();
+    } else {
+      removeEmptyKeys(constraints);
+    }
+    this.placementConstraints = constraints;
+  }
+
+  private void removeEmptyKeys(
+      Map<Set<String>, PlacementConstraint> constraintMap) {
+    Iterator<Set<String>> iter = constraintMap.keySet().iterator();
+    while (iter.hasNext()) {
+      Set<String> aTags = iter.next();
+      if (aTags.size() == 0) {
+        iter.remove();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b9faf58/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
index 8694651..ebd66af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
@@ -22,12 +22,19 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.junit.Assert;
 
 import java.lang.reflect.*;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints
+    .PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+
 /**
  * Generic helper class to validate protocol records.
  */
@@ -85,6 +92,10 @@ public class BasePBImplRecordsTest {
         ByteBuffer buff = ByteBuffer.allocate(4);
         rand.nextBytes(buff.array());
         return buff;
+      } else if (type.equals(PlacementConstraint.class)) {
+        PlacementConstraint.AbstractConstraint sConstraintExpr =
+            targetIn(NODE, allocationTag("foo"));
+        ret = PlacementConstraints.build(sConstraintExpr);
       }
     } else if (type instanceof ParameterizedType) {
       ParameterizedType pt = (ParameterizedType)type;


---------------------------------------------------------------------
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-11673. [READ] Handle failures of Datanode with PROVIDED storage

Posted by kk...@apache.org.
HDFS-11673. [READ] Handle failures of Datanode 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/546b95f4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/546b95f4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/546b95f4

Branch: refs/heads/YARN-6592
Commit: 546b95f4843f3cbbbdf72d90d202cad551696082
Parents: 55ade54
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Jun 1 16:01:31 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockInfo.java  | 12 +++-
 .../server/blockmanagement/BlockManager.java    |  5 +-
 .../server/blockmanagement/BlockProvider.java   | 18 +++--
 .../blockmanagement/ProvidedStorageMap.java     | 54 +++++++++++++--
 .../blockmanagement/TestProvidedStorageMap.java | 10 ++-
 .../TestNameNodeProvidedImplementation.java     | 72 +++++++++++++++++++-
 6 files changed, 150 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/546b95f4/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 e9d235c..eb09b7b 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
@@ -24,6 +24,7 @@ import java.util.NoSuchElementException;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -188,8 +189,15 @@ public abstract class BlockInfo extends Block
     int len = getCapacity();
     for(int idx = 0; idx < len; idx++) {
       DatanodeStorageInfo cur = getStorageInfo(idx);
-      if(cur != null && cur.getDatanodeDescriptor() == dn) {
-        return cur;
+      if(cur != null) {
+        if (cur.getStorageType() == StorageType.PROVIDED) {
+          //if block resides on provided storage, only match the storage ids
+          if (dn.getStorageInfo(cur.getStorageID()) != null) {
+            return cur;
+          }
+        } else if (cur.getDatanodeDescriptor() == dn) {
+          return cur;
+        }
       }
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/546b95f4/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 0e3eab3..07502c1 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
@@ -1514,6 +1514,7 @@ public class BlockManager implements BlockStatsMXBean {
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
+    providedStorageMap.removeDatanode(node);
     for (DatanodeStorageInfo storage : node.getStorageInfos()) {
       final Iterator<BlockInfo> it = storage.getBlockIterator();
       //add the BlockInfos to a new collection as the
@@ -2462,7 +2463,7 @@ public class BlockManager implements BlockStatsMXBean {
       // !#! 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);
+          providedStorageMap.getStorage(node, storage, context);
 
       if (storageInfo == null) {
         // We handle this for backwards compatibility.
@@ -2589,7 +2590,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
   
-  private Collection<Block> processReport(
+  Collection<Block> processReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report,
       BlockReportContext context) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/546b95f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
index d8bed16..2214868 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.io.IOException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap.ProvidedBlockList;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,14 +53,23 @@ public abstract class BlockProvider implements Iterable<Block> {
    * start the processing of block report for provided blocks.
    * @throws IOException
    */
-  void start() throws IOException {
+  void start(BlockReportContext context) throws IOException {
     assert lock.hasWriteLock() : "Not holding write lock";
     if (hasDNs) {
       return;
     }
-    LOG.info("Calling process first blk report from storage: " + storage);
-    // first pass; periodic refresh should call bm.processReport
-    bm.processFirstBlockReport(storage, new ProvidedBlockList(iterator()));
+    if (storage.getBlockReportCount() == 0) {
+      LOG.info("Calling process first blk report from storage: " + storage);
+      // first pass; periodic refresh should call bm.processReport
+      bm.processFirstBlockReport(storage, new ProvidedBlockList(iterator()));
+    } else {
+      bm.processReport(storage, new ProvidedBlockList(iterator()), context);
+    }
     hasDNs = true;
   }
+
+  void stop() {
+    assert lock.hasWriteLock() : "Not holding write lock";
+    hasDNs = false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/546b95f4/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 0faf16d..5717e0c 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -103,17 +104,18 @@ public class ProvidedStorageMap {
   /**
    * @param dn datanode descriptor
    * @param s data node storage
+   * @param context the block report context
    * @return the {@link DatanodeStorageInfo} for the specified datanode.
    * If {@code s} corresponds to a provided storage, the storage info
    * representing provided storage is returned.
    * @throws IOException
    */
-  DatanodeStorageInfo getStorage(DatanodeDescriptor dn, DatanodeStorage s)
-      throws IOException {
+  DatanodeStorageInfo getStorage(DatanodeDescriptor dn, DatanodeStorage s,
+      BlockReportContext context) throws IOException {
     if (providedEnabled && storageId.equals(s.getStorageID())) {
       if (StorageType.PROVIDED.equals(s.getStorageType())) {
         // poll service, initiate
-        blockProvider.start();
+        blockProvider.start(context);
         dn.injectStorage(providedStorageInfo);
         return providedDescriptor.getProvidedStorage(dn, s);
       }
@@ -134,6 +136,15 @@ public class ProvidedStorageMap {
     return new ProvidedBlocksBuilder(maxValue);
   }
 
+  public void removeDatanode(DatanodeDescriptor dnToRemove) {
+    if (providedDescriptor != null) {
+      int remainingDatanodes = providedDescriptor.remove(dnToRemove);
+      if (remainingDatanodes == 0) {
+        blockProvider.stop();
+      }
+    }
+  }
+
   /**
    * Builder used for creating {@link LocatedBlocks} when a block is provided.
    */
@@ -282,7 +293,7 @@ public class ProvidedStorageMap {
 
     DatanodeStorageInfo createProvidedStorage(DatanodeStorage ds) {
       assert null == storageMap.get(ds.getStorageID());
-      DatanodeStorageInfo storage = new DatanodeStorageInfo(this, ds);
+      DatanodeStorageInfo storage = new ProvidedDatanodeStorageInfo(this, ds);
       storage.setHeartbeatedSinceFailover(true);
       storageMap.put(storage.getStorageID(), storage);
       return storage;
@@ -381,6 +392,22 @@ public class ProvidedStorageMap {
       }
     }
 
+    int remove(DatanodeDescriptor dnToRemove) {
+      // this operation happens under the FSNamesystem lock;
+      // no additional synchronization required.
+      if (dnToRemove != null) {
+        DatanodeDescriptor storedDN = dns.get(dnToRemove.getDatanodeUuid());
+        if (storedDN != null) {
+          dns.remove(dnToRemove.getDatanodeUuid());
+        }
+      }
+      return dns.size();
+    }
+
+    int activeProvidedDatanodes() {
+      return dns.size();
+    }
+
     @Override
     public boolean equals(Object obj) {
       return (this == obj) || super.equals(obj);
@@ -393,6 +420,25 @@ public class ProvidedStorageMap {
   }
 
   /**
+   * The DatanodeStorageInfo used for the provided storage.
+   */
+  static class ProvidedDatanodeStorageInfo extends DatanodeStorageInfo {
+
+    ProvidedDatanodeStorageInfo(ProvidedDescriptor dn, DatanodeStorage ds) {
+      super(dn, ds);
+    }
+
+    @Override
+    boolean removeBlock(BlockInfo b) {
+      ProvidedDescriptor dn = (ProvidedDescriptor) getDatanodeDescriptor();
+      if (dn.activeProvidedDatanodes() == 0) {
+        return super.removeBlock(b);
+      } else {
+        return false;
+      }
+    }
+  }
+  /**
    * Used to emulate block reports for provided blocks.
    */
   static class ProvidedBlockList extends BlockListAsLongs {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/546b95f4/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 50e2fed..2296c82 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
@@ -119,9 +119,9 @@ public class TestProvidedStorageMap {
 
     when(nameSystemLock.hasWriteLock()).thenReturn(true);
     DatanodeStorageInfo dns1Provided = providedMap.getStorage(dn1,
-            dn1ProvidedStorage);
+            dn1ProvidedStorage, null);
     DatanodeStorageInfo dns1Disk = providedMap.getStorage(dn1,
-            dn1DiskStorage);
+            dn1DiskStorage, null);
 
     assertTrue("The provided storages should be equal",
             dns1Provided == providedMapStorage);
@@ -131,7 +131,7 @@ public class TestProvidedStorageMap {
     DatanodeStorageInfo dnsDisk = new DatanodeStorageInfo(dn1, dn1DiskStorage);
     dn1.injectStorage(dnsDisk);
     assertTrue("Disk storage must match the injected storage info",
-            dnsDisk == providedMap.getStorage(dn1, dn1DiskStorage));
+            dnsDisk == providedMap.getStorage(dn1, dn1DiskStorage, null));
 
     //create a 2nd datanode
     DatanodeDescriptor dn2 = createDatanodeDescriptor(5010);
@@ -142,12 +142,10 @@ public class TestProvidedStorageMap {
             StorageType.PROVIDED);
 
     DatanodeStorageInfo dns2Provided = providedMap.getStorage(
-            dn2, dn2ProvidedStorage);
+            dn2, dn2ProvidedStorage, null);
     assertTrue("The provided storages should be equal",
             dns2Provided == providedMapStorage);
     assertTrue("The DatanodeDescriptor should contain the provided storage",
             dn2.getStorageInfo(providedStorageID) == providedMapStorage);
-
-
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/546b95f4/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 e171557..60b306f 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
@@ -45,11 +45,14 @@ 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.blockmanagement.BlockFormatProvider;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
 import org.apache.hadoop.hdfs.server.common.BlockFormat;
 import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
 import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
 import org.apache.hadoop.hdfs.server.common.TextFileRegionProvider;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 
 import org.junit.After;
@@ -406,9 +409,9 @@ public class TestNameNodeProvidedImplementation {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
     startCluster(NNDIRPATH, 2, null,
-        new StorageType[][] {
-                {StorageType.PROVIDED},
-                {StorageType.DISK}},
+        new StorageType[][]{
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
         false);
 
     String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
@@ -433,4 +436,67 @@ public class TestNameNodeProvidedImplementation {
     assertEquals(cluster.getDataNodes().get(0).getDatanodeUuid(),
         infos[0].getDatanodeUuid());
   }
+
+  @Test
+  public void testProvidedDatanodeFailures() throws Exception {
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+            FixedBlockResolver.class);
+    startCluster(NNDIRPATH, 3, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED},
+            {StorageType.PROVIDED},
+            {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));
+
+    if (numFiles >= 1) {
+      String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
+
+      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      //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, 1);
+      assertEquals(providedDatanode2.getDatanodeUuid(),
+          dnInfos[0].getDatanodeUuid());
+
+      //stop the 2nd provided datanode
+      cluster.stopDataNode(1);
+      // make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode2.getDatanodeId().getXferAddr());
+
+      getAndCheckBlockLocations(client, filename, 0);
+
+      //restart the provided datanode
+      cluster.restartDataNode(providedDNProperties1, true);
+      cluster.waitActive();
+
+      //should find the block on the 1st provided datanode now
+      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      //not comparing UUIDs as the datanode can now have a different one.
+      assertEquals(providedDatanode1.getDatanodeId().getXferAddr(),
+          dnInfos[0].getXferAddr());
+    }
+  }
 }


---------------------------------------------------------------------
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-7664. Several javadoc errors. Contributed by Sean Mackrory.

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


[50/50] [abbrv] hadoop git commit: YARN-6593. [API] Introduce Placement Constraint object. (Konstantinos Karanasos via wangda)

Posted by kk...@apache.org.
YARN-6593. [API] Introduce Placement Constraint object. (Konstantinos Karanasos via wangda)

Change-Id: Id00edb7185fdf01cce6e40f920cac3585f8cbe9c


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

Branch: refs/heads/YARN-6592
Commit: 45b1ca602814b03a5df35494f2edc7beab3d3db2
Parents: c7a4dda
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Aug 3 14:03:55 2017 -0700
Committer: Konstantinos Karanasos <kk...@apache.org>
Committed: Mon Dec 18 16:07:00 2017 -0800

----------------------------------------------------------------------
 .../yarn/api/resource/PlacementConstraint.java  | 567 +++++++++++++++++++
 .../yarn/api/resource/PlacementConstraints.java | 286 ++++++++++
 .../hadoop/yarn/api/resource/package-info.java  |  23 +
 .../src/main/proto/yarn_protos.proto            |  55 ++
 .../api/resource/TestPlacementConstraints.java  | 106 ++++
 .../PlacementConstraintFromProtoConverter.java  | 116 ++++
 .../pb/PlacementConstraintToProtoConverter.java | 174 ++++++
 .../apache/hadoop/yarn/api/pb/package-info.java |  23 +
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  27 +
 .../PlacementConstraintTransformations.java     | 209 +++++++
 .../hadoop/yarn/api/resource/package-info.java  |  23 +
 .../TestPlacementConstraintPBConversion.java    | 195 +++++++
 .../TestPlacementConstraintTransformations.java | 183 ++++++
 13 files changed, 1987 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
new file mode 100644
index 0000000..f0e3982
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
@@ -0,0 +1,567 @@
+/**
+ * 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.resource;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * {@code PlacementConstraint} represents a placement constraint for a resource
+ * allocation.
+ */
+@Public
+@Unstable
+public class PlacementConstraint {
+
+  /**
+   * The constraint expression tree.
+   */
+  private AbstractConstraint constraintExpr;
+
+  public PlacementConstraint(AbstractConstraint constraintExpr) {
+    this.constraintExpr = constraintExpr;
+  }
+
+  /**
+   * Get the constraint expression of the placement constraint.
+   *
+   * @return the constraint expression
+   */
+  public AbstractConstraint getConstraintExpr() {
+    return constraintExpr;
+  }
+
+  /**
+   * Interface used to enable the elements of the constraint tree to be visited.
+   */
+  @Private
+  public interface Visitable {
+    /**
+     * Visitor pattern.
+     *
+     * @param visitor visitor to be used
+     * @param <T> defines the type that the visitor will use and the return type
+     *          of the accept.
+     * @return the result of visiting a given object.
+     */
+    <T> T accept(Visitor<T> visitor);
+
+  }
+
+  /**
+   * Visitor API for a constraint tree.
+   *
+   * @param <T> determines the return type of the visit methods.
+   */
+  @Private
+  public interface Visitor<T> {
+    T visit(SingleConstraint constraint);
+
+    T visit(TargetExpression target);
+
+    T visit(TargetConstraint constraint);
+
+    T visit(CardinalityConstraint constraint);
+
+    T visit(And constraint);
+
+    T visit(Or constraint);
+
+    T visit(DelayedOr constraint);
+
+    T visit(TimedPlacementConstraint constraint);
+  }
+
+  /**
+   * Abstract class that acts as the superclass of all placement constraint
+   * classes.
+   */
+  public abstract static class AbstractConstraint implements Visitable {
+    public PlacementConstraint build() {
+      return new PlacementConstraint(this);
+    }
+  }
+
+  static final String NODE_SCOPE = "node";
+  static final String RACK_SCOPE = "rack";
+
+  /**
+   * Consider a set of nodes N that belongs to the scope specified in the
+   * constraint. If the target expressions are satisfied at least minCardinality
+   * times and at most max-cardinality times in the node set N, then the
+   * constraint is satisfied.
+   *
+   * For example, a constraint of the form {@code {RACK, 2, 10,
+   * allocationTag("zk")}}, requires an allocation to be placed within a rack
+   * that has at least 2 and at most 10 other allocations with tag "zk".
+   */
+  public static class SingleConstraint extends AbstractConstraint {
+    private String scope;
+    private int minCardinality;
+    private int maxCardinality;
+    private Set<TargetExpression> targetExpressions;
+
+    public SingleConstraint(String scope, int minCardinality,
+        int maxCardinality, Set<TargetExpression> targetExpressions) {
+      this.scope = scope;
+      this.minCardinality = minCardinality;
+      this.maxCardinality = maxCardinality;
+      this.targetExpressions = targetExpressions;
+    }
+
+    public SingleConstraint(String scope, int minC, int maxC,
+        TargetExpression... targetExpressions) {
+      this(scope, minC, maxC, new HashSet<>(Arrays.asList(targetExpressions)));
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the minimum cardinality of the constraint.
+     *
+     * @return the minimum cardinality of the constraint
+     */
+    public int getMinCardinality() {
+      return minCardinality;
+    }
+
+    /**
+     * Get the maximum cardinality of the constraint.
+     *
+     * @return the maximum cardinality of the constraint
+     */
+    public int getMaxCardinality() {
+      return maxCardinality;
+    }
+
+    /**
+     * Get the target expressions of the constraint.
+     *
+     * @return the set of target expressions
+     */
+    public Set<TargetExpression> getTargetExpressions() {
+      return targetExpressions;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class representing the target expressions that are used in placement
+   * constraints. They might refer to expressions on node attributes, allocation
+   * tags, or be self-targets (referring to the allocation to which the
+   * constraint is attached).
+   */
+  public static class TargetExpression implements Visitable {
+    /**
+     * Enum specifying the type of the target expression.
+     */
+    public enum TargetType {
+      NODE_ATTRIBUTE, ALLOCATION_TAG, SELF
+    }
+
+    private TargetType targetType;
+    private String targetKey;
+    private Set<String> targetValues;
+
+    public TargetExpression(TargetType targetType, String targetKey,
+        Set<String> targetValues) {
+      this.targetType = targetType;
+      this.targetKey = targetKey;
+      this.targetValues = targetValues;
+    }
+
+    public TargetExpression(TargetType targetType) {
+      this(targetType, null, new HashSet<>());
+    }
+
+    public TargetExpression(TargetType targetType, String targetKey,
+        String... targetValues) {
+      this(targetType, targetKey, new HashSet<>(Arrays.asList(targetValues)));
+    }
+
+    /**
+     * Get the type of the target expression.
+     *
+     * @return the type of the target expression
+     */
+    public TargetType getTargetType() {
+      return targetType;
+    }
+
+    /**
+     * Get the key of the target expression.
+     *
+     * @return the key of the target expression
+     */
+    public String getTargetKey() {
+      return targetKey;
+    }
+
+    /**
+     * Get the set of values of the target expression.
+     *
+     * @return the set of values of the target expression
+     */
+    public Set<String> getTargetValues() {
+      return targetValues;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = targetType != null ? targetType.hashCode() : 0;
+      result = 31 * result + (targetKey != null ? targetKey.hashCode() : 0);
+      result =
+          31 * result + (targetValues != null ? targetValues.hashCode() : 0);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null) {
+        return false;
+      }
+      if (!(o instanceof TargetExpression)) {
+        return false;
+      }
+
+      TargetExpression that = (TargetExpression) o;
+      if (targetType != that.targetType) {
+        return false;
+      }
+      if (targetKey != null ? !targetKey.equals(that.targetKey)
+          : that.targetKey != null) {
+        return false;
+      }
+      return targetValues != null ? targetValues.equals(that.targetValues)
+          : that.targetValues == null;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a target constraint. Such a constraint requires an
+   * allocation to be placed within a scope that satisfies some specified
+   * expressions on node attributes and allocation tags.
+   *
+   * It is a specialized version of the {@link SingleConstraint}, where the
+   * minimum and the maximum cardinalities take specific values based on the
+   * {@link TargetOperator} used.
+   */
+  public static class TargetConstraint extends AbstractConstraint {
+    enum TargetOperator {
+      IN, NOT_IN
+    }
+
+    private TargetOperator op;
+    private String scope;
+    private Set<TargetExpression> targetExpressions;
+
+    public TargetConstraint(TargetOperator op, String scope,
+        Set<TargetExpression> targetExpressions) {
+      this.op = op;
+      this.scope = scope;
+      this.targetExpressions = targetExpressions;
+    }
+
+    /**
+     * Get the target operator of the constraint.
+     *
+     * @return the target operator
+     */
+    public TargetOperator getOp() {
+      return op;
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the set of target expressions.
+     *
+     * @return the set of target expressions
+     */
+    public Set<TargetExpression> getTargetExpressions() {
+      return targetExpressions;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a cardinality constraint. Such a constraint the
+   * number of allocations within a given scope to some minimum and maximum
+   * values.
+   *
+   * It is a specialized version of the {@link SingleConstraint}, where the
+   * target is self (i.e., the allocation to which the constraint is attached).
+   */
+  public static class CardinalityConstraint extends AbstractConstraint {
+    private String scope;
+    private int minCardinality;
+    private int maxCardinality;
+
+    public CardinalityConstraint(String scope, int minCardinality,
+        int maxCardinality) {
+      this.scope = scope;
+      this.minCardinality = minCardinality;
+      this.maxCardinality = maxCardinality;
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the minimum cardinality of the constraint.
+     *
+     * @return the minimum cardinality of the constraint
+     */
+    public int getMinCardinality() {
+      return minCardinality;
+    }
+
+    /**
+     * Get the maximum cardinality of the constraint.
+     *
+     * @return the maximum cardinality of the constraint
+     */
+    public int getMaxCardinality() {
+      return maxCardinality;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents composite constraints, which comprise other
+   * constraints, forming a constraint tree.
+   *
+   * @param <R> the type of constraints that are used as children of the
+   *          specific composite constraint
+   */
+  public abstract static class CompositeConstraint<R extends Visitable>
+      extends AbstractConstraint {
+
+    /**
+     * Get the children of this composite constraint.
+     *
+     * @return the children of the composite constraint
+     */
+    public abstract List<R> getChildren();
+  }
+
+  /**
+   * Class that represents a composite constraint that is a conjunction of other
+   * constraints.
+   */
+  public static class And extends CompositeConstraint<AbstractConstraint> {
+    private List<AbstractConstraint> children;
+
+    public And(List<AbstractConstraint> children) {
+      this.children = children;
+    }
+
+    public And(AbstractConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<AbstractConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a composite constraint that is a disjunction of other
+   * constraints.
+   */
+  public static class Or extends CompositeConstraint<AbstractConstraint> {
+    private List<AbstractConstraint> children;
+
+    public Or(List<AbstractConstraint> children) {
+      this.children = children;
+    }
+
+    public Or(AbstractConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<AbstractConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a composite constraint that comprises a list of timed
+   * placement constraints (see {@link TimedPlacementConstraint}). The scheduler
+   * should try to satisfy first the first timed child constraint within the
+   * specified time window. If this is not possible, it should attempt to
+   * satisfy the second, and so on.
+   */
+  public static class DelayedOr
+      extends CompositeConstraint<TimedPlacementConstraint> {
+    private List<TimedPlacementConstraint> children = new ArrayList<>();
+
+    public DelayedOr(List<TimedPlacementConstraint> children) {
+      this.children = children;
+    }
+
+    public DelayedOr(TimedPlacementConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<TimedPlacementConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Represents a timed placement constraint that has to be satisfied within a
+   * time window.
+   */
+  public static class TimedPlacementConstraint implements Visitable {
+    /**
+     * The unit of scheduling delay.
+     */
+    public enum DelayUnit {
+      MILLISECONDS, OPPORTUNITIES
+    }
+
+    private AbstractConstraint constraint;
+    private long schedulingDelay;
+    private DelayUnit delayUnit;
+
+    public TimedPlacementConstraint(AbstractConstraint constraint,
+        long schedulingDelay, DelayUnit delayUnit) {
+      this.constraint = constraint;
+      this.schedulingDelay = schedulingDelay;
+      this.delayUnit = delayUnit;
+    }
+
+    public TimedPlacementConstraint(AbstractConstraint constraint,
+        long schedulingDelay) {
+      this(constraint, schedulingDelay, DelayUnit.MILLISECONDS);
+    }
+
+    public TimedPlacementConstraint(AbstractConstraint constraint) {
+      this(constraint, Long.MAX_VALUE, DelayUnit.MILLISECONDS);
+    }
+
+    /**
+     * Get the constraint that has to be satisfied within the time window.
+     *
+     * @return the constraint to be satisfied
+     */
+    public AbstractConstraint getConstraint() {
+      return constraint;
+    }
+
+    /**
+     * Sets the constraint that has to be satisfied within the time window.
+     *
+     * @param constraint the constraint to be satisfied
+     */
+    public void setConstraint(AbstractConstraint constraint) {
+      this.constraint = constraint;
+    }
+
+    /**
+     * Get the scheduling delay value that determines the time window within
+     * which the constraint has to be satisfied.
+     *
+     * @return the value of the scheduling delay
+     */
+    public long getSchedulingDelay() {
+      return schedulingDelay;
+    }
+
+    /**
+     * The unit of the scheduling delay.
+     *
+     * @return the unit of the delay
+     */
+    public DelayUnit getDelayUnit() {
+      return delayUnit;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
new file mode 100644
index 0000000..8e84280
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
@@ -0,0 +1,286 @@
+/**
+ * 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.resource;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+
+/**
+ * This class contains various static methods for the applications to create
+ * placement constraints (see also {@link PlacementConstraint}).
+ */
+@Public
+@Unstable
+public final class PlacementConstraints {
+
+  // Suppresses default constructor, ensuring non-instantiability.
+  private PlacementConstraints() {
+  }
+
+  // Creation of simple constraints.
+
+  public static final String NODE = PlacementConstraint.NODE_SCOPE;
+  public static final String RACK = PlacementConstraint.RACK_SCOPE;
+
+  /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * satisfy all target expressions within the given scope (e.g., node or rack).
+   *
+   * For example, {@code targetIn(RACK, allocationTag("hbase-m"))}, allows
+   * allocations on nodes that belong to a rack that has at least one tag with
+   * value "hbase-m".
+   *
+   * @param scope the scope within which the target expressions should be
+   *          satisfied
+   * @param targetExpressions the expressions that need to be satisfied within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetIn(String scope,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, 1, Integer.MAX_VALUE, targetExpressions);
+  }
+
+  /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * belong to a scope (e.g., node or rack) that does not satisfy any of the
+   * target expressions.
+   *
+   * @param scope the scope within which the target expressions should not be
+   *          true
+   * @param targetExpressions the expressions that need to not be true within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetNotIn(String scope,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, 0, 0, targetExpressions);
+  }
+
+  /**
+   * Creates a constraint that restricts the number of allocations within a
+   * given scope (e.g., node or rack).
+   *
+   * For example, {@code cardinality(NODE, 3, 10)}, restricts the number of
+   * allocations per node to be no less than 3 and no more than 10.
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality determines the minimum number of allocations within
+   *          the scope
+   * @param maxCardinality determines the maximum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint cardinality(String scope, int minCardinality,
+      int maxCardinality) {
+    return new SingleConstraint(scope, minCardinality, maxCardinality,
+        PlacementTargets.self());
+  }
+
+  /**
+   * Similar to {@link #cardinality(String, int, int)}, but determines only the
+   * minimum cardinality (the maximum cardinality is unbound).
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality determines the minimum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint minCardinality(String scope,
+      int minCardinality) {
+    return cardinality(scope, minCardinality, Integer.MAX_VALUE);
+  }
+
+  /**
+   * Similar to {@link #cardinality(String, int, int)}, but determines only the
+   * maximum cardinality (the minimum can be as low as 0).
+   *
+   * @param scope the scope of the constraint
+   * @param maxCardinality determines the maximum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint maxCardinality(String scope,
+      int maxCardinality) {
+    return cardinality(scope, 0, maxCardinality);
+  }
+
+  /**
+   * This constraint generalizes the cardinality and target constraints.
+   *
+   * Consider a set of nodes N that belongs to the scope specified in the
+   * constraint. If the target expressions are satisfied at least minCardinality
+   * times and at most max-cardinality times in the node set N, then the
+   * constraint is satisfied.
+   *
+   * For example, {@code targetCardinality(RACK, 2, 10, allocationTag("zk"))},
+   * requires an allocation to be placed within a rack that has at least 2 and
+   * at most 10 other allocations with tag "zk".
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality the minimum number of times the target expressions
+   *          have to be satisfied with the given scope
+   * @param maxCardinality the maximum number of times the target expressions
+   *          have to be satisfied with the given scope
+   * @param targetExpressions the target expressions
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetCardinality(String scope,
+      int minCardinality, int maxCardinality,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, minCardinality, maxCardinality,
+        targetExpressions);
+  }
+
+  // Creation of target expressions to be used in simple constraints.
+
+  /**
+   * Class with static methods for constructing target expressions to be used in
+   * placement constraints.
+   */
+  public static class PlacementTargets {
+
+    /**
+     * Constructs a target expression on a node attribute. It is satisfied if
+     * the specified node attribute has one of the specified values.
+     *
+     * @param attributeKey the name of the node attribute
+     * @param attributeValues the set of values that the attribute should take
+     *          values from
+     * @return the resulting expression on the node attribute
+     */
+    public static TargetExpression nodeAttribute(String attributeKey,
+        String... attributeValues) {
+      return new TargetExpression(TargetType.NODE_ATTRIBUTE, attributeKey,
+          attributeValues);
+    }
+
+    /**
+     * Constructs a target expression on an allocation tag. It is satisfied if
+     * the there are allocations with one of the given tags.
+     *
+     * @param allocationTags the set of tags that the attribute should take
+     *          values from
+     * @return the resulting expression on the allocation tags
+     */
+    public static TargetExpression allocationTag(String... allocationTags) {
+      return new TargetExpression(TargetType.ALLOCATION_TAG, null,
+          allocationTags);
+    }
+
+    /**
+     * The default target expression that uses as target the allocation that
+     * specifies the constraint.
+     *
+     * @return the self-target
+     */
+    public static TargetExpression self() {
+      return new TargetExpression(TargetType.SELF);
+    }
+  }
+
+  // Creation of compound constraints.
+
+  /**
+   * A conjunction of constraints.
+   *
+   * @param children the children constraints that should all be satisfied
+   * @return the resulting placement constraint
+   */
+  public static And and(AbstractConstraint... children) {
+    return new And(children);
+  }
+
+  /**
+   * A disjunction of constraints.
+   *
+   * @param children the children constraints, one of which should be satisfied
+   * @return the resulting placement constraint
+   */
+  public static Or or(AbstractConstraint... children) {
+    return new Or(children);
+  }
+
+  /**
+   * Creates a composite constraint that includes a list of timed placement
+   * constraints. The scheduler should try to satisfy first the first timed
+   * child constraint within the specified time window. If this is not possible,
+   * it should attempt to satisfy the second, and so on.
+   *
+   * @param children the timed children constraints
+   * @return the resulting composite constraint
+   */
+  public static DelayedOr delayedOr(TimedPlacementConstraint... children) {
+    return new DelayedOr(children);
+  }
+
+  // Creation of timed constraints to be used in a DELAYED_OR constraint.
+
+  /**
+   * Creates a placement constraint that has to be satisfied within a time
+   * window.
+   *
+   * @param constraint the placement constraint
+   * @param delay the length of the time window within which the constraint has
+   *          to be satisfied
+   * @param timeUnit the unit of time of the time window
+   * @return the resulting timed placement constraint
+   */
+  public static TimedPlacementConstraint timedClockConstraint(
+      AbstractConstraint constraint, long delay, TimeUnit timeUnit) {
+    return new TimedPlacementConstraint(constraint, timeUnit.toMillis(delay),
+        TimedPlacementConstraint.DelayUnit.MILLISECONDS);
+  }
+
+  /**
+   * Creates a placement constraint that has to be satisfied within a number of
+   * placement opportunities (invocations of the scheduler).
+   *
+   * @param constraint the placement constraint
+   * @param delay the number of scheduling opportunities within which the
+   *          constraint has to be satisfied
+   * @return the resulting timed placement constraint
+   */
+  public static TimedPlacementConstraint timedOpportunitiesConstraint(
+      AbstractConstraint constraint, long delay) {
+    return new TimedPlacementConstraint(constraint, delay,
+        TimedPlacementConstraint.DelayUnit.OPPORTUNITIES);
+  }
+
+  /**
+   * Creates a {@link PlacementConstraint} given a constraint expression.
+   *
+   * @param constraintExpr the constraint expression
+   * @return the placement constraint
+   */
+  public static PlacementConstraint build(AbstractConstraint constraintExpr) {
+    return constraintExpr.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
new file mode 100644
index 0000000..660dc02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * API related to resources.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.resource;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/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..968b75e 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
@@ -578,6 +578,61 @@ enum SignalContainerCommandProto {
   FORCEFUL_SHUTDOWN = 3;
 }
 
+////////////////////////////////////////////////////////////////////////
+////// Placement constraints ///////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////
+
+message PlacementConstraintProto {
+  optional SimplePlacementConstraintProto simpleConstraint = 1;
+  optional CompositePlacementConstraintProto compositeConstraint = 2;
+}
+
+message SimplePlacementConstraintProto {
+  required string scope = 1;
+  repeated PlacementConstraintTargetProto targetExpressions = 2;
+  optional int32 minCardinality = 3;
+  optional int32 maxCardinality = 4;
+}
+
+message PlacementConstraintTargetProto {
+  enum TargetType {
+    NODE_ATTRIBUTE = 1;
+    ALLOCATION_TAG = 2;
+    SELF = 3;
+  }
+
+  required TargetType targetType = 1;
+  optional string targetKey = 2;
+  repeated string targetValues = 3;
+}
+
+message TimedPlacementConstraintProto {
+  enum DelayUnit {
+    MILLISECONDS = 1;
+    OPPORTUNITIES = 2;
+  }
+
+  required PlacementConstraintProto placementConstraint = 1;
+  required int64 schedulingDelay = 2;
+  optional DelayUnit delayUnit = 3 [ default = MILLISECONDS ];
+}
+
+message CompositePlacementConstraintProto {
+  enum CompositeType {
+    // All children constraints have to be satisfied.
+    AND = 1;
+    // One of the children constraints has to be satisfied.
+    OR = 2;
+    // Attempt to satisfy the first child constraint for delays[0] units (e.g.,
+    // millisec or heartbeats). If this fails, try to satisfy the second child
+    // constraint for delays[1] units and so on.
+    DELAYED_OR = 3;
+  }
+
+  required CompositeType compositeType = 1;
+  repeated PlacementConstraintProto childConstraints = 2;
+  repeated TimedPlacementConstraintProto timedChildConstraints = 3;
+}
 
 ////////////////////////////////////////////////////////////////////////
 ////// From reservation_protocol /////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
new file mode 100644
index 0000000..e25d477
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
@@ -0,0 +1,106 @@
+/**
+ * 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.resource;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.nodeAttribute;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for the various static methods in
+ * {@link org.apache.hadoop.yarn.api.resource.PlacementConstraints}.
+ */
+public class TestPlacementConstraints {
+
+  @Test
+  public void testNodeAffinityToTag() {
+    AbstractConstraint constraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+
+    SingleConstraint sConstraint = (SingleConstraint) constraintExpr;
+    Assert.assertEquals(NODE, sConstraint.getScope());
+    Assert.assertEquals(1, sConstraint.getMinCardinality());
+    Assert.assertEquals(Integer.MAX_VALUE, sConstraint.getMaxCardinality());
+
+    Assert.assertEquals(1, sConstraint.getTargetExpressions().size());
+    TargetExpression tExpr =
+        sConstraint.getTargetExpressions().iterator().next();
+    Assert.assertNull(tExpr.getTargetKey());
+    Assert.assertEquals(TargetType.ALLOCATION_TAG, tExpr.getTargetType());
+    Assert.assertEquals(1, tExpr.getTargetValues().size());
+    Assert.assertEquals("hbase-m", tExpr.getTargetValues().iterator().next());
+
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Assert.assertNotNull(constraint.getConstraintExpr());
+  }
+
+  @Test
+  public void testNodeAntiAffinityToAttribute() {
+    AbstractConstraint constraintExpr =
+        targetNotIn(NODE, nodeAttribute("java", "1.8"));
+
+    SingleConstraint sConstraint = (SingleConstraint) constraintExpr;
+    Assert.assertEquals(NODE, sConstraint.getScope());
+    Assert.assertEquals(0, sConstraint.getMinCardinality());
+    Assert.assertEquals(0, sConstraint.getMaxCardinality());
+
+    Assert.assertEquals(1, sConstraint.getTargetExpressions().size());
+    TargetExpression tExpr =
+        sConstraint.getTargetExpressions().iterator().next();
+    Assert.assertEquals("java", tExpr.getTargetKey());
+    Assert.assertEquals(TargetType.NODE_ATTRIBUTE, tExpr.getTargetType());
+    Assert.assertEquals(1, tExpr.getTargetValues().size());
+    Assert.assertEquals("1.8", tExpr.getTargetValues().iterator().next());
+  }
+
+  @Test
+  public void testAndConstraint() {
+    AbstractConstraint constraintExpr =
+        and(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+
+    And andExpr = (And) constraintExpr;
+    Assert.assertEquals(3, andExpr.getChildren().size());
+    SingleConstraint sConstr = (SingleConstraint) andExpr.getChildren().get(0);
+    TargetExpression tExpr = sConstr.getTargetExpressions().iterator().next();
+    Assert.assertEquals("spark", tExpr.getTargetValues().iterator().next());
+
+    sConstr = (SingleConstraint) andExpr.getChildren().get(1);
+    Assert.assertEquals(0, sConstr.getMinCardinality());
+    Assert.assertEquals(3, sConstr.getMaxCardinality());
+
+    sConstr = (SingleConstraint) andExpr.getChildren().get(2);
+    Assert.assertEquals(2, sConstr.getMinCardinality());
+    Assert.assertEquals(10, sConstr.getMaxCardinality());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
new file mode 100644
index 0000000..926b6fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
@@ -0,0 +1,116 @@
+/**
+ * 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.pb;
+
+import static org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType.AND;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
+
+/**
+ * {@code PlacementConstraintFromProtoConverter} generates an
+ * {@link PlacementConstraint.AbstractConstraint} given a
+ * {@link PlacementConstraintProto}.
+ */
+@Private
+public class PlacementConstraintFromProtoConverter {
+
+  private PlacementConstraintProto constraintProto;
+
+  public PlacementConstraintFromProtoConverter(
+      PlacementConstraintProto constraintProto) {
+    this.constraintProto = constraintProto;
+  }
+
+  public PlacementConstraint convert() {
+    return new PlacementConstraint(convert(constraintProto));
+  }
+
+  private AbstractConstraint convert(PlacementConstraintProto proto) {
+    return proto.hasSimpleConstraint() ? convert(proto.getSimpleConstraint())
+        : convert(proto.getCompositeConstraint());
+  }
+
+  private SingleConstraint convert(SimplePlacementConstraintProto proto) {
+    Set<TargetExpression> targets = new HashSet<>();
+    for (PlacementConstraintTargetProto tp : proto.getTargetExpressionsList()) {
+      targets.add(convert(tp));
+    }
+
+    return new SingleConstraint(proto.getScope(), proto.getMinCardinality(),
+        proto.getMaxCardinality(), targets);
+  }
+
+  private TargetExpression convert(PlacementConstraintTargetProto proto) {
+    return new TargetExpression(
+        ProtoUtils.convertFromProtoFormat(proto.getTargetType()),
+        proto.hasTargetKey() ? proto.getTargetKey() : null,
+        new HashSet<>(proto.getTargetValuesList()));
+  }
+
+  private AbstractConstraint convert(CompositePlacementConstraintProto proto) {
+    switch (proto.getCompositeType()) {
+    case AND:
+    case OR:
+      List<AbstractConstraint> children = new ArrayList<>();
+      for (PlacementConstraintProto cp : proto.getChildConstraintsList()) {
+        children.add(convert(cp));
+      }
+      return (proto.getCompositeType() == AND) ? new And(children)
+          : new Or(children);
+    case DELAYED_OR:
+      List<TimedPlacementConstraint> tChildren = new ArrayList<>();
+      for (TimedPlacementConstraintProto cp : proto
+          .getTimedChildConstraintsList()) {
+        tChildren.add(convert(cp));
+      }
+      return new DelayedOr(tChildren);
+    default:
+      throw new YarnRuntimeException(
+          "Encountered unexpected type of composite constraint.");
+    }
+  }
+
+  private TimedPlacementConstraint convert(
+      TimedPlacementConstraintProto proto) {
+    AbstractConstraint pConstraint = convert(proto.getPlacementConstraint());
+
+    return new TimedPlacementConstraint(pConstraint, proto.getSchedulingDelay(),
+        ProtoUtils.convertFromProtoFormat(proto.getDelayUnit()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
new file mode 100644
index 0000000..7816e18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
@@ -0,0 +1,174 @@
+/**
+ * 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.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CompositeConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
+
+import com.google.protobuf.GeneratedMessage;
+
+/**
+ * {@code PlacementConstraintToProtoConverter} generates a
+ * {@link PlacementConstraintProto} given a
+ * {@link PlacementConstraint.AbstractConstraint}.
+ */
+@Private
+public class PlacementConstraintToProtoConverter
+    implements PlacementConstraint.Visitor<GeneratedMessage> {
+
+  private PlacementConstraint placementConstraint;
+
+  public PlacementConstraintToProtoConverter(
+      PlacementConstraint placementConstraint) {
+    this.placementConstraint = placementConstraint;
+  }
+
+  public PlacementConstraintProto convert() {
+    return (PlacementConstraintProto) placementConstraint.getConstraintExpr()
+        .accept(this);
+  }
+
+  @Override
+  public GeneratedMessage visit(SingleConstraint constraint) {
+    SimplePlacementConstraintProto.Builder sb =
+        SimplePlacementConstraintProto.newBuilder();
+
+    if (constraint.getScope() != null) {
+      sb.setScope(constraint.getScope());
+    }
+    sb.setMinCardinality(constraint.getMinCardinality());
+    sb.setMaxCardinality(constraint.getMaxCardinality());
+    if (constraint.getTargetExpressions() != null) {
+      for (TargetExpression target : constraint.getTargetExpressions()) {
+        sb.addTargetExpressions(
+            (PlacementConstraintTargetProto) target.accept(this));
+      }
+
+    }
+    SimplePlacementConstraintProto sProto = sb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setSimpleConstraint(sProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TargetExpression target) {
+    PlacementConstraintTargetProto.Builder tb =
+        PlacementConstraintTargetProto.newBuilder();
+
+    tb.setTargetType(ProtoUtils.convertToProtoFormat(target.getTargetType()));
+    if (target.getTargetKey() != null) {
+      tb.setTargetKey(target.getTargetKey());
+    }
+    if (target.getTargetValues() != null) {
+      tb.addAllTargetValues(target.getTargetValues());
+    }
+    return tb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TargetConstraint constraint) {
+    throw new YarnRuntimeException("Unexpected TargetConstraint found.");
+  }
+
+  @Override
+  public GeneratedMessage visit(CardinalityConstraint constraint) {
+    throw new YarnRuntimeException("Unexpected CardinalityConstraint found.");
+  }
+
+  private GeneratedMessage visitAndOr(
+      CompositeConstraint<AbstractConstraint> composite, CompositeType type) {
+    CompositePlacementConstraintProto.Builder cb =
+        CompositePlacementConstraintProto.newBuilder();
+
+    cb.setCompositeType(type);
+
+    for (AbstractConstraint c : composite.getChildren()) {
+      cb.addChildConstraints((PlacementConstraintProto) c.accept(this));
+    }
+    CompositePlacementConstraintProto cProto = cb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setCompositeConstraint(cProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(And constraint) {
+    return visitAndOr(constraint, CompositeType.AND);
+  }
+
+  @Override
+  public GeneratedMessage visit(Or constraint) {
+    return visitAndOr(constraint, CompositeType.OR);
+  }
+
+  @Override
+  public GeneratedMessage visit(DelayedOr constraint) {
+    CompositePlacementConstraintProto.Builder cb =
+        CompositePlacementConstraintProto.newBuilder();
+
+    cb.setCompositeType(CompositeType.DELAYED_OR);
+
+    for (TimedPlacementConstraint c : constraint.getChildren()) {
+      cb.addTimedChildConstraints(
+          (TimedPlacementConstraintProto) c.accept(this));
+    }
+    CompositePlacementConstraintProto cProto = cb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setCompositeConstraint(cProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TimedPlacementConstraint constraint) {
+    TimedPlacementConstraintProto.Builder tb =
+        TimedPlacementConstraintProto.newBuilder();
+
+    tb.setDelayUnit(ProtoUtils.convertToProtoFormat(constraint.getDelayUnit()));
+    tb.setSchedulingDelay(constraint.getSchedulingDelay());
+    tb.setPlacementConstraint(
+        (PlacementConstraintProto) constraint.getConstraint().accept(this));
+
+    return tb.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
new file mode 100644
index 0000000..18da80f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * API related to protobuf objects that are not backed by PBImpl classes.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.pb;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index f3e665b..168d864 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
 import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
@@ -70,10 +72,12 @@ import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceVisibilityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryPolicyProto;
@@ -507,6 +511,29 @@ public class ProtoUtils {
     }
     return ret;
   }
+
+  public static PlacementConstraintTargetProto.TargetType convertToProtoFormat(
+          TargetExpression.TargetType t) {
+    return PlacementConstraintTargetProto.TargetType.valueOf(t.name());
+  }
+
+  public static TargetExpression.TargetType convertFromProtoFormat(
+          PlacementConstraintTargetProto.TargetType t) {
+    return TargetExpression.TargetType.valueOf(t.name());
+  }
+
+  /*
+   * TimedPlacementConstraint.DelayUnit
+   */
+  public static TimedPlacementConstraintProto.DelayUnit convertToProtoFormat(
+          TimedPlacementConstraint.DelayUnit u) {
+    return TimedPlacementConstraintProto.DelayUnit.valueOf(u.name());
+  }
+
+  public static TimedPlacementConstraint.DelayUnit convertFromProtoFormat(
+          TimedPlacementConstraintProto.DelayUnit u) {
+    return TimedPlacementConstraint.DelayUnit.valueOf(u.name());
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
new file mode 100644
index 0000000..e9eda6f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
@@ -0,0 +1,209 @@
+/**
+ * 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.resource;
+
+import java.util.ListIterator;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CompositeConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint.TargetOperator;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * This class contains inner classes that define transformation on a
+ * {@link PlacementConstraint} expression.
+ */
+@Private
+public class PlacementConstraintTransformations {
+
+  /**
+   * The default implementation of the {@link PlacementConstraint.Visitor} that
+   * does a traversal of the constraint tree, performing no action for the lead
+   * constraints.
+   */
+  public static class AbstractTransformer
+      implements PlacementConstraint.Visitor<AbstractConstraint> {
+
+    private PlacementConstraint placementConstraint;
+
+    public AbstractTransformer(PlacementConstraint placementConstraint) {
+      this.placementConstraint = placementConstraint;
+    }
+
+    /**
+     * This method performs the transformation of the
+     * {@link #placementConstraint}.
+     *
+     * @return the transformed placement constraint.
+     */
+    public PlacementConstraint transform() {
+      AbstractConstraint constraintExpr =
+          placementConstraint.getConstraintExpr();
+
+      // Visit the constraint tree to perform the transformation.
+      constraintExpr = constraintExpr.accept(this);
+
+      return new PlacementConstraint(constraintExpr);
+    }
+
+    @Override
+    public AbstractConstraint visit(SingleConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetExpression expression) {
+      // Do nothing.
+      return null;
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(CardinalityConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    private AbstractConstraint visitAndOr(
+        CompositeConstraint<AbstractConstraint> constraint) {
+      for (ListIterator<AbstractConstraint> iter =
+          constraint.getChildren().listIterator(); iter.hasNext();) {
+        AbstractConstraint child = iter.next();
+        child = child.accept(this);
+        iter.set(child);
+      }
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(And constraint) {
+      return visitAndOr(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(Or constraint) {
+      return visitAndOr(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(DelayedOr constraint) {
+      constraint.getChildren().forEach(
+          child -> child.setConstraint(child.getConstraint().accept(this)));
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(TimedPlacementConstraint constraint) {
+      // Do nothing.
+      return null;
+    }
+  }
+
+  /**
+   * Visits a {@link PlacementConstraint} tree and substitutes each
+   * {@link TargetConstraint} and {@link CardinalityConstraint} with an
+   * equivalent {@link SingleConstraint}.
+   */
+  public static class SingleConstraintTransformer extends AbstractTransformer {
+
+    public SingleConstraintTransformer(PlacementConstraint constraint) {
+      super(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetConstraint constraint) {
+      AbstractConstraint newConstraint;
+      if (constraint.getOp() == TargetOperator.IN) {
+        newConstraint = new SingleConstraint(constraint.getScope(), 1,
+            Integer.MAX_VALUE, constraint.getTargetExpressions());
+      } else if (constraint.getOp() == TargetOperator.NOT_IN) {
+        newConstraint = new SingleConstraint(constraint.getScope(), 0, 0,
+            constraint.getTargetExpressions());
+      } else {
+        throw new YarnRuntimeException(
+            "Encountered unexpected type of constraint target operator: "
+                + constraint.getOp());
+      }
+      return newConstraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(CardinalityConstraint constraint) {
+      return new SingleConstraint(constraint.getScope(),
+          constraint.getMinCardinality(), constraint.getMaxCardinality(),
+          new TargetExpression(TargetExpression.TargetType.SELF));
+    }
+  }
+
+  /**
+   * Visits a {@link PlacementConstraint} tree and, whenever possible,
+   * substitutes each {@link SingleConstraint} with a {@link TargetConstraint}
+   * or a {@link CardinalityConstraint}. When such a substitution is not
+   * possible, we keep the original {@link SingleConstraint}.
+   */
+  public static class SpecializedConstraintTransformer
+      extends AbstractTransformer {
+
+    public SpecializedConstraintTransformer(PlacementConstraint constraint) {
+      super(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(SingleConstraint constraint) {
+      AbstractConstraint transformedConstraint = constraint;
+      // Check if it is a cardinality constraint.
+      if (constraint.getTargetExpressions().size() == 1) {
+        TargetExpression targetExpr =
+            constraint.getTargetExpressions().iterator().next();
+        if (targetExpr.getTargetType() == TargetExpression.TargetType.SELF) {
+          transformedConstraint = new CardinalityConstraint(
+              constraint.getScope(), constraint.getMinCardinality(),
+              constraint.getMaxCardinality());
+        }
+      }
+      // Check if it is a target constraint.
+      if (constraint.getMinCardinality() == 1
+          && constraint.getMaxCardinality() == Integer.MAX_VALUE) {
+        transformedConstraint = new TargetConstraint(TargetOperator.IN,
+            constraint.getScope(), constraint.getTargetExpressions());
+      } else if (constraint.getMinCardinality() == 0
+          && constraint.getMaxCardinality() == 0) {
+        transformedConstraint = new TargetConstraint(TargetOperator.NOT_IN,
+            constraint.getScope(), constraint.getTargetExpressions());
+      }
+
+      return transformedConstraint;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
new file mode 100644
index 0000000..660dc02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * API related to resources.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.resource;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
new file mode 100644
index 0000000..bd245e2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
@@ -0,0 +1,195 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link PlacementConstraintToProtoConverter} and
+ * {@link PlacementConstraintFromProtoConverter}.
+ */
+public class TestPlacementConstraintPBConversion {
+
+  @Test
+  public void testTargetConstraintProtoConverter() {
+    AbstractConstraint sConstraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(sConstraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    Assert.assertTrue(protoConstraint.hasSimpleConstraint());
+    Assert.assertFalse(protoConstraint.hasCompositeConstraint());
+    SimplePlacementConstraintProto sProto =
+        protoConstraint.getSimpleConstraint();
+    Assert.assertEquals(single.getScope(), sProto.getScope());
+    Assert.assertEquals(single.getMinCardinality(), sProto.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(), sProto.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions().size(),
+        sProto.getTargetExpressionsList().size());
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof SingleConstraint);
+    SingleConstraint newSingle = (SingleConstraint) newConstraintExpr;
+    Assert.assertEquals(single.getScope(), newSingle.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        newSingle.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        newSingle.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        newSingle.getTargetExpressions());
+  }
+
+  @Test
+  public void testCardinalityConstraintProtoConverter() {
+    AbstractConstraint sConstraintExpr = cardinality(RACK, 3, 10);
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(sConstraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    compareSimpleConstraintToProto(single, protoConstraint);
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof SingleConstraint);
+    SingleConstraint newSingle = (SingleConstraint) newConstraintExpr;
+    compareSimpleConstraints(single, newSingle);
+  }
+
+  @Test
+  public void testCompositeConstraintProtoConverter() {
+    AbstractConstraint constraintExpr =
+        or(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+    Assert.assertTrue(constraintExpr instanceof Or);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Or orExpr = (Or) constraintExpr;
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(constraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    Assert.assertFalse(protoConstraint.hasSimpleConstraint());
+    Assert.assertTrue(protoConstraint.hasCompositeConstraint());
+    CompositePlacementConstraintProto cProto =
+        protoConstraint.getCompositeConstraint();
+
+    Assert.assertEquals(CompositeType.OR, cProto.getCompositeType());
+    Assert.assertEquals(3, cProto.getChildConstraintsCount());
+    Assert.assertEquals(0, cProto.getTimedChildConstraintsCount());
+    Iterator<AbstractConstraint> orChildren = orExpr.getChildren().iterator();
+    Iterator<PlacementConstraintProto> orProtoChildren =
+        cProto.getChildConstraintsList().iterator();
+    while (orChildren.hasNext() && orProtoChildren.hasNext()) {
+      AbstractConstraint orChild = orChildren.next();
+      PlacementConstraintProto orProtoChild = orProtoChildren.next();
+      compareSimpleConstraintToProto((SingleConstraint) orChild, orProtoChild);
+    }
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof Or);
+    Or newOrExpr = (Or) newConstraintExpr;
+    Assert.assertEquals(3, newOrExpr.getChildren().size());
+    orChildren = orExpr.getChildren().iterator();
+    Iterator<AbstractConstraint> newOrChildren =
+        newOrExpr.getChildren().iterator();
+    while (orChildren.hasNext() && newOrChildren.hasNext()) {
+      AbstractConstraint orChild = orChildren.next();
+      AbstractConstraint newOrChild = newOrChildren.next();
+      compareSimpleConstraints((SingleConstraint) orChild,
+          (SingleConstraint) newOrChild);
+    }
+  }
+
+  private void compareSimpleConstraintToProto(SingleConstraint constraint,
+      PlacementConstraintProto proto) {
+    Assert.assertTrue(proto.hasSimpleConstraint());
+    Assert.assertFalse(proto.hasCompositeConstraint());
+    SimplePlacementConstraintProto sProto = proto.getSimpleConstraint();
+    Assert.assertEquals(constraint.getScope(), sProto.getScope());
+    Assert.assertEquals(constraint.getMinCardinality(),
+        sProto.getMinCardinality());
+    Assert.assertEquals(constraint.getMaxCardinality(),
+        sProto.getMaxCardinality());
+    Assert.assertEquals(constraint.getTargetExpressions().size(),
+        sProto.getTargetExpressionsList().size());
+  }
+
+  private void compareSimpleConstraints(SingleConstraint single,
+      SingleConstraint newSingle) {
+    Assert.assertEquals(single.getScope(), newSingle.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        newSingle.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        newSingle.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        newSingle.getTargetExpressions());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45b1ca60/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
new file mode 100644
index 0000000..1763735
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
@@ -0,0 +1,183 @@
+/**
+ * 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.resource;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint.TargetOperator;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SingleConstraintTransformer;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SpecializedConstraintTransformer;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link PlacementConstraintTransformations}.
+ */
+public class TestPlacementConstraintTransformations {
+
+  @Test
+  public void testTargetConstraint() {
+    AbstractConstraint sConstraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Transform from SimpleConstraint to specialized TargetConstraint
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(sConstraint);
+    PlacementConstraint tConstraint = specTransformer.transform();
+
+    AbstractConstraint tConstraintExpr = tConstraint.getConstraintExpr();
+    Assert.assertTrue(tConstraintExpr instanceof TargetConstraint);
+
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    TargetConstraint target = (TargetConstraint) tConstraintExpr;
+    Assert.assertEquals(single.getScope(), target.getScope());
+    Assert.assertEquals(TargetOperator.IN, target.getOp());
+    Assert.assertEquals(single.getTargetExpressions(),
+        target.getTargetExpressions());
+
+    // Transform from specialized TargetConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(tConstraint);
+    sConstraint = singleTransformer.transform();
+
+    sConstraintExpr = sConstraint.getConstraintExpr();
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+
+    single = (SingleConstraint) sConstraintExpr;
+    Assert.assertEquals(target.getScope(), single.getScope());
+    Assert.assertEquals(1, single.getMinCardinality());
+    Assert.assertEquals(Integer.MAX_VALUE, single.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        target.getTargetExpressions());
+  }
+
+  @Test
+  public void testCardinalityConstraint() {
+    AbstractConstraint sConstraintExpr = cardinality(RACK, 3, 10);
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Transform from SimpleConstraint to specialized CardinalityConstraint
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(sConstraint);
+    PlacementConstraint cConstraint = specTransformer.transform();
+
+    AbstractConstraint cConstraintExpr = cConstraint.getConstraintExpr();
+    Assert.assertTrue(cConstraintExpr instanceof CardinalityConstraint);
+
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    CardinalityConstraint cardinality = (CardinalityConstraint) cConstraintExpr;
+    Assert.assertEquals(single.getScope(), cardinality.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        cardinality.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        cardinality.getMaxCardinality());
+
+    // Transform from specialized CardinalityConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(cConstraint);
+    sConstraint = singleTransformer.transform();
+
+    sConstraintExpr = sConstraint.getConstraintExpr();
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+
+    single = (SingleConstraint) sConstraintExpr;
+    Assert.assertEquals(cardinality.getScope(), single.getScope());
+    Assert.assertEquals(cardinality.getMinCardinality(),
+        single.getMinCardinality());
+    Assert.assertEquals(cardinality.getMaxCardinality(),
+        single.getMaxCardinality());
+    Assert.assertEquals(new HashSet<>(Arrays.asList(PlacementTargets.self())),
+        single.getTargetExpressions());
+  }
+
+  @Test
+  public void testTargetCardinalityConstraint() {
+    AbstractConstraint constraintExpr =
+        targetCardinality(RACK, 3, 10, allocationTag("zk"));
+    Assert.assertTrue(constraintExpr instanceof SingleConstraint);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+
+    // Apply transformation. Should be a no-op.
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(constraint);
+    PlacementConstraint newConstraint = specTransformer.transform();
+
+    // The constraint expression should be the same.
+    Assert.assertEquals(constraintExpr, newConstraint.getConstraintExpr());
+  }
+
+  @Test
+  public void testCompositeConstraint() {
+    AbstractConstraint constraintExpr =
+        or(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+    Assert.assertTrue(constraintExpr instanceof Or);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Or orExpr = (Or) constraintExpr;
+    for (AbstractConstraint child : orExpr.getChildren()) {
+      Assert.assertTrue(child instanceof SingleConstraint);
+    }
+
+    // Apply transformation. Should transform target and cardinality constraints
+    // included in the composite constraint to specialized ones.
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(constraint);
+    PlacementConstraint specConstraint = specTransformer.transform();
+
+    Or specOrExpr = (Or) specConstraint.getConstraintExpr();
+    List<AbstractConstraint> specChildren = specOrExpr.getChildren();
+    Assert.assertEquals(3, specChildren.size());
+    Assert.assertTrue(specChildren.get(0) instanceof TargetConstraint);
+    Assert.assertTrue(specChildren.get(1) instanceof CardinalityConstraint);
+    Assert.assertTrue(specChildren.get(2) instanceof SingleConstraint);
+
+    // Transform from specialized TargetConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(specConstraint);
+    PlacementConstraint simConstraint = singleTransformer.transform();
+    Assert.assertTrue(constraintExpr instanceof Or);
+    Or simOrExpr = (Or) specConstraint.getConstraintExpr();
+    for (AbstractConstraint child : simOrExpr.getChildren()) {
+      Assert.assertTrue(child instanceof SingleConstraint);
+    }
+  }
+
+}


---------------------------------------------------------------------
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: HDFS-12789. [READ] Image generation tool does not close an opened stream

Posted by kk...@apache.org.
HDFS-12789. [READ] Image generation tool does not close an opened stream


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

Branch: refs/heads/YARN-6592
Commit: 87dc026beec5d69a84771631ebca5fadb2f7195b
Parents: c293cc8
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Nov 8 10:28:50 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:39 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/ImageWriter.java   | 17 ++++++++++++-----
 1 file changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/87dc026b/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 ea1888a..390bb39 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
@@ -165,16 +165,23 @@ public class ImageWriter implements Closeable {
 
     // create directory and inode sections as side-files.
     // The details are written to files to avoid keeping them in memory.
-    dirsTmp = File.createTempFile("fsimg_dir", null);
-    dirsTmp.deleteOnExit();
-    dirs = beginSection(new FileOutputStream(dirsTmp));
+    FileOutputStream dirsTmpStream = null;
+    try {
+      dirsTmp = File.createTempFile("fsimg_dir", null);
+      dirsTmp.deleteOnExit();
+      dirsTmpStream = new FileOutputStream(dirsTmp);
+      dirs = beginSection(dirsTmpStream);
+    } catch (IOException e) {
+      IOUtils.cleanupWithLogger(null, raw, dirsTmpStream);
+      throw e;
+    }
+
     try {
       inodesTmp = File.createTempFile("fsimg_inode", null);
       inodesTmp.deleteOnExit();
       inodes = new FileOutputStream(inodesTmp);
     } catch (IOException e) {
-      // appropriate to close raw?
-      IOUtils.cleanup(null, raw, dirs);
+      IOUtils.cleanupWithLogger(null, raw, dirsTmpStream, dirs);
       throw e;
     }
 


---------------------------------------------------------------------
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-13974. S3Guard CLI to support list/purge of pending multipart commits. Contributed by Aaron Fabbri

Posted by kk...@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-6592
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


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

Posted by kk...@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-6592
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


[04/50] [abbrv] hadoop git commit: HDFS-12584. [READ] Fix errors in image generation tool from latest rebase

Posted by kk...@apache.org.
HDFS-12584. [READ] Fix errors in image generation tool from latest rebase


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

Branch: refs/heads/YARN-6592
Commit: 17052c4aff104cb02701bc1e8dc9cd73d1a325fb
Parents: aca023b
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Oct 3 14:44:17 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:38 2017 -0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-fs2img/pom.xml              |  4 +--
 .../hdfs/server/namenode/RandomTreeWalk.java    | 28 +++++++++-----------
 2 files changed, 14 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/17052c4a/hadoop-tools/hadoop-fs2img/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/pom.xml b/hadoop-tools/hadoop-fs2img/pom.xml
index 36096b7..e1411f8 100644
--- a/hadoop-tools/hadoop-fs2img/pom.xml
+++ b/hadoop-tools/hadoop-fs2img/pom.xml
@@ -17,12 +17,12 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-alpha3-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-fs2img</artifactId>
-  <version>3.0.0-alpha3-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>fs2img</description>
   <name>fs2img</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17052c4a/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 c82c489..d002e4a 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
@@ -113,22 +113,18 @@ public class RandomTreeWalk extends TreeWalk {
     final long len = isDir ? 0 : r.nextInt(Integer.MAX_VALUE);
     final int nblocks = 0 == len ? 0 : (((int)((len - 1) / blocksize)) + 1);
     BlockLocation[] blocks = genBlocks(r, nblocks, blocksize, len);
-    try {
-      return new LocatedFileStatus(new FileStatus(
-          len,              /* long length,             */
-          isDir,            /* boolean isdir,           */
-          1,                /* int block_replication,   */
-          blocksize,        /* long blocksize,          */
-          0L,               /* long modification_time,  */
-          0L,               /* long access_time,        */
-          null,             /* FsPermission permission, */
-          "hadoop",         /* String owner,            */
-          "hadoop",         /* String group,            */
-          name),            /* Path path                */
-          blocks);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    return new LocatedFileStatus(new FileStatus(
+        len,              /* long length,             */
+        isDir,            /* boolean isdir,           */
+        1,                /* int block_replication,   */
+        blocksize,        /* long blocksize,          */
+        0L,               /* long modification_time,  */
+        0L,               /* long access_time,        */
+        null,             /* FsPermission permission, */
+        "hadoop",         /* String owner,            */
+        "hadoop",         /* String group,            */
+        name),            /* Path path                */
+        blocks);
   }
 
   BlockLocation[] genBlocks(Random r, int nblocks, int blocksize, long len) {


---------------------------------------------------------------------
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: HDFS-12685. [READ] FsVolumeImpl exception when scanning Provided storage volume

Posted by kk...@apache.org.
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-6592
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