Browse Source

HDFS-11902. [READ] Merge BlockFormatProvider and FileRegionProvider.

Virajith Jalaparti 7 years ago
parent
commit
98f5ed5aa3
18 changed files with 240 additions and 489 deletions
  1. 7 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  2. 0 91
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
  3. 0 75
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
  4. 46 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
  5. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
  6. 13 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
  7. 23 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
  8. 8 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
  9. 49 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
  10. 7 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  11. 5 36
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
  12. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
  13. 47 28
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
  14. 2 2
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
  15. 12 13
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
  16. 3 4
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java
  17. 4 4
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
  18. 8 17
      hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java

+ 7 - 10
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 String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
   public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;
   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_DF_CLASS = "dfs.provided.df.class";
   public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";
   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_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 String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;

+ 0 - 91
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java

@@ -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);
-    }
-  }
-
-}

+ 0 - 75
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java

@@ -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;
-  }
-}

+ 46 - 17
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.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.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.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;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -61,7 +64,11 @@ public class ProvidedStorageMap {
       LoggerFactory.getLogger(ProvidedStorageMap.class);
       LoggerFactory.getLogger(ProvidedStorageMap.class);
 
 
   // limit to a single provider for now
   // 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 String storageId;
   private final ProvidedDescriptor providedDescriptor;
   private final ProvidedDescriptor providedDescriptor;
   private final DatanodeStorageInfo providedStorageInfo;
   private final DatanodeStorageInfo providedStorageInfo;
@@ -79,7 +86,7 @@ public class ProvidedStorageMap {
 
 
     if (!providedEnabled) {
     if (!providedEnabled) {
       // disable mapping
       // disable mapping
-      blockProvider = null;
+      aliasMap = null;
       providedDescriptor = null;
       providedDescriptor = null;
       providedStorageInfo = null;
       providedStorageInfo = null;
       return;
       return;
@@ -90,15 +97,17 @@ public class ProvidedStorageMap {
     providedDescriptor = new ProvidedDescriptor();
     providedDescriptor = new ProvidedDescriptor();
     providedStorageInfo = providedDescriptor.createProvidedStorage(ds);
     providedStorageInfo = providedDescriptor.createProvidedStorage(ds);
 
 
+    this.bm = bm;
+    this.lock = lock;
+
     // load block reader into storage
     // 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 {
       BlockReportContext context) throws IOException {
     if (providedEnabled && storageId.equals(s.getStorageID())) {
     if (providedEnabled && storageId.equals(s.getStorageID())) {
       if (StorageType.PROVIDED.equals(s.getStorageType())) {
       if (StorageType.PROVIDED.equals(s.getStorageType())) {
-        // poll service, initiate
-        blockProvider.start(context);
+        processProvidedStorageReport(context);
         dn.injectStorage(providedStorageInfo);
         dn.injectStorage(providedStorageInfo);
         return providedDescriptor.getProvidedStorage(dn, s);
         return providedDescriptor.getProvidedStorage(dn, s);
       }
       }
@@ -124,6 +132,26 @@ public class ProvidedStorageMap {
     return dn.getStorageInfo(s.getStorageID());
     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
   @VisibleForTesting
   public DatanodeStorageInfo getProvidedStorageInfo() {
   public DatanodeStorageInfo getProvidedStorageInfo() {
     return providedStorageInfo;
     return providedStorageInfo;
@@ -137,10 +165,11 @@ public class ProvidedStorageMap {
   }
   }
 
 
   public void removeDatanode(DatanodeDescriptor dnToRemove) {
   public void removeDatanode(DatanodeDescriptor dnToRemove) {
-    if (providedDescriptor != null) {
+    if (providedEnabled) {
+      assert lock.hasWriteLock() : "Not holding write lock";
       int remainingDatanodes = providedDescriptor.remove(dnToRemove);
       int remainingDatanodes = providedDescriptor.remove(dnToRemove);
       if (remainingDatanodes == 0) {
       if (remainingDatanodes == 0) {
-        blockProvider.stop();
+        hasDNs = false;
       }
       }
     }
     }
   }
   }
@@ -443,9 +472,9 @@ public class ProvidedStorageMap {
    */
    */
   static class ProvidedBlockList extends BlockListAsLongs {
   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;
       this.inner = inner;
     }
     }
 
 
@@ -454,7 +483,7 @@ public class ProvidedStorageMap {
       return new Iterator<BlockReportReplica>() {
       return new Iterator<BlockReportReplica>() {
         @Override
         @Override
         public BlockReportReplica next() {
         public BlockReportReplica next() {
-          return new BlockReportReplica(inner.next());
+          return new BlockReportReplica(inner.next().getBlock());
         }
         }
         @Override
         @Override
         public boolean hasNext() {
         public boolean hasNext() {

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java

@@ -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();
-  }
-}

+ 13 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java

@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.common;
+package org.apache.hadoop.hdfs.server.common.blockaliasmap;
 
 
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 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.
  * An abstract class used to read and write block maps for provided blocks.
  */
  */
-public abstract class BlockFormat<T extends BlockAlias>  {
+public abstract class BlockAliasMap<T extends BlockAlias> {
 
 
   /**
   /**
    * An abstract class that is used to read {@link BlockAlias}es
    * An abstract class that is used to read {@link BlockAlias}es
@@ -39,14 +40,19 @@ public abstract class BlockFormat<T extends BlockAlias>  {
      */
      */
     public interface 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;
     public abstract U resolve(Block ident) throws IOException;
 
 
   }
   }
 
 
   /**
   /**
-   * Returns the reader for the provided block map.
+   * Returns a reader to the alias map.
    * @param opts reader options
    * @param opts reader options
-   * @return {@link Reader} to the block map.
+   * @return {@link Reader} to the alias map.
    * @throws IOException
    * @throws IOException
    */
    */
   public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
   public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
@@ -66,15 +72,15 @@ public abstract class BlockFormat<T extends BlockAlias>  {
   }
   }
 
 
   /**
   /**
-   * Returns the writer for the provided block map.
+   * Returns the writer for the alias map.
    * @param opts writer options.
    * @param opts writer options.
-   * @return {@link Writer} to the block map.
+   * @return {@link Writer} to the alias map.
    * @throws IOException
    * @throws IOException
    */
    */
   public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
   public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
 
 
   /**
   /**
-   * Refresh based on the underlying block map.
+   * Refresh the alias map.
    * @throws IOException
    * @throws IOException
    */
    */
   public abstract void refresh() throws IOException;
   public abstract void refresh() throws IOException;

+ 23 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
-package org.apache.hadoop.hdfs.server.common;
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
@@ -40,6 +40,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.MultipleIOException;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
@@ -52,15 +54,15 @@ import com.google.common.annotations.VisibleForTesting;
  * This class is used for block maps stored as text files,
  * This class is used for block maps stored as text files,
  * with a specified delimiter.
  * with a specified delimiter.
  */
  */
-public class TextFileRegionFormat
-    extends BlockFormat<FileRegion> implements Configurable {
+public class TextFileRegionAliasMap
+    extends BlockAliasMap<FileRegion> implements Configurable {
 
 
   private Configuration conf;
   private Configuration conf;
   private ReaderOptions readerOpts = TextReader.defaults();
   private ReaderOptions readerOpts = TextReader.defaults();
   private WriterOptions writerOpts = TextWriter.defaults();
   private WriterOptions writerOpts = TextWriter.defaults();
 
 
   public static final Logger LOG =
   public static final Logger LOG =
-      LoggerFactory.getLogger(TextFileRegionFormat.class);
+      LoggerFactory.getLogger(TextFileRegionAliasMap.class);
   @Override
   @Override
   public void setConf(Configuration conf) {
   public void setConf(Configuration conf) {
     readerOpts.setConf(conf);
     readerOpts.setConf(conf);
@@ -137,27 +139,28 @@ public class TextFileRegionFormat
   }
   }
 
 
   /**
   /**
-   * Class specifying reader options for the {@link TextFileRegionFormat}.
+   * Class specifying reader options for the {@link TextFileRegionAliasMap}.
    */
    */
   public static class ReaderOptions
   public static class ReaderOptions
       implements TextReader.Options, Configurable {
       implements TextReader.Options, Configurable {
 
 
     private Configuration conf;
     private Configuration conf;
     private String delim =
     private String delim =
-        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
     private Path file = new Path(
     private Path file = new Path(
-        new File(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT)
-        .toURI().toString());
+        new File(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT).toURI()
+            .toString());
 
 
     @Override
     @Override
     public void setConf(Configuration conf) {
     public void setConf(Configuration conf) {
       this.conf = conf;
       this.conf = conf;
-      String tmpfile = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
+      String tmpfile =
+          conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
       file = new Path(tmpfile);
       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());
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+      LOG.info("TextFileRegionAliasMap: read path " + tmpfile.toString());
     }
     }
 
 
     @Override
     @Override
@@ -179,7 +182,7 @@ public class TextFileRegionFormat
   }
   }
 
 
   /**
   /**
-   * Class specifying writer options for the {@link TextFileRegionFormat}.
+   * Class specifying writer options for the {@link TextFileRegionAliasMap}.
    */
    */
   public static class WriterOptions
   public static class WriterOptions
       implements TextWriter.Options, Configurable {
       implements TextWriter.Options, Configurable {
@@ -187,19 +190,19 @@ public class TextFileRegionFormat
     private Configuration conf;
     private Configuration conf;
     private String codec = null;
     private String codec = null;
     private Path file =
     private Path file =
-        new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);;
     private String delim =
     private String delim =
-        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
 
 
     @Override
     @Override
     public void setConf(Configuration conf) {
     public void setConf(Configuration conf) {
       this.conf = conf;
       this.conf = conf;
       String tmpfile = conf.get(
       String tmpfile = conf.get(
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_WRITE_PATH, file.toString());
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH, file.toString());
       file = new Path(tmpfile);
       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);
+      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
     @Override

+ 8 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java

@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * distributed with this work for additional information
@@ -15,23 +15,13 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-
-package org.apache.hadoop.hdfs.server.common;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.common.blockaliasmap;
 
 
 /**
 /**
- * This class is a stub for reading file regions from the block map.
+ * The AliasMap defines mapping of PROVIDED HDFS blocks to data in remote
+ * storage systems.
  */
  */
-public class FileRegionProvider implements Iterable<FileRegion> {
-  @Override
-  public Iterator<FileRegion> iterator() {
-    return Collections.emptyListIterator();
-  }
-
-  public void refresh() throws IOException {
-    return;
-  }
-}
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 49 - 27
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.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 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.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.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
@@ -68,7 +68,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   static class ProvidedBlockPoolSlice {
   static class ProvidedBlockPoolSlice {
     private ProvidedVolumeImpl providedVolume;
     private ProvidedVolumeImpl providedVolume;
 
 
-    private FileRegionProvider provider;
+    private BlockAliasMap<FileRegion> aliasMap;
     private Configuration conf;
     private Configuration conf;
     private String bpid;
     private String bpid;
     private ReplicaMap bpVolumeMap;
     private ReplicaMap bpVolumeMap;
@@ -77,29 +77,35 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
         Configuration conf) {
         Configuration conf) {
       this.providedVolume = volume;
       this.providedVolume = volume;
       bpVolumeMap = new ReplicaMap(new AutoCloseableLock());
       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.conf = conf;
       this.bpid = bpid;
       this.bpid = bpid;
       bpVolumeMap.initBlockPool(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
     @VisibleForTesting
-    void setFileRegionProvider(FileRegionProvider newProvider) {
-      this.provider = newProvider;
+    void setFileRegionProvider(BlockAliasMap<FileRegion> blockAliasMap) {
+      this.aliasMap = blockAliasMap;
     }
     }
 
 
     public void getVolumeMap(ReplicaMap volumeMap,
     public void getVolumeMap(ReplicaMap volumeMap,
         RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
         RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
         throws IOException {
         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()) {
       while (iter.hasNext()) {
         FileRegion region = iter.next();
         FileRegion region = iter.next();
         if (region.getBlockPoolId() != null
         if (region.getBlockPoolId() != null
@@ -140,14 +146,20 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
     public void compileReport(LinkedList<ScanInfo> report,
     public void compileReport(LinkedList<ScanInfo> report,
         ReportCompiler reportCompiler)
         ReportCompiler reportCompiler)
             throws IOException, InterruptedException {
             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
        * the assumption here is that the block ids in the external
        * block map, after the refresh, are consistent with those
        * block map, after the refresh, are consistent with those
        * from before the refresh, i.e., for blocks which did not change,
        * from before the refresh, i.e., for blocks which did not change,
        * the ids remain the same.
        * 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()) {
       while(iter.hasNext()) {
         reportCompiler.throttle();
         reportCompiler.throttle();
         FileRegion region = iter.next();
         FileRegion region = iter.next();
@@ -284,15 +296,15 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
 
     private String bpid;
     private String bpid;
     private String name;
     private String name;
-    private FileRegionProvider provider;
+    private BlockAliasMap<FileRegion> blockAliasMap;
     private Iterator<FileRegion> blockIterator;
     private Iterator<FileRegion> blockIterator;
     private ProvidedBlockIteratorState state;
     private ProvidedBlockIteratorState state;
 
 
     ProviderBlockIteratorImpl(String bpid, String name,
     ProviderBlockIteratorImpl(String bpid, String name,
-        FileRegionProvider provider) {
+        BlockAliasMap<FileRegion> blockAliasMap) {
       this.bpid = bpid;
       this.bpid = bpid;
       this.name = name;
       this.name = name;
-      this.provider = provider;
+      this.blockAliasMap = blockAliasMap;
       rewind();
       rewind();
     }
     }
 
 
@@ -330,7 +342,17 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
 
     @Override
     @Override
     public void rewind() {
     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();
       state = new ProvidedBlockIteratorState();
     }
     }
 
 
@@ -372,14 +394,14 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   @Override
   @Override
   public BlockIterator newBlockIterator(String bpid, String name) {
   public BlockIterator newBlockIterator(String bpid, String name) {
     return new ProviderBlockIteratorImpl(bpid, name,
     return new ProviderBlockIteratorImpl(bpid, name,
-        bpSlices.get(bpid).getFileRegionProvider());
+        bpSlices.get(bpid).getBlockAliasMap());
   }
   }
 
 
   @Override
   @Override
   public BlockIterator loadBlockIterator(String bpid, String name)
   public BlockIterator loadBlockIterator(String bpid, String name)
       throws IOException {
       throws IOException {
     ProviderBlockIteratorImpl iter = new ProviderBlockIteratorImpl(bpid, name,
     ProviderBlockIteratorImpl iter = new ProviderBlockIteratorImpl(bpid, name,
-        bpSlices.get(bpid).getFileRegionProvider());
+        bpSlices.get(bpid).getBlockAliasMap());
     iter.load();
     iter.load();
     return iter;
     return iter;
   }
   }
@@ -425,8 +447,8 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  FileRegionProvider getFileRegionProvider(String bpid) throws IOException {
-    return getProvidedBlockPoolSlice(bpid).getFileRegionProvider();
+  BlockAliasMap<FileRegion> getBlockFormat(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).getBlockAliasMap();
   }
   }
 
 
   @Override
   @Override
@@ -571,12 +593,12 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  void setFileRegionProvider(String bpid, FileRegionProvider provider)
-      throws IOException {
+  void setFileRegionProvider(String bpid,
+      BlockAliasMap<FileRegion> blockAliasMap) throws IOException {
     ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
     ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
     if (bp == null) {
     if (bp == null) {
       throw new IOException("block pool " + bpid + " is not found");
       throw new IOException("block pool " + bpid + " is not found");
     }
     }
-    bp.setFileRegionProvider(provider);
+    bp.setFileRegionProvider(blockAliasMap);
   }
   }
 }
 }

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

@@ -4629,26 +4629,6 @@
     </description>
     </description>
   </property>
   </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>
   <property>
     <name>dfs.provided.df.class</name>
     <name>dfs.provided.df.class</name>
     <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
     <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
@@ -4666,12 +4646,12 @@
   </property>
   </property>
 
 
   <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>
     <description>
       The class that is used to specify the input format of the blocks on
       The class that is used to specify the input format of the blocks on
       provided storages. The default is
       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
       file regions to describe blocks. The file regions are specified as a
       delimited text file. Each file region is a 6-tuple containing the
       delimited text file. Each file region is a 6-tuple containing the
       block id, remote file path, offset into file, length of block, the
       block id, remote file path, offset into file, length of block, the
@@ -4681,7 +4661,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <name>dfs.provided.textprovider.delimiter</name>
+    <name>dfs.provided.aliasmap.text.delimiter</name>
     <value>,</value>
     <value>,</value>
     <description>
     <description>
         The delimiter used when the provided block map is specified as
         The delimiter used when the provided block map is specified as
@@ -4690,7 +4670,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <name>dfs.provided.textprovider.read.path</name>
+    <name>dfs.provided.aliasmap.text.read.path</name>
     <value></value>
     <value></value>
     <description>
     <description>
         The path specifying the provided block map as a text file, specified as
         The path specifying the provided block map as a text file, specified as
@@ -4699,7 +4679,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <name>dfs.provided.textprovider.read.codec</name>
+    <name>dfs.provided.aliasmap.text.codec</name>
     <value></value>
     <value></value>
     <description>
     <description>
         The codec used to de-compress the provided block map.
         The codec used to de-compress the provided block map.
@@ -4707,7 +4687,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <name>dfs.provided.textprovider.write.path</name>
+    <name>dfs.provided.aliasmap.text.write.path</name>
     <value></value>
     <value></value>
     <description>
     <description>
         The path to which the provided block map should be written as a text
         The path to which the provided block map should be written as a text

+ 5 - 36
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;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 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.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.Iterator;
 
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
@@ -47,37 +46,6 @@ public class TestProvidedStorageMap {
   private RwLock nameSystemLock;
   private RwLock nameSystemLock;
   private String providedStorageID;
   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
   @Before
   public void setup() {
   public void setup() {
     providedStorageID = DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT;
     providedStorageID = DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT;
@@ -85,8 +53,9 @@ public class TestProvidedStorageMap {
     conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
     conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
             providedStorageID);
             providedStorageID);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
     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);
     bm = mock(BlockManager.class);
     nameSystemLock = mock(RwLock.class);
     nameSystemLock = mock(RwLock.class);

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.common;
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
@@ -25,7 +25,8 @@ import java.util.Iterator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat.*;
+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.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -36,13 +37,13 @@ import static org.junit.Assert.*;
 /**
 /**
  * Test for the text based block format for provided block maps.
  * Test for the text based block format for provided block maps.
  */
  */
-public class TestTextBlockFormat {
+public class TestTextBlockAliasMap {
 
 
   static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
   static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
 
 
   void check(TextWriter.Options opts, final Path vp,
   void check(TextWriter.Options opts, final Path vp,
       final Class<? extends CompressionCodec> vc) throws IOException {
       final Class<? extends CompressionCodec> vc) throws IOException {
-    TextFileRegionFormat mFmt = new TextFileRegionFormat() {
+    TextFileRegionAliasMap mFmt = new TextFileRegionAliasMap() {
       @Override
       @Override
       public TextWriter createWriter(Path file, CompressionCodec codec,
       public TextWriter createWriter(Path file, CompressionCodec codec,
           String delim, Configuration conf) throws IOException {
           String delim, Configuration conf) throws IOException {
@@ -63,7 +64,7 @@ public class TestTextBlockFormat {
     TextWriter.Options opts = TextWriter.defaults();
     TextWriter.Options opts = TextWriter.defaults();
     assertTrue(opts instanceof WriterOptions);
     assertTrue(opts instanceof WriterOptions);
     WriterOptions wopts = (WriterOptions) opts;
     WriterOptions wopts = (WriterOptions) opts;
-    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
+    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
     assertEquals(def, wopts.getFile());
     assertEquals(def, wopts.getFile());
     assertNull(wopts.getCodec());
     assertNull(wopts.getCodec());
 
 

+ 47 - 28
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.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 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.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 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.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.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 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 Configuration conf;
     private int minId;
     private int minId;
     private int numBlocks;
     private int numBlocks;
     private Iterator<FileRegion> suppliedIterator;
     private Iterator<FileRegion> suppliedIterator;
 
 
-    TestFileRegionProvider() {
+    TestFileRegionBlockAliasMap() {
       this(null, MIN_BLK_ID, NUM_PROVIDED_BLKS);
       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.suppliedIterator = iterator;
       this.minId = minId;
       this.minId = minId;
       this.numBlocks = numBlocks;
       this.numBlocks = numBlocks;
     }
     }
 
 
     @Override
     @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
     @Override
-    public Configuration getConf() {
-      return conf;
+    public Writer<FileRegion> getWriter(Writer.Options opts)
+        throws IOException {
+      // not implemented
+      return null;
     }
     }
 
 
     @Override
     @Override
-    public void refresh() {
-      //do nothing!
+    public void refresh() throws IOException {
+      // do nothing!
     }
     }
 
 
     public void setMinBlkId(int minId) {
     public void setMinBlkId(int minId) {
@@ -359,8 +377,8 @@ public class TestProvidedImpl {
         new ShortCircuitRegistry(conf);
         new ShortCircuitRegistry(conf);
     when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry);
     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,
     conf.setClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
         TestProvidedVolumeDF.class, ProvidedVolumeDF.class);
         TestProvidedVolumeDF.class, ProvidedVolumeDF.class);
 
 
@@ -496,12 +514,13 @@ public class TestProvidedImpl {
     conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
     for (int i = 0; i < providedVolumes.size(); i++) {
     for (int i = 0; i < providedVolumes.size(); i++) {
       ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(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
       //equivalent to two new blocks appearing
-      provider.setBlockCount(NUM_PROVIDED_BLKS + 2);
+      testBlockFormat.setBlockCount(NUM_PROVIDED_BLKS + 2);
       //equivalent to deleting the first block
       //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);
       DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf);
       scanner.reconcile();
       scanner.reconcile();
@@ -525,7 +544,7 @@ public class TestProvidedImpl {
     for (int i = 0; i < providedVolumes.size(); i++) {
     for (int i = 0; i < providedVolumes.size(); i++) {
       ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
       ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
       vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID],
       vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID],
-          new TestFileRegionProvider(fileRegionIterator, minBlockId,
+          new TestFileRegionBlockAliasMap(fileRegionIterator, minBlockId,
               numBlocks));
               numBlocks));
       ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
       ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
       vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null);
       vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null);

+ 2 - 2
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.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
@@ -103,7 +103,7 @@ public class FileSystemImage implements Tool {
         break;
         break;
       case "b":
       case "b":
         opts.blocks(
         opts.blocks(
-            Class.forName(o.getValue()).asSubclass(BlockFormat.class));
+            Class.forName(o.getValue()).asSubclass(BlockAliasMap.class));
         break;
         break;
       case "i":
       case "i":
         opts.blockIds(
         opts.blockIds(

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

+ 3 - 4
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockFormat.java → hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/NullBlockAliasMap.java

@@ -22,17 +22,16 @@ import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.NoSuchElementException;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 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;
 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.
  * Null sink for region information emitted from FSImage.
  */
  */
-public class NullBlockFormat extends BlockFormat<FileRegion> {
+public class NullBlockAliasMap extends BlockAliasMap<FileRegion> {
 
 
   @Override
   @Override
-  public Reader<FileRegion> getReader(Options opts) throws IOException {
+  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
     return new Reader<FileRegion>() {
     return new Reader<FileRegion>() {
       @Override
       @Override
       public Iterator<FileRegion> iterator() {
       public Iterator<FileRegion> iterator() {

+ 4 - 4
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.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 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.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.INode;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
@@ -70,7 +70,7 @@ public class TreePath {
   }
   }
 
 
   public INode toINode(UGIResolver ugi, BlockResolver blk,
   public INode toINode(UGIResolver ugi, BlockResolver blk,
-      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+                       BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
           throws IOException {
           throws IOException {
     if (stat.isFile()) {
     if (stat.isFile()) {
       return toFile(ugi, blk, out, blockPoolID);
       return toFile(ugi, blk, out, blockPoolID);
@@ -101,14 +101,14 @@ public class TreePath {
 
 
   void writeBlock(long blockId, long offset, long length,
   void writeBlock(long blockId, long offset, long length,
       long genStamp, String blockPoolID,
       long genStamp, String blockPoolID,
-      BlockFormat.Writer<FileRegion> out) throws IOException {
+      BlockAliasMap.Writer<FileRegion> out) throws IOException {
     FileStatus s = getFileStatus();
     FileStatus s = getFileStatus();
     out.store(new FileRegion(blockId, s.getPath(), offset, length,
     out.store(new FileRegion(blockId, s.getPath(), offset, length,
         blockPoolID, genStamp));
         blockPoolID, genStamp));
   }
   }
 
 
   INode toFile(UGIResolver ugi, BlockResolver blk,
   INode toFile(UGIResolver ugi, BlockResolver blk,
-      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+               BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
           throws IOException {
           throws IOException {
     final FileStatus s = getFileStatus();
     final FileStatus s = getFileStatus();
     // TODO should this store resolver's user/group?
     // TODO should this store resolver's user/group?

+ 8 - 17
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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 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 org.apache.hadoop.hdfs.server.datanode.DataNode;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
@@ -103,18 +99,13 @@ public class TestNameNodeProvidedImplementation {
         DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
         DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
     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());
         BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
         BLOCKFILE.toString());
         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,
     conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
         new File(NAMEPATH.toUri()).toString());
         new File(NAMEPATH.toUri()).toString());
@@ -167,7 +158,7 @@ public class TestNameNodeProvidedImplementation {
     ImageWriter.Options opts = ImageWriter.defaults();
     ImageWriter.Options opts = ImageWriter.defaults();
     opts.setConf(conf);
     opts.setConf(conf);
     opts.output(out.toString())
     opts.output(out.toString())
-        .blocks(TextFileRegionFormat.class)
+        .blocks(TextFileRegionAliasMap.class)
         .blockIds(blockIdsClass);
         .blockIds(blockIdsClass);
     try (ImageWriter w = new ImageWriter(opts)) {
     try (ImageWriter w = new ImageWriter(opts)) {
       for (TreePath e : t) {
       for (TreePath e : t) {