You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by vi...@apache.org on 2017/12/15 18:39:18 UTC

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
index 1ef2f2b..faf1f83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.junit.Before;
 import org.junit.Test;
-
 import java.io.IOException;
 
 import static org.junit.Assert.assertNotNull;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
new file mode 100644
index 0000000..4a9661b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+/**
+ * Tests the {@link InMemoryLevelDBAliasMapClient}.
+ */
+public class TestInMemoryLevelDBAliasMapClient {
+
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+  private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
+  private File tempDir;
+  private Configuration conf;
+
+  @Before
+  public void setUp() throws IOException {
+    levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+    conf = new Configuration();
+    int port = 9876;
+
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:" + port);
+    tempDir = Files.createTempDir();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDir.getAbsolutePath());
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    levelDBAliasMapServer.close();
+    inMemoryLevelDBAliasMapClient.close();
+    FileUtils.deleteDirectory(tempDir);
+  }
+
+  @Test
+  public void writeRead() throws Exception {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block = new Block(42, 43, 44);
+    byte[] nonce = "blackbird".getBytes();
+    ProvidedStorageLocation providedStorageLocation
+        = new ProvidedStorageLocation(new Path("cuckoo"),
+        45, 46, nonce);
+    BlockAliasMap.Writer<FileRegion> writer =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+    writer.store(new FileRegion(block, providedStorageLocation));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    Optional<FileRegion> fileRegion = reader.resolve(block);
+    assertEquals(new FileRegion(block, providedStorageLocation),
+        fileRegion.get());
+  }
+
+  @Test
+  public void iterateSingleBatch() throws Exception {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    byte[] nonce1 = "blackbird".getBytes();
+    byte[] nonce2 = "cuckoo".getBytes();
+    ProvidedStorageLocation providedStorageLocation1 =
+        new ProvidedStorageLocation(new Path("eagle"),
+        46, 47, nonce1);
+    ProvidedStorageLocation providedStorageLocation2 =
+        new ProvidedStorageLocation(new Path("falcon"),
+            46, 47, nonce2);
+    BlockAliasMap.Writer<FileRegion> writer1 =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+    writer1.store(new FileRegion(block1, providedStorageLocation1));
+    BlockAliasMap.Writer<FileRegion> writer2 =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+    writer2.store(new FileRegion(block2, providedStorageLocation2));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    List<FileRegion> actualFileRegions =
+        Lists.newArrayListWithCapacity(2);
+    for (FileRegion fileRegion : reader) {
+      actualFileRegions.add(fileRegion);
+    }
+
+    assertArrayEquals(
+        new FileRegion[] {new FileRegion(block1, providedStorageLocation1),
+            new FileRegion(block2, providedStorageLocation2)},
+        actualFileRegions.toArray());
+  }
+
+  @Test
+  public void iterateThreeBatches() throws Exception {
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE, "2");
+    levelDBAliasMapServer.setConf(conf);
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.start();
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    Block block3 = new Block(44, 45, 46);
+    Block block4 = new Block(47, 48, 49);
+    Block block5 = new Block(50, 51, 52);
+    Block block6 = new Block(53, 54, 55);
+    byte[] nonce1 = "blackbird".getBytes();
+    byte[] nonce2 = "cuckoo".getBytes();
+    byte[] nonce3 = "sparrow".getBytes();
+    byte[] nonce4 = "magpie".getBytes();
+    byte[] nonce5 = "seagull".getBytes();
+    byte[] nonce6 = "finch".getBytes();
+    ProvidedStorageLocation providedStorageLocation1 =
+        new ProvidedStorageLocation(new Path("eagle"),
+            46, 47, nonce1);
+    ProvidedStorageLocation providedStorageLocation2 =
+        new ProvidedStorageLocation(new Path("falcon"),
+            48, 49, nonce2);
+    ProvidedStorageLocation providedStorageLocation3 =
+        new ProvidedStorageLocation(new Path("robin"),
+            50, 51, nonce3);
+    ProvidedStorageLocation providedStorageLocation4 =
+        new ProvidedStorageLocation(new Path("parakeet"),
+            52, 53, nonce4);
+    ProvidedStorageLocation providedStorageLocation5 =
+        new ProvidedStorageLocation(new Path("heron"),
+            54, 55, nonce5);
+    ProvidedStorageLocation providedStorageLocation6 =
+        new ProvidedStorageLocation(new Path("duck"),
+            56, 57, nonce6);
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block1, providedStorageLocation1));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block2, providedStorageLocation2));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block3, providedStorageLocation3));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block4, providedStorageLocation4));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block5, providedStorageLocation5));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null)
+        .store(new FileRegion(block6, providedStorageLocation6));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    List<FileRegion> actualFileRegions =
+        Lists.newArrayListWithCapacity(6);
+    for (FileRegion fileRegion : reader) {
+      actualFileRegions.add(fileRegion);
+    }
+
+    FileRegion[] expectedFileRegions =
+        new FileRegion[] {new FileRegion(block1, providedStorageLocation1),
+            new FileRegion(block2, providedStorageLocation2),
+            new FileRegion(block3, providedStorageLocation3),
+            new FileRegion(block4, providedStorageLocation4),
+            new FileRegion(block5, providedStorageLocation5),
+            new FileRegion(block6, providedStorageLocation6)};
+    assertArrayEquals(expectedFileRegions, actualFileRegions.toArray());
+  }
+
+
+  class ReadThread implements Runnable {
+    private final Block block;
+    private final BlockAliasMap.Reader<FileRegion> reader;
+    private int delay;
+    private Optional<FileRegion> fileRegionOpt;
+
+    ReadThread(Block block, BlockAliasMap.Reader<FileRegion> reader,
+        int delay) {
+      this.block = block;
+      this.reader = reader;
+      this.delay = delay;
+    }
+
+    public Optional<FileRegion> getFileRegion() {
+      return fileRegionOpt;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(delay);
+        fileRegionOpt = reader.resolve(block);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  class WriteThread implements Runnable {
+    private final Block block;
+    private final BlockAliasMap.Writer<FileRegion> writer;
+    private final ProvidedStorageLocation providedStorageLocation;
+    private int delay;
+
+    WriteThread(Block block, ProvidedStorageLocation providedStorageLocation,
+        BlockAliasMap.Writer<FileRegion> writer, int delay) {
+      this.block = block;
+      this.writer = writer;
+      this.providedStorageLocation = providedStorageLocation;
+      this.delay = delay;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(delay);
+        writer.store(new FileRegion(block, providedStorageLocation));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public FileRegion generateRandomFileRegion(int seed) {
+    Block block = new Block(seed, seed + 1, seed + 2);
+    Path path = new Path("koekoek");
+    byte[] nonce = new byte[0];
+    ProvidedStorageLocation providedStorageLocation =
+        new ProvidedStorageLocation(path, seed + 3, seed + 4, nonce);
+    return new FileRegion(block, providedStorageLocation);
+  }
+
+  @Test
+  public void multipleReads() throws IOException {
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+
+    Random r = new Random();
+    List<FileRegion> expectedFileRegions = r.ints(0, 200)
+        .limit(50)
+        .boxed()
+        .map(i -> generateRandomFileRegion(i))
+        .collect(Collectors.toList());
+
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null);
+    BlockAliasMap.Writer<FileRegion> writer =
+        inMemoryLevelDBAliasMapClient.getWriter(null);
+
+    ExecutorService executor = Executors.newCachedThreadPool();
+
+    List<ReadThread> readThreads = expectedFileRegions
+        .stream()
+        .map(fileRegion -> new ReadThread(fileRegion.getBlock(),
+            reader,
+            4000))
+        .collect(Collectors.toList());
+
+
+    List<? extends Future<?>> readFutures =
+        readThreads.stream()
+            .map(readThread -> executor.submit(readThread))
+            .collect(Collectors.toList());
+
+    List<? extends Future<?>> writeFutures = expectedFileRegions
+        .stream()
+        .map(fileRegion -> new WriteThread(fileRegion.getBlock(),
+            fileRegion.getProvidedStorageLocation(),
+            writer,
+            1000))
+        .map(writeThread -> executor.submit(writeThread))
+        .collect(Collectors.toList());
+
+    readFutures.stream()
+        .map(readFuture -> {
+          try {
+            return readFuture.get();
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          } catch (ExecutionException e) {
+            throw new RuntimeException(e);
+          }
+        })
+        .collect(Collectors.toList());
+
+    List<FileRegion> actualFileRegions = readThreads.stream()
+        .map(readThread -> readThread.getFileRegion().get())
+        .collect(Collectors.toList());
+
+    assertThat(actualFileRegions).containsExactlyInAnyOrder(
+        expectedFileRegions.toArray(new FileRegion[0]));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
new file mode 100644
index 0000000..43fc68c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.iq80.leveldb.DBException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.File;
+import java.io.IOException;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests the in-memory alias map with a mock level-db implementation.
+ */
+public class TestLevelDbMockAliasMapClient {
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+  private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
+  private File tempDir;
+  private Configuration conf;
+  private InMemoryAliasMap aliasMapMock;
+
+  @Before
+  public void setUp() throws IOException {
+    aliasMapMock = mock(InMemoryAliasMap.class);
+    levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
+        config -> aliasMapMock);
+    conf = new Configuration();
+    int port = 9877;
+
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:" + port);
+    tempDir = Files.createTempDir();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDir.getAbsolutePath());
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    levelDBAliasMapServer.close();
+    inMemoryLevelDBAliasMapClient.close();
+    FileUtils.deleteDirectory(tempDir);
+  }
+
+  @Test
+  public void readFailure() throws Exception {
+    Block block = new Block(42, 43, 44);
+    doThrow(new IOException())
+        .doThrow(new DBException())
+        .when(aliasMapMock)
+        .read(block);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getReader(null).resolve(block));
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getReader(null).resolve(block));
+  }
+
+  @Test
+  public void writeFailure() throws IOException {
+    Block block = new Block(42, 43, 44);
+    byte[] nonce = new byte[0];
+    Path path = new Path("koekoek");
+    ProvidedStorageLocation providedStorageLocation =
+        new ProvidedStorageLocation(path, 45, 46, nonce);
+
+    doThrow(new IOException())
+        .when(aliasMapMock)
+        .write(block, providedStorageLocation);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getWriter(null)
+                .store(new FileRegion(block, providedStorageLocation)));
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getWriter(null)
+                .store(new FileRegion(block, providedStorageLocation)));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
index 4190730..8bdbaa4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -43,6 +43,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
@@ -214,7 +215,8 @@ public class TestProvidedImpl {
             }
 
             @Override
-            public FileRegion resolve(Block ident) throws IOException {
+            public Optional<FileRegion> resolve(Block ident)
+                throws IOException {
               return null;
             }
           };
@@ -232,6 +234,11 @@ public class TestProvidedImpl {
     public void refresh() throws IOException {
       // do nothing!
     }
+
+    @Override
+    public void close() throws IOException {
+      // do nothing
+    }
   }
 
   private static Storage.StorageDirectory createLocalStorageDirectory(

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 09e8f97..70e4c33 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -27,11 +27,13 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Files;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Random;
 import java.util.Set;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -39,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.BlockMissingException;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -48,6 +51,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -56,6 +61,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 
@@ -172,16 +178,16 @@ public class TestNameNodeProvidedImplementation {
 
   void createImage(TreeWalk t, Path out,
       Class<? extends BlockResolver> blockIdsClass) throws Exception {
-    createImage(t, out, blockIdsClass, "");
+    createImage(t, out, blockIdsClass, "", TextFileRegionAliasMap.class);
   }
 
   void createImage(TreeWalk t, Path out,
-      Class<? extends BlockResolver> blockIdsClass, String clusterID)
-      throws Exception {
+      Class<? extends BlockResolver> blockIdsClass, String clusterID,
+      Class<? extends BlockAliasMap> aliasMapClass) throws Exception {
     ImageWriter.Options opts = ImageWriter.defaults();
     opts.setConf(conf);
     opts.output(out.toString())
-        .blocks(TextFileRegionAliasMap.class)
+        .blocks(aliasMapClass)
         .blockIds(blockIdsClass)
         .clusterID(clusterID);
     try (ImageWriter w = new ImageWriter(opts)) {
@@ -389,17 +395,8 @@ public class TestNameNodeProvidedImplementation {
     return ret;
   }
 
-  @Test(timeout=30000)
-  public void testBlockRead() throws Exception {
-    conf.setClass(ImageWriter.Options.UGI_CLASS,
-        FsUGIResolver.class, UGIResolver.class);
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3,
-        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
-        false);
+  private void verifyFileSystemContents() throws Exception {
     FileSystem fs = cluster.getFileSystem();
-    Thread.sleep(2000);
     int count = 0;
     // read NN metadata, verify contents match
     for (TreePath e : new FSTreeWalk(NAMEPATH, conf)) {
@@ -683,7 +680,7 @@ public class TestNameNodeProvidedImplementation {
   public void testSetClusterID() throws Exception {
     String clusterID = "PROVIDED-CLUSTER";
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class, clusterID);
+        FixedBlockResolver.class, clusterID, TextFileRegionAliasMap.class);
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
@@ -744,4 +741,42 @@ public class TestNameNodeProvidedImplementation {
       verifyFileLocation(i, expectedLocations);
     }
   }
+
+
+  // This test will fail until there is a refactoring of the FileRegion
+  // (HDFS-12713).
+  @Test(expected=BlockMissingException.class)
+  public void testInMemoryAliasMap() throws Exception {
+    conf.setClass(ImageWriter.Options.UGI_CLASS,
+        FsUGIResolver.class, UGIResolver.class);
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        InMemoryLevelDBAliasMapClient.class, BlockAliasMap.class);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:32445");
+    File tempDirectory =
+        Files.createTempDirectory("in-memory-alias-map").toFile();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDirectory.getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
+
+    InMemoryLevelDBAliasMapServer levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+
+    createImage(new FSTreeWalk(NAMEPATH, conf),
+        NNDIRPATH,
+        FixedBlockResolver.class, "",
+        InMemoryLevelDBAliasMapClient.class);
+    levelDBAliasMapServer.close();
+
+    // start cluster with two datanodes,
+    // each with 1 PROVIDED volume and other DISK volume
+    startCluster(NNDIRPATH, 2,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
+        null, false);
+    verifyFileSystemContents();
+    FileUtils.deleteDirectory(tempDirectory);
+  }
+
 }


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