Selaa lähdekoodia

HDFS-14001. [PROVIDED Storage] bootstrapStandby should manage the InMemoryAliasMap. Contributed by Virajith Jalaparti.

Inigo Goiri 6 vuotta sitten
vanhempi
commit
8fc0d04517
14 muutettua tiedostoa jossa 669 lisäystä ja 76 poistoa
  1. 202 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
  2. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
  3. 29 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
  4. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  5. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  6. 20 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  7. 90 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  8. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  9. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  10. 30 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
  11. 0 45
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
  12. 0 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
  13. 99 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapAliasmap.java
  14. 136 3
      hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java

+ 202 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java

@@ -19,10 +19,14 @@ 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.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream;
 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.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
@@ -30,17 +34,28 @@ 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.apache.hadoop.hdfs.server.namenode.ImageServlet;
+import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
+import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.io.IOUtils;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBIterator;
 import org.iq80.leveldb.Options;
+import org.iq80.leveldb.ReadOptions;
+import org.iq80.leveldb.Snapshot;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nonnull;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Map;
 import java.util.Optional;
@@ -57,6 +72,9 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
   private static final Logger LOG = LoggerFactory
       .getLogger(InMemoryAliasMap.class);
 
+  private static final String SNAPSHOT_COPY_DIR = "aliasmap_snapshot";
+  private static final String TAR_NAME = "aliasmap.tar.gz";
+  private final URI aliasMapURI;
   private final DB levelDb;
   private Configuration conf;
   private String blockPoolID;
@@ -71,22 +89,15 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     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,
       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);
+    if (directory == null) {
+      throw new IOException("InMemoryAliasMap location is null");
+    }
     File levelDBpath;
     if (blockPoolID != null) {
       levelDBpath = new File(directory, blockPoolID);
@@ -94,17 +105,23 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
       levelDBpath = new File(directory);
     }
     if (!levelDBpath.exists()) {
-      String error = createPathErrorMessage(directory);
-      throw new IOException(error);
+      LOG.warn("InMemoryAliasMap location {} is missing. Creating it.",
+          levelDBpath);
+      if(!levelDBpath.mkdirs()) {
+        throw new IOException(
+            "Unable to create missing aliasmap location: " + levelDBpath);
+      }
     }
     DB levelDb = JniDBFactory.factory.open(levelDBpath, options);
-    InMemoryAliasMap aliasMap = new InMemoryAliasMap(levelDb, blockPoolID);
+    InMemoryAliasMap aliasMap =  new InMemoryAliasMap(levelDBpath.toURI(),
+        levelDb, blockPoolID);
     aliasMap.setConf(conf);
     return aliasMap;
   }
 
   @VisibleForTesting
-  InMemoryAliasMap(DB levelDb, String blockPoolID) {
+  InMemoryAliasMap(URI aliasMapURI, DB levelDb, String blockPoolID) {
+    this.aliasMapURI = aliasMapURI;
     this.levelDb = levelDb;
     this.blockPoolID = blockPoolID;
   }
@@ -208,6 +225,177 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     return blockOutputStream.toByteArray();
   }
 
+  /**
+   * Transfer this aliasmap for bootstrapping standby Namenodes. The map is
+   * transferred as a tar.gz archive. This archive needs to be extracted on the
+   * standby Namenode.
+   *
+   * @param response http response.
+   * @param conf configuration to use.
+   * @param aliasMap aliasmap to transfer.
+   * @throws IOException
+   */
+  public static void transferForBootstrap(HttpServletResponse response,
+      Configuration conf, InMemoryAliasMap aliasMap) throws IOException {
+    File aliasMapSnapshot = null;
+    File compressedAliasMap = null;
+    try {
+      aliasMapSnapshot = createSnapshot(aliasMap);
+      // compress the snapshot that is associated with the
+      // block pool id of the aliasmap.
+      compressedAliasMap = getCompressedAliasMap(
+          new File(aliasMapSnapshot, aliasMap.blockPoolID));
+      try (FileInputStream fis = new FileInputStream(compressedAliasMap)) {
+        ImageServlet.setVerificationHeadersForGet(response, compressedAliasMap);
+        ImageServlet.setFileNameHeaders(response, compressedAliasMap);
+        // send file
+        DataTransferThrottler throttler =
+            ImageServlet.getThrottlerForBootstrapStandby(conf);
+        TransferFsImage.copyFileToStream(response.getOutputStream(),
+            compressedAliasMap, fis, throttler);
+      }
+    } finally {
+      // cleanup the temporary snapshot and compressed files.
+      StringBuilder errMessage = new StringBuilder();
+      if (compressedAliasMap != null
+          && !FileUtil.fullyDelete(compressedAliasMap)) {
+        errMessage.append("Failed to fully delete compressed aliasmap ")
+            .append(compressedAliasMap.getAbsolutePath()).append("\n");
+      }
+      if (aliasMapSnapshot != null && !FileUtil.fullyDelete(aliasMapSnapshot)) {
+        errMessage.append("Failed to fully delete the aliasmap snapshot ")
+            .append(aliasMapSnapshot.getAbsolutePath()).append("\n");
+      }
+      if (errMessage.length() > 0) {
+        throw new IOException(errMessage.toString());
+      }
+    }
+  }
+
+  /**
+   * Create a new LevelDB store which is a snapshot copy of the original
+   * aliasmap.
+   *
+   * @param aliasMap original aliasmap.
+   * @return the {@link File} where the snapshot is created.
+   * @throws IOException
+   */
+  static File createSnapshot(InMemoryAliasMap aliasMap) throws IOException {
+    File originalAliasMapDir = new File(aliasMap.aliasMapURI);
+    String bpid = originalAliasMapDir.getName();
+    File snapshotDir =
+        new File(originalAliasMapDir.getParent(), SNAPSHOT_COPY_DIR);
+    File newLevelDBDir = new File(snapshotDir, bpid);
+    if (!newLevelDBDir.mkdirs()) {
+      throw new IOException(
+          "Unable to create aliasmap snapshot directory " + newLevelDBDir);
+    }
+    // get a snapshot for the original DB.
+    DB originalDB = aliasMap.levelDb;
+    try (Snapshot snapshot = originalDB.getSnapshot()) {
+      // create a new DB for the snapshot and copy all K,V pairs.
+      Options options = new Options();
+      options.createIfMissing(true);
+      try (DB snapshotDB = JniDBFactory.factory.open(newLevelDBDir, options)) {
+        try (DBIterator iterator =
+            originalDB.iterator(new ReadOptions().snapshot(snapshot))) {
+          iterator.seekToFirst();
+          while (iterator.hasNext()) {
+            Map.Entry<byte[], byte[]> entry = iterator.next();
+            snapshotDB.put(entry.getKey(), entry.getValue());
+          }
+        }
+      }
+    }
+
+    return snapshotDir;
+  }
+
+  /**
+   * Compress the given aliasmap directory as tar.gz.
+   *
+   * @return a reference to the compressed aliasmap.
+   * @throws IOException
+   */
+  private static File getCompressedAliasMap(File aliasMapDir)
+      throws IOException {
+    File outCompressedFile = new File(aliasMapDir.getParent(), TAR_NAME);
+    BufferedOutputStream bOut = null;
+    GzipCompressorOutputStream gzOut = null;
+    TarArchiveOutputStream tOut = null;
+    try {
+      bOut = new BufferedOutputStream(new FileOutputStream(outCompressedFile));
+      gzOut = new GzipCompressorOutputStream(bOut);
+      tOut = new TarArchiveOutputStream(gzOut);
+      addFileToTarGzRecursively(tOut, aliasMapDir, "", new Configuration());
+    } finally {
+      if (tOut != null) {
+        tOut.finish();
+      }
+      IOUtils.cleanupWithLogger(null, tOut, gzOut, bOut);
+    }
+    return outCompressedFile;
+  }
+
+  /**
+   * Add all contents of the given file to the archive.
+   *
+   * @param tOut archive to use.
+   * @param file file to archive.
+   * @param prefix path prefix.
+   * @throws IOException
+   */
+  private static void addFileToTarGzRecursively(TarArchiveOutputStream tOut,
+      File file, String prefix, Configuration conf) throws IOException {
+    String entryName = prefix + file.getName();
+    TarArchiveEntry tarEntry = new TarArchiveEntry(file, entryName);
+    tOut.putArchiveEntry(tarEntry);
+
+    LOG.debug("Adding entry {} to alias map archive", entryName);
+    if (file.isFile()) {
+      try (FileInputStream in = new FileInputStream(file)) {
+        IOUtils.copyBytes(in, tOut, conf, false);
+      }
+      tOut.closeArchiveEntry();
+    } else {
+      tOut.closeArchiveEntry();
+      File[] children = file.listFiles();
+      if (children != null) {
+        for (File child : children) {
+          // skip the LOCK file
+          if (!child.getName().equals("LOCK")) {
+            addFileToTarGzRecursively(tOut, child, entryName + "/", conf);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Extract the aliasmap archive to complete the bootstrap process. This method
+   * has to be called after the aliasmap archive is transfered from the primary
+   * Namenode.
+   *
+   * @param aliasMap location of the aliasmap.
+   * @throws IOException
+   */
+  public static void completeBootstrapTransfer(File aliasMap)
+      throws IOException {
+    File tarname = new File(aliasMap, TAR_NAME);
+    if (!tarname.exists()) {
+      throw new IOException(
+          "Aliasmap archive (" + tarname + ") does not exist");
+    }
+    try {
+      FileUtil.unTar(tarname, aliasMap);
+    } finally {
+      // delete the archive.
+      if(!FileUtil.fullyDelete(tarname)) {
+        LOG.warn("Failed to fully delete aliasmap archive: " + tarname);
+      }
+    }
+  }
+
   /**
    * CheckedFunction is akin to {@link java.util.function.Function} but
    * specifies an IOException.

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java

@@ -146,6 +146,14 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
     return conf;
   }
 
+  /**
+   * Get the {@link InMemoryAliasMap} used by this server.
+   * @return the inmemoryaliasmap used.
+   */
+  public InMemoryAliasMap getAliasMap() {
+    return aliasMap;
+  }
+
   @Override
   public void close() {
     LOG.info("Stopping InMemoryLevelDBAliasMapServer");

+ 29 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
 import org.apache.hadoop.hdfs.server.common.Util;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
@@ -140,6 +141,16 @@ public class ImageServlet extends HttpServlet {
               long elapsed = monotonicNow() - start;
               metrics.addGetEdit(elapsed);
             }
+          } else if (parsedParams.isGetAliasMap()) {
+            InMemoryAliasMap aliasMap =
+                NameNodeHttpServer.getAliasMapFromContext(context);
+            long start = monotonicNow();
+            InMemoryAliasMap.transferForBootstrap(response, conf, aliasMap);
+            // Metrics non-null only when used inside name node
+            if (metrics != null) {
+              long elapsed = monotonicNow() - start;
+              metrics.addGetAliasMap(elapsed);
+            }
           }
           return null;
         }
@@ -230,7 +241,7 @@ public class ImageServlet extends HttpServlet {
     return throttler;
   }
 
-  private static DataTransferThrottler getThrottlerForBootstrapStandby(
+  public static DataTransferThrottler getThrottlerForBootstrapStandby(
       Configuration conf) {
     long transferBandwidth =
         conf.getLongBytes(
@@ -337,6 +348,11 @@ public class ImageServlet extends HttpServlet {
           remoteStorageInfo.toColonSeparatedString();
   }
 
+  static String getParamStringForAliasMap(
+        boolean isBootstrapStandby) {
+    return "getaliasmap=1&" + IS_BOOTSTRAP_STANDBY + "=" + isBootstrapStandby;
+  }
+
   static class GetImageParams {
     private boolean isGetImage;
     private boolean isGetEdit;
@@ -345,6 +361,7 @@ public class ImageServlet extends HttpServlet {
     private String storageInfoString;
     private boolean fetchLatest;
     private boolean isBootstrapStandby;
+    private boolean isGetAliasMap;
 
     /**
      * @param request the object from which this servlet reads the url contents
@@ -385,10 +402,16 @@ public class ImageServlet extends HttpServlet {
           endTxId = ServletUtil.parseLongParam(request, END_TXID_PARAM);
         } else if (key.equals(STORAGEINFO_PARAM)) {
           storageInfoString = val[0];
+        } else if (key.equals("getaliasmap")) {
+          isGetAliasMap = true;
+          String bootstrapStandby = ServletUtil.getParameter(request,
+              IS_BOOTSTRAP_STANDBY);
+          isBootstrapStandby = bootstrapStandby != null &&
+              Boolean.parseBoolean(bootstrapStandby);
         }
       }
 
-      int numGets = (isGetImage?1:0) + (isGetEdit?1:0);
+      int numGets = (isGetImage?1:0) + (isGetEdit?1:0) + (isGetAliasMap?1:0);
       if ((numGets > 1) || (numGets == 0)) {
         throw new IOException("Illegal parameters to TransferFsImage");
       }
@@ -429,7 +452,10 @@ public class ImageServlet extends HttpServlet {
     boolean shouldFetchLatest() {
       return fetchLatest;
     }
-    
+
+    boolean isGetAliasMap() {
+      return isGetAliasMap;
+    }
   }
 
   /**

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -713,12 +713,20 @@ public class NameNode extends ReconfigurableBase implements
     if (NamenodeRole.NAMENODE == role) {
       httpServer.setNameNodeAddress(getNameNodeAddress());
       httpServer.setFSImage(getFSImage());
+      if (levelDBAliasMapServer != null) {
+        httpServer.setAliasMap(levelDBAliasMapServer.getAliasMap());
+      }
     }
 
     startCommonServices(conf);
     startMetricsLogger(conf);
   }
 
+  @VisibleForTesting
+  public InMemoryLevelDBAliasMapServer getAliasMapServer() {
+    return levelDBAliasMapServer;
+  }
+
   private void startAliasMapServerIfNecessary(Configuration conf)
       throws IOException {
     if (conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED,
@@ -795,6 +803,9 @@ public class NameNode extends ReconfigurableBase implements
       startHttpServer(conf);
       httpServer.setNameNodeAddress(getNameNodeAddress());
       httpServer.setFSImage(getFSImage());
+      if (levelDBAliasMapServer != null) {
+        httpServer.setAliasMap(levelDBAliasMapServer.getAliasMap());
+      }
     }
     rpcServer.start();
     try {

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -68,6 +69,7 @@ public class NameNodeHttpServer {
   public static final String FSIMAGE_ATTRIBUTE_KEY = "name.system.image";
   protected static final String NAMENODE_ATTRIBUTE_KEY = "name.node";
   public static final String STARTUP_PROGRESS_ATTRIBUTE_KEY = "startup.progress";
+  public static final String ALIASMAP_ATTRIBUTE_KEY = "name.system.aliasmap";
 
   NameNodeHttpServer(Configuration conf, NameNode nn,
       InetSocketAddress bindAddress) {
@@ -288,6 +290,15 @@ public class NameNodeHttpServer {
     httpServer.setAttribute(STARTUP_PROGRESS_ATTRIBUTE_KEY, prog);
   }
 
+  /**
+   * Sets the aliasmap URI.
+   *
+   * @param aliasMap the alias map used.
+   */
+  void setAliasMap(InMemoryAliasMap aliasMap) {
+    httpServer.setAttribute(ALIASMAP_ATTRIBUTE_KEY, aliasMap);
+  }
+
   private static void setupServlets(HttpServer2 httpServer, Configuration conf) {
     httpServer.addInternalServlet("startupProgress",
         StartupProgressServlet.PATH_SPEC, StartupProgressServlet.class);
@@ -312,6 +323,10 @@ public class NameNodeHttpServer {
     return (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
   }
 
+  static InMemoryAliasMap getAliasMapFromContext(ServletContext context) {
+    return (InMemoryAliasMap) context.getAttribute(ALIASMAP_ATTRIBUTE_KEY);
+  }
+
   public static InetSocketAddress getNameNodeAddressFromContext(
       ServletContext context) {
     return (InetSocketAddress)context.getAttribute(

+ 20 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -25,6 +25,7 @@ import java.io.OutputStream;
 import java.net.HttpURLConnection;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -32,6 +33,7 @@ import java.util.Map.Entry;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -192,7 +194,24 @@ public class TransferFsImage {
       }
     }
   }
- 
+
+  /**
+   * Download the InMemoryAliasMap from the remote NN.
+   * @param fsName http address of remote NN.
+   * @param aliasMap location of the alias map.
+   * @param isBootstrapStandby flag to indicate if for bootstrap of standby.
+   * @throws IOException
+   */
+  public static void downloadAliasMap(URL fsName, File aliasMap,
+        boolean isBootstrapStandby) throws IOException {
+    String paramString = ImageServlet.getParamStringForAliasMap(
+        isBootstrapStandby);
+    getFileClient(fsName, paramString, Arrays.asList(aliasMap), null, false);
+    LOG.info("Downloaded file " + aliasMap.getName() + " size " +
+        aliasMap.length() + " bytes.");
+    InMemoryAliasMap.completeBootstrapTransfer(aliasMap);
+  }
+
   /**
    * Requests that the NameNode download an image from this node.
    *

+ 90 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -20,23 +20,27 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -91,6 +95,9 @@ public class BootstrapStandby implements Tool, Configurable {
   private boolean interactive = true;
   private boolean skipSharedEditsCheck = false;
 
+  private boolean inMemoryAliasMapEnabled;
+  private String aliasMapPath;
+
   // Exit/return codes.
   static final int ERR_CODE_FAILED_CONNECT = 2;
   static final int ERR_CODE_INVALID_VERSION = 3;
@@ -235,6 +242,12 @@ public class BootstrapStandby implements Tool, Configurable {
     if (!isUpgradeFinalized) {
       doUpgrade(storage);
     }
+
+    if (inMemoryAliasMapEnabled) {
+      return formatAndDownloadAliasMap(aliasMapPath, proxyInfo);
+    } else {
+      LOG.info("Skipping InMemoryAliasMap bootstrap as it was not configured");
+    }
     return 0;
   }
 
@@ -392,7 +405,7 @@ public class BootstrapStandby implements Tool, Configurable {
   private boolean checkLayoutVersion(NamespaceInfo nsInfo) throws IOException {
     return (nsInfo.getLayoutVersion() == HdfsServerConstants.NAMENODE_LAYOUT_VERSION);
   }
-  
+
   private void parseConfAndFindOtherNN() throws IOException {
     Configuration conf = getConf();
     nsId = DFSUtil.getNamenodeNameServiceId(conf);
@@ -433,6 +446,82 @@ public class BootstrapStandby implements Tool, Configurable {
     editUrisToFormat = FSNamesystem.getNamespaceEditsDirs(
         conf, false);
     sharedEditsUris = FSNamesystem.getSharedEditsDirs(conf);
+
+    parseProvidedConfigurations(conf);
+  }
+
+  private void parseProvidedConfigurations(Configuration configuration)
+      throws IOException {
+    // if provided and in-memory aliasmap are enabled,
+    // get the aliasmap location.
+    boolean providedEnabled = configuration.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED,
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT);
+    boolean inmemoryAliasmapConfigured = configuration.getBoolean(
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED,
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED_DEFAULT);
+    if (providedEnabled && inmemoryAliasmapConfigured) {
+      inMemoryAliasMapEnabled = true;
+      aliasMapPath = configuration.get(
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
+    } else {
+      inMemoryAliasMapEnabled = false;
+      aliasMapPath = null;
+    }
+  }
+
+  /**
+   * A storage directory for aliasmaps. This is primarily used for the
+   * StorageDirectory#hasSomeData for formatting aliasmap directories.
+   */
+  private static class AliasMapStorageDirectory extends StorageDirectory {
+
+    AliasMapStorageDirectory(File aliasMapDir) {
+      super(aliasMapDir);
+    }
+
+    @Override
+    public String toString() {
+      return "AliasMap directory = " + this.getRoot();
+    }
+  }
+
+  /**
+   * Format, if needed, and download the aliasmap.
+   * @param pathAliasMap the path where the aliasmap should be downloaded.
+   * @param proxyInfo remote namenode to get the aliasmap from.
+   * @return 0 on a successful transfer, and error code otherwise.
+   * @throws IOException
+   */
+  private int formatAndDownloadAliasMap(String pathAliasMap,
+      RemoteNameNodeInfo proxyInfo) throws IOException {
+    LOG.info("Bootstrapping the InMemoryAliasMap from "
+        + proxyInfo.getHttpAddress());
+    if (pathAliasMap == null) {
+      throw new IOException("InMemoryAliasMap enabled with null location");
+    }
+    File aliasMapFile = new File(pathAliasMap);
+    if (aliasMapFile.exists()) {
+      AliasMapStorageDirectory aliasMapSD =
+          new AliasMapStorageDirectory(aliasMapFile);
+      if (!Storage.confirmFormat(
+          Arrays.asList(aliasMapSD), force, interactive)) {
+        return ERR_CODE_ALREADY_FORMATTED;
+      } else {
+        if (!FileUtil.fullyDelete(aliasMapFile)) {
+          throw new IOException(
+              "Cannot remove current alias map: " + aliasMapFile);
+        }
+      }
+    }
+
+    // create the aliasmap location.
+    if (!aliasMapFile.mkdirs()) {
+      throw new IOException("Cannot create directory " + aliasMapFile);
+    }
+    TransferFsImage.downloadAliasMap(proxyInfo.getHttpAddress(), aliasMapFile,
+        true);
+    return 0;
   }
 
   @Override

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -151,6 +151,8 @@ public class NameNodeMetrics {
   MutableRate getEdit;
   @Metric("GetImageServlet getImage")
   MutableRate getImage;
+  @Metric("GetImageServlet getAliasMap")
+  MutableRate getAliasMap;
   @Metric("GetImageServlet putImage")
   MutableRate putImage;
 
@@ -394,6 +396,10 @@ public class NameNodeMetrics {
     getImage.add(latency);
   }
 
+  public void addGetAliasMap(long latency) {
+    getAliasMap.add(latency);
+  }
+
   public void addPutImage(long latency) {
     putImage.add(latency);
   }

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -67,6 +67,8 @@ import java.util.concurrent.TimeoutException;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -3312,6 +3314,27 @@ public class MiniDFSCluster implements AutoCloseable {
     }
   }
 
+  /**
+   * Setup the namenode-level PROVIDED configurations, using the
+   * {@link InMemoryLevelDBAliasMapClient}.
+   *
+   * @param conf Configuration, which is modified, to enable provided storage.
+   *        This cannot be null.
+   */
+  public static void setupNamenodeProvidedConfiguration(Configuration conf) {
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
+    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        InMemoryLevelDBAliasMapClient.class, BlockAliasMap.class);
+    File tempDirectory = new File(GenericTestUtils.getRandomizedTestDir(),
+        "in-memory-alias-map");
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDirectory.getAbsolutePath());
+    conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH,
+        tempDirectory.getAbsolutePath());
+  }
+
   @Override
   public void close() {
     shutdown();

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java

@@ -125,5 +125,35 @@ public class ITestInMemoryAliasMap {
     // no more results expected
     assertFalse(list.getNextBlock().isPresent());
   }
+
+  @Test
+  public void testSnapshot() throws Exception {
+    Block block1 = new Block(100);
+    Block block2 = new Block(200);
+    Path path = new Path("users", "alice");
+    ProvidedStorageLocation remoteLocation =
+        new ProvidedStorageLocation(path, 0, 1000, new byte[0]);
+    // write the first block
+    aliasMap.write(block1, remoteLocation);
+    // create snapshot
+    File snapshotFile = InMemoryAliasMap.createSnapshot(aliasMap);
+    // write the 2nd block after the snapshot
+    aliasMap.write(block2, remoteLocation);
+    // creata a new aliasmap object from the snapshot
+    InMemoryAliasMap snapshotAliasMap = null;
+    Configuration newConf = new Configuration();
+    newConf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        snapshotFile.getAbsolutePath());
+    try {
+      snapshotAliasMap = InMemoryAliasMap.init(newConf, bpid);
+      // now the snapshot should have the first block but not the second one.
+      assertTrue(snapshotAliasMap.read(block1).isPresent());
+      assertFalse(snapshotAliasMap.read(block2).isPresent());
+    } finally {
+      if (snapshotAliasMap != null) {
+        snapshotAliasMap.close();
+      }
+    }
+  }
 }
 

+ 0 - 45
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java

@@ -1,45 +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.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, "bpid")).withMessage(
-            InMemoryAliasMap.createPathErrorMessage(nonExistingDirectory));
-  }
-}

+ 0 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java

@@ -357,15 +357,6 @@ public class TestInMemoryLevelDBAliasMapClient {
     writeRead();
   }
 
-  @Test
-  public void testNonExistentFile() throws Exception {
-    // delete alias map location
-    FileUtils.deleteDirectory(tempDir);
-    // expect a RuntimeException when the aliasmap is started.
-    exception.expect(RuntimeException.class);
-    levelDBAliasMapServer.setConf(conf);
-  }
-
   @Test
   public void testNonExistentBlock() throws Exception {
     inMemoryLevelDBAliasMapClient.setConf(conf);

+ 99 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapAliasmap.java

@@ -0,0 +1,99 @@
+/**
+ * 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.ha;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+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.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.net.URL;
+import java.util.Optional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test for aliasmap bootstrap.
+ */
+public class TestBootstrapAliasmap {
+
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void setup() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster.setupNamenodeProvidedConfiguration(conf);
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1)
+        .build();
+    cluster.waitActive();
+  }
+
+  @Test
+  public void testAliasmapBootstrap() throws Exception {
+    InMemoryLevelDBAliasMapServer aliasMapServer =
+        cluster.getNameNode().getAliasMapServer();
+    // write some blocks to the aliasmap.
+    Block block1 = new Block(1000), block2 = new Block(1002);
+    Path path = new Path("/test1.dat");
+    aliasMapServer.write(new Block(block1),
+        new ProvidedStorageLocation(path, 0, 100, new byte[0]));
+    aliasMapServer.write(new Block(block2),
+        new ProvidedStorageLocation(path, 101, 200, new byte[0]));
+
+    File newLocation = GenericTestUtils.getRandomizedTestDir();
+    NameNode nn = cluster.getNameNode();
+    Configuration conf = cluster.getConfiguration(0);
+    String scheme = DFSUtil.getHttpClientScheme(conf);
+    URL nnHttpURL = DFSUtil.getInfoServerWithDefaultHost(
+        nn.getNameNodeAddress().getHostName(), conf, scheme).toURL();
+    // transfer the aliasmap.
+    newLocation.mkdirs();
+    TransferFsImage.downloadAliasMap(nnHttpURL, newLocation, true);
+
+    // create config for new aliasmap server at the new location.
+    Configuration newConfig = new Configuration();
+    newConfig.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        newLocation.getAbsolutePath());
+    newConfig.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "127.0.0.1:" + NetUtils.getFreeSocketPort());
+    String blockPoolId = nn.getNamesystem().getBlockPoolId();
+    InMemoryLevelDBAliasMapServer newServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, blockPoolId);
+    newServer.setConf(newConfig);
+    newServer.start();
+    // the server should have only 2 blocks.
+    assertEquals(2, newServer.list(Optional.empty()).getFileRegions().size());
+    assertNotNull(newServer.read(block1));
+    assertNotNull(newServer.read(block2));
+    assertEquals(blockPoolId, newServer.getBlockPoolId());
+  }
+}

+ 136 - 3
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java

@@ -36,6 +36,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
 
@@ -55,9 +56,12 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.Block;
 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.InMemoryAliasMapProtocol;
+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;
@@ -66,6 +70,7 @@ 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.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
@@ -78,6 +83,7 @@ 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.namenode.ha.BootstrapStandby;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -95,6 +101,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMOR
 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_BIND_HOST;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH;
+import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR;
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
 import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
@@ -262,9 +269,9 @@ public class ITestProvidedImplementation {
       if ((topo.isHA() || topo.isFederated()) && !doFormat) {
         builder.manageNameDfsDirs(true);
         builder.enableManagedDfsDirsRedundancy(false);
-        builder.manageNameDfsSharedDirs(false);
+        builder.manageNameDfsSharedDirs(true);
         List<File> nnDirs =
-            getProvidedNamenodeDirs(MiniDFSCluster.getBaseDirectory(), topo);
+            getProvidedNamenodeDirs(conf.get(HDFS_MINIDFS_BASEDIR), topo);
         for (File nnDir : nnDirs) {
           MiniDFSCluster.copyNameDirs(
               Collections.singletonList(nspath.toUri()),
@@ -932,7 +939,7 @@ public class ITestProvidedImplementation {
         // configure the InMemoryAliasMp.
         conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
         String directory = conf.get(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
-        if (directory == null) {
+        if (directory == null || !new File(directory).exists()) {
           throw new IllegalArgumentException("In-memory alias map configured"
               + "with the proper location; Set "
               + DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
@@ -948,6 +955,9 @@ public class ITestProvidedImplementation {
         conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
             new File(new Path(nnDir, dirName).toUri()).getAbsolutePath());
         conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
+
+        // format the shared edits dir with the proper VERSION file.
+        NameNode.initializeSharedEdits(conf);
       } else {
         if (!completedNNs.contains(nnIndex)) {
           // format the NN directories for non-provided namespaces
@@ -1197,4 +1207,127 @@ public class ITestProvidedImplementation {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testBootstrapAliasMap() throws Exception {
+    int numNamenodes = 3;
+    MiniDFSNNTopology topology =
+        MiniDFSNNTopology.simpleHATopology(numNamenodes);
+
+    createInMemoryAliasMapImage();
+    conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
+    conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
+    providedNameservice = topology.getNameservices().get(0).getId();
+    // configure the AliasMap addresses
+    configureAliasMapAddresses(topology, providedNameservice);
+    startCluster(nnDirPath, 2,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false, null, topology, new MiniDFSClusterBuilderAliasMap(conf));
+
+    // make NN with index 0 the active, shutdown and delete the directories
+    // of others. This will delete the aliasmap on these namenodes as well.
+    cluster.transitionToActive(0);
+    verifyFileSystemContents(0);
+    for (int nnIndex = 1; nnIndex < numNamenodes; nnIndex++) {
+      cluster.shutdownNameNode(nnIndex);
+      // delete the namenode directories including alias map.
+      for (URI u : cluster.getNameDirs(nnIndex)) {
+        File dir = new File(u.getPath());
+        assertTrue(FileUtil.fullyDelete(dir));
+      }
+    }
+
+    // start the other namenodes and bootstrap them
+    for (int index = 1; index < numNamenodes; index++) {
+      // add some content to aliasmap dir
+      File aliasMapDir = new File(fBASE, "aliasmap-" + index);
+      // create a directory inside aliasMapDir
+      if (!new File(aliasMapDir, "tempDir").mkdirs()) {
+        throw new IOException("Unable to create directory " + aliasMapDir);
+      }
+      Configuration currNNConf = cluster.getConfiguration(index);
+      currNNConf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+          aliasMapDir.getAbsolutePath());
+      // without force this should fail as aliasmap is not empty.
+      int rc =
+          BootstrapStandby.run(new String[] {"-nonInteractive"}, currNNConf);
+      assertNotEquals(0, rc);
+      // force deletes the contents of the aliasmap.
+      rc = BootstrapStandby.run(new String[] {"-nonInteractive", "-force"},
+          currNNConf);
+      assertEquals(0, rc);
+    }
+
+    // check if aliasmap files are the same on all NNs
+    checkInMemoryAliasMapContents(0, numNamenodes);
+    // restart the killed namenodes.
+    for (int i = 1; i < numNamenodes; i++) {
+      cluster.restartNameNode(i, false);
+    }
+
+    cluster.waitClusterUp();
+    cluster.waitActive();
+
+    // transition to namenode 1 as the active
+    int nextNN = 1;
+    cluster.shutdownNameNode(0);
+    cluster.transitionToActive(nextNN);
+    // all files must be accessible from nextNN.
+    verifyFileSystemContents(nextNN);
+  }
+
+  /**
+   * Check if the alias map contents of the namenodes are the same as the base.
+   *
+   * @param baseNN index of the namenode to compare against.
+   * @param numNamenodes total number of namenodes in the cluster.
+   */
+  private void checkInMemoryAliasMapContents(int baseNN, int numNamenodes)
+      throws Exception {
+    InMemoryLevelDBAliasMapServer baseAliasMap =
+        cluster.getNameNode(baseNN).getAliasMapServer();
+    for (int i = 0; i < numNamenodes; i++) {
+      if (baseNN == i) {
+        continue;
+      }
+      InMemoryLevelDBAliasMapServer otherAliasMap =
+          cluster.getNameNode(baseNN).getAliasMapServer();
+      verifyAliasMapEquals(baseAliasMap, otherAliasMap);
+    }
+  }
+
+  /**
+   * Verify that the contents of the aliasmaps are the same.
+   *
+   * @param aliasMap1
+   * @param aliasMap2
+   */
+  private void verifyAliasMapEquals(InMemoryLevelDBAliasMapServer aliasMap1,
+      InMemoryLevelDBAliasMapServer aliasMap2) throws Exception {
+    Set<FileRegion> fileRegions1 = getFileRegions(aliasMap1);
+    Set<FileRegion> fileRegions2 = getFileRegions(aliasMap2);
+    assertTrue(fileRegions1.equals(fileRegions2));
+  }
+
+  /**
+   * Get all the aliases the aliasmap contains.
+   *
+   * @param aliasMap aliasmap to explore.
+   * @return set of all aliases.
+   * @throws IOException
+   */
+  private Set<FileRegion> getFileRegions(InMemoryLevelDBAliasMapServer aliasMap)
+      throws IOException {
+    Set<FileRegion> fileRegions = new HashSet<>();
+    Optional<Block> marker = Optional.empty();
+    while (true) {
+      InMemoryAliasMapProtocol.IterationResult result = aliasMap.list(marker);
+      fileRegions.addAll(result.getFileRegions());
+      marker = result.getNextBlock();
+      if (!marker.isPresent()) {
+        break;
+      }
+    }
+    return fileRegions;
+  }
 }