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:38:47 UTC

[01/50] [abbrv] hadoop git commit: HDFS-12910. Secure Datanode Starter should log the port when it fails to bind. Contributed by Stephen O'Donnell and Nanda kumar. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-9806 cf772b81b -> be27bcb90 (forced update)


HDFS-12910. Secure Datanode Starter should log the port when it fails to bind. Contributed by Stephen O'Donnell and Nanda kumar.


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

Branch: refs/heads/HDFS-9806
Commit: e1cb278cd0287ce47f923941147d17540b199a99
Parents: 95d4ec7
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Dec 14 19:20:57 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Dec 14 19:22:51 2017 -0800

----------------------------------------------------------------------
 .../server/datanode/SecureDataNodeStarter.java  | 29 +++++++--
 .../datanode/TestStartSecureDataNode.java       | 64 +++++++++++++++++++-
 2 files changed, 86 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1cb278c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
index cea7816..accf2bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.nio.channels.ServerSocketChannel;
+import java.net.BindException;
 
 /**
  * Utility class to start a datanode in a secure cluster, first obtaining 
@@ -102,7 +103,13 @@ public class SecureDataNodeStarter implements Daemon {
 
     ServerSocket ss = (socketWriteTimeout > 0) ? 
         ServerSocketChannel.open().socket() : new ServerSocket();
-    ss.bind(streamingAddr, backlogLength);
+    try {
+      ss.bind(streamingAddr, backlogLength);
+    } catch (BindException e) {
+      BindException newBe = appendMessageToBindException(e,
+          streamingAddr.toString());
+      throw newBe;
+    }
 
     // Check that we got the port we need
     if (ss.getLocalPort() != streamingAddr.getPort()) {
@@ -126,13 +133,20 @@ public class SecureDataNodeStarter implements Daemon {
     if (policy.isHttpEnabled()) {
       httpChannel = ServerSocketChannel.open();
       InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
-      httpChannel.socket().bind(infoSocAddr);
+      try {
+        httpChannel.socket().bind(infoSocAddr);
+      } catch (BindException e) {
+        BindException newBe = appendMessageToBindException(e,
+            infoSocAddr.toString());
+        throw newBe;
+      }
       InetSocketAddress localAddr = (InetSocketAddress) httpChannel.socket()
         .getLocalSocketAddress();
 
       if (localAddr.getPort() != infoSocAddr.getPort()) {
-        throw new RuntimeException("Unable to bind on specified info port in secure " +
-            "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+        throw new RuntimeException("Unable to bind on specified info port in " +
+            "secure context. Needed " + infoSocAddr.getPort() + ", got " +
+             ss.getLocalPort());
       }
       System.err.println("Successfully obtained privileged resources (streaming port = "
           + ss + " ) (http listener port = " + localAddr.getPort() +")");
@@ -149,4 +163,11 @@ public class SecureDataNodeStarter implements Daemon {
     return new SecureResources(ss, httpChannel);
   }
 
+  private static BindException appendMessageToBindException(BindException e,
+      String msg) {
+    BindException newBe = new BindException(e.getMessage() + " " + msg);
+    newBe.initCause(e.getCause());
+    newBe.setStackTrace(e.getStackTrace());
+    return newBe;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1cb278c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java
index 540c05f..51a843b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java
@@ -26,9 +26,14 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning;
+import org.apache.hadoop.net.NetUtils;
 import org.junit.Assume;
-import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
 
 /**
  * This test starts a 1 NameNode 1 DataNode MiniDFSCluster with
@@ -48,16 +53,18 @@ import org.junit.Test;
  *   dfs.datanode.keytab.file
  */
 public class TestStartSecureDataNode {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
   final static private int NUM_OF_DATANODES = 1;
 
-  @Before
-  public void testExternalKdcRunning() {
+  private void testExternalKdcRunning() {
     // Tests are skipped if external KDC is not running.
     Assume.assumeTrue(isExternalKdcRunning());
   }
 
   @Test
   public void testSecureNameNode() throws Exception {
+    testExternalKdcRunning();
     MiniDFSCluster cluster = null;
     try {
       String nnPrincipal =
@@ -104,4 +111,55 @@ public class TestStartSecureDataNode {
       }
     }
   }
+
+  /**
+   * This test doesn't require KDC or other security settings as it expects
+   * {@link java.net.BindException}. Testing is done with unprivileged port
+   * for {@code dfs.datanode.address}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testStreamingAddrBindException() throws Exception {
+    ServerSocket ss = new ServerSocket();
+    try {
+      ss.bind(new InetSocketAddress("localhost", 0));
+      thrown.expect(BindException.class);
+      thrown.expectMessage("localhost/127.0.0.1:" + ss.getLocalPort());
+
+      Configuration conf = new HdfsConfiguration();
+      conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY,
+          "localhost:" + ss.getLocalPort());
+      SecureDataNodeStarter.getSecureResources(conf);
+    } finally {
+      ss.close();
+    }
+  }
+
+  /**
+   * This test doesn't require KDC or other security settings as it expects
+   * {@link java.net.BindException}. Testing is done with unprivileged port
+   * for {@code dfs.datanode.http.address}.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWebServerAddrBindException() throws Exception {
+    ServerSocket ss = new ServerSocket();
+    try {
+      ss.bind(new InetSocketAddress("localhost", 0));
+      thrown.expect(BindException.class);
+      thrown.expectMessage("localhost/127.0.0.1:" + ss.getLocalPort());
+
+      Configuration conf = new HdfsConfiguration();
+      conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY,
+          "localhost:" + NetUtils.getFreeSocketPort());
+      conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY,
+          "localhost:" + ss.getLocalPort());
+
+      SecureDataNodeStarter.getSecureResources(conf);
+    } finally {
+      ss.close();
+    }
+  }
 }


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


[04/50] [abbrv] hadoop git commit: YARN-7638. Unit tests related to preemption for auto created leaf queues feature.Contributed by Suma Shivaprasad.

Posted by vi...@apache.org.
YARN-7638. Unit tests related to preemption for auto created leaf queues feature.Contributed by Suma Shivaprasad.


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

Branch: refs/heads/HDFS-9806
Commit: 890d3d06456a026d9551a0cf15ce3986b0641454
Parents: ee028bf
Author: Sunil G <su...@apache.org>
Authored: Fri Dec 15 13:00:57 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Dec 15 13:00:57 2017 +0530

----------------------------------------------------------------------
 .../scheduler/capacity/LeafQueue.java           |  10 +-
 ...stCapacitySchedulerAutoCreatedQueueBase.java |  38 +---
 ...citySchedulerAutoCreatedQueuePreemption.java | 202 +++++++++++++++++++
 ...TestCapacitySchedulerSurgicalPreemption.java |  62 ++++--
 4 files changed, 260 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/890d3d06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index ac1a26c..329f7de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -296,6 +296,10 @@ public class LeafQueue extends AbstractCSQueue {
               + queueCapacities.getAbsoluteMaximumCapacity()
               + " [= 1.0 maximumCapacity undefined, "
               + "(parentAbsoluteMaxCapacity * maximumCapacity) / 100 otherwise ]"
+              + "\n" + "effectiveMinResource=" +
+              getEffectiveCapacity(CommonNodeLabelsManager.NO_LABEL) + "\n"
+              + " , effectiveMaxResource=" +
+              getEffectiveMaxCapacity(CommonNodeLabelsManager.NO_LABEL)
               + "\n" + "userLimit = " + usersManager.getUserLimit()
               + " [= configuredUserLimit ]" + "\n" + "userLimitFactor = "
               + usersManager.getUserLimitFactor()
@@ -502,7 +506,11 @@ public class LeafQueue extends AbstractCSQueue {
           + ", " + "usedResources=" + queueUsage.getUsed() + ", "
           + "usedCapacity=" + getUsedCapacity() + ", " + "absoluteUsedCapacity="
           + getAbsoluteUsedCapacity() + ", " + "numApps=" + getNumApplications()
-          + ", " + "numContainers=" + getNumContainers();
+          + ", " + "numContainers=" + getNumContainers() + ", "
+          + "effectiveMinResource=" +
+          getEffectiveCapacity(CommonNodeLabelsManager.NO_LABEL) +
+          " , effectiveMaxResource=" +
+          getEffectiveMaxCapacity(CommonNodeLabelsManager.NO_LABEL);
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/890d3d06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java
index 035c460..d8d71c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java
@@ -184,7 +184,7 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
         ResourceScheduler.class);
 
-    setupQueueMappings(conf, PARENT_QUEUE, true, new int[] {0, 1, 2, 3});
+    setupQueueMappings(conf, PARENT_QUEUE, true, new int[] { 0, 1, 2, 3 });
 
     dispatcher = new SpyDispatcher();
     rmAppEventEventHandler = new SpyDispatcher.SpyRMAppEventHandler();
@@ -233,8 +233,8 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
     queuePlacementRules.add(YarnConfiguration.USER_GROUP_PLACEMENT_RULE);
     conf.setQueuePlacementRules(queuePlacementRules);
 
-    List<UserGroupMappingPlacementRule.QueueMapping> existingMappings = conf
-        .getQueueMappings();
+    List<UserGroupMappingPlacementRule.QueueMapping> existingMappings =
+        conf.getQueueMappings();
 
     //set queue mapping
     List<UserGroupMappingPlacementRule.QueueMapping> queueMappings =
@@ -244,8 +244,8 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
       UserGroupMappingPlacementRule.QueueMapping userQueueMapping =
           new UserGroupMappingPlacementRule.QueueMapping(
               UserGroupMappingPlacementRule.QueueMapping.MappingType.USER,
-              USER + userIds[i], getQueueMapping(parentQueue, USER +
-              userIds[i]));
+              USER + userIds[i],
+              getQueueMapping(parentQueue, USER + userIds[i]));
       queueMappings.add(userQueueMapping);
     }
 
@@ -439,34 +439,6 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
     return newMockRM;
   }
 
-  protected void checkQueueCapacities(CapacityScheduler newCS, float capacityC,
-      float capacityD) {
-    CSQueue rootQueue = newCS.getRootQueue();
-    CSQueue queueC = tcs.findQueue(rootQueue, C);
-    CSQueue queueD = tcs.findQueue(rootQueue, D);
-    CSQueue queueC1 = tcs.findQueue(queueC, C1);
-    CSQueue queueC2 = tcs.findQueue(queueC, C2);
-    CSQueue queueC3 = tcs.findQueue(queueC, C3);
-
-    float capC = capacityC / 100.0f;
-    float capD = capacityD / 100.0f;
-
-    tcs.checkQueueCapacity(queueC, capC, capC, 1.0f, 1.0f);
-    tcs.checkQueueCapacity(queueD, capD, capD, 1.0f, 1.0f);
-    tcs.checkQueueCapacity(queueC1, C1_CAPACITY / 100.0f,
-        (C1_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
-    tcs.checkQueueCapacity(queueC2, C2_CAPACITY / 100.0f,
-        (C2_CAPACITY / 100.0f) * capC, 1.0f, 1.0f);
-
-    if (queueC3 != null) {
-      ManagedParentQueue parentQueue = (ManagedParentQueue) queueC;
-      QueueCapacities cap =
-          parentQueue.getLeafQueueTemplate().getQueueCapacities();
-      tcs.checkQueueCapacity(queueC3, cap.getCapacity(),
-          (cap.getCapacity()) * capC, 1.0f, 1.0f);
-    }
-  }
-
   static String getQueueMapping(String parentQueue, String leafQueue) {
     return parentQueue + DOT + leafQueue;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/890d3d06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueuePreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueuePreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueuePreemption.java
new file mode 100644
index 0000000..a025f1e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueuePreemption.java
@@ -0,0 +1,202 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.C;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.D;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.E;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER0;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER1;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER2;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER3;
+
+/**
+ * Tests various preemption cases on auto-created leaf queues. All
+ * auto-created leaf queues will end up having same priority since they are set
+ * from template. Priority on ManagedParent Queues can be set however and
+ * priority based premption cases are based on that.
+ */
+public class TestCapacitySchedulerAutoCreatedQueuePreemption
+    extends TestCapacitySchedulerSurgicalPreemption {
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  public static CapacitySchedulerConfiguration
+      setupQueueConfigurationForSimpleSurgicalPreemption(
+      CapacitySchedulerConfiguration conf) {
+
+    //set up auto created queue configs
+    TestCapacitySchedulerAutoCreatedQueueBase.setupQueueMappings(conf, "c",
+        true, new int[] { 1, 2 });
+    //setup new queues with one of them auto enabled
+    // Define top-level queues
+    // Set childQueue for root
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "c" });
+    conf.setCapacity(C, 100f);
+
+    conf.setUserLimitFactor(C, 1.0f);
+    conf.setAutoCreateChildQueueEnabled(C, true);
+
+    //Setup leaf queue template configs
+    conf.setAutoCreatedLeafQueueConfigCapacity(C, 30.0f);
+    conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
+
+    return conf;
+  }
+
+  protected CapacitySchedulerConfiguration
+      setupQueueConfigurationForPriorityBasedPreemption(
+      CapacitySchedulerConfiguration conf) {
+
+    //set up auto created queue configs
+    TestCapacitySchedulerAutoCreatedQueueBase.setupQueueMappings(conf, "c",
+        true, new int[] { 1, 2 });
+
+    TestCapacitySchedulerAutoCreatedQueueBase.setupQueueMappings(conf, "d",
+        true, new int[] { 3, 4 });
+
+    TestCapacitySchedulerAutoCreatedQueueBase.setupQueueMappings(conf, "e",
+        true, new int[] { 0 });
+    //setup new queues with one of them auto enabled
+    // Define top-level queues
+    // Set childQueue for root
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "c", "d", "e" });
+    conf.setCapacity(C, 45f);
+    conf.setCapacity(D, 45f);
+    conf.setCapacity(E, 10f);
+
+    conf.setUserLimitFactor(E, 3.0f);
+    conf.setUserLimitFactor(C, 3.0f);
+    conf.setUserLimitFactor(D, 3.0f);
+    conf.setAutoCreateChildQueueEnabled(C, true);
+    conf.setAutoCreateChildQueueEnabled(D, true);
+    conf.setAutoCreateChildQueueEnabled(E, true);
+
+    //Setup leaf queue template configs
+    conf.setAutoCreatedLeafQueueConfigCapacity(C, 100f);
+    conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
+
+    conf.setAutoCreatedLeafQueueConfigCapacity(D, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigMaxCapacity(D, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimit(D, 100);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(D, 3.0f);
+
+    conf.setAutoCreatedLeafQueueConfigCapacity(E, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigMaxCapacity(E, 100.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimit(E, 100);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(E, 3.0f);
+
+    conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".c", 1);
+    conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".d", 2);
+
+    return conf;
+  }
+
+  @Test(timeout = 60000)
+  public void testSimpleSurgicalPreemptionOnAutoCreatedLeafQueues()
+      throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *                    C
+     *            /       |     \
+     *           USER1   USER2   USER3
+     *          30      30        30
+     * </pre>
+     *
+     * 1) Two nodes (n1/n2) in the cluster, each of them has 20G.
+     *
+     * 2) app1 submit to queue-USER1 first, it asked 32 * 1G containers
+     * We will allocate 16 on n1 and 16 on n2.
+     *
+     * 3) app2 submit to queue-USER2, ask for one 1G container (for AM)
+     *
+     * 4) app2 asks for another 6G container, it will be reserved on n1
+     *
+     * Now: we have:
+     * n1: 17 from app1, 1 from app2, and 1 reserved from app2
+     * n2: 16 from app1.
+     *
+     * After preemption, we should expect:
+     * Preempt 4 containers from app1 on n1.
+     */
+    setupQueueConfigurationForSimpleSurgicalPreemption(conf);
+    testSimpleSurgicalPreemption(USER1, USER2, USER1, USER2);
+  }
+
+  @Test(timeout = 600000)
+  public void
+      testPreemptionFromHighestPriorityManagedParentQueueAndOldestContainer()
+      throws Exception {
+    /**
+     * Test case: Submit two application (app1/app2) to different queues, queue
+     * structure:
+     *
+     * <pre>
+     *             Root
+     *            /  |  \
+     *           c   d   e
+     *          45  45  10
+     * </pre>
+     *
+     * Priority of queue_c = 1
+     * Priority of queue_d = 2
+     *
+     * 1) 5 nodes (n0-n4) in the cluster, each of them has 4G.
+     *
+     * 2) app1 submit to queue-e first (AM=1G), it asked 4 * 1G containers
+     *    We will allocate 1 container on each of n0-n4. AM on n4.
+     *
+     * 3) app2 submit to queue-c, AM container=0.5G, allocated on n0
+     *    Ask for 2 * 3.5G containers. (Reserved on n0/n1)
+     *
+     * 4) app2 submit to queue-d, AM container=0.5G, allocated on n2
+     *    Ask for 2 * 3.5G containers. (Reserved on n2/n3)
+     *
+     * First we will preempt container on n2 since it is the oldest container of
+     * Highest priority queue (d)
+     */
+
+    // Total preemption = 1G per round, which is 5% of cluster resource (20G)
+    setupQueueConfigurationForPriorityBasedPreemption(conf);
+    testPriorityPreemptionFromHighestPriorityQueueAndOldestContainer(
+        new String[] { USER1, USER3, USER0 },
+        new String[] { USER1, USER3, USER0 });
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/890d3d06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
index 8a7e03f..c20e091 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
+
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels
+    .RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -86,8 +89,19 @@ public class TestCapacitySchedulerSurgicalPreemption
      * After preemption, we should expect:
      * Preempt 4 containers from app1 on n1.
      */
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
+    testSimpleSurgicalPreemption("a", "c", "user", "user");
+  }
+
+  protected void testSimpleSurgicalPreemption(String queue1, String queue2,
+      String user1, String user2)
+      throws Exception {
+
+    MockRM rm1 = new MockRM(conf) {
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
     rm1.start();
 
     MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB);
@@ -97,7 +111,7 @@ public class TestCapacitySchedulerSurgicalPreemption
     RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
 
     // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a");
+    RMApp app1 = rm1.submitApp(1 * GB, "app", user1, null, queue1);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
 
     am1.allocate("*", 1 * GB, 32, new ArrayList<ContainerId>());
@@ -120,7 +134,7 @@ public class TestCapacitySchedulerSurgicalPreemption
 
 
     // Submit app2 to queue-c and asks for a 1G container for AM
-    RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    RMApp app2 = rm1.submitApp(1 * GB, "app", user2, null, queue2);
     MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
 
     // NM1/NM2 has available resource = 2G/4G
@@ -632,6 +646,21 @@ public class TestCapacitySchedulerSurgicalPreemption
      * Highest priority queue (b)
      */
 
+    // A/B has higher priority
+    conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".a" , 1);
+    conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".b", 2);
+    conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 45f);
+    conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".b", 45f);
+    conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".c", 10f);
+
+    testPriorityPreemptionFromHighestPriorityQueueAndOldestContainer(new
+        String[] {"a", "b", "c"}, new String[] {"user", "user", "user"});
+
+  }
+
+  protected void
+  testPriorityPreemptionFromHighestPriorityQueueAndOldestContainer(String[]
+      queues, String[] users) throws Exception {
     // Total preemption = 1G per round, which is 5% of cluster resource (20G)
     conf.setFloat(CapacitySchedulerConfiguration.TOTAL_PREEMPTION_PER_ROUND,
         0.05f);
@@ -641,15 +670,11 @@ public class TestCapacitySchedulerSurgicalPreemption
     conf.setQueueOrderingPolicy(CapacitySchedulerConfiguration.ROOT,
         CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY);
 
-    // A/B has higher priority
-    conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".a", 1);
-    conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".b", 2);
-    conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 45f);
-    conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".b", 45f);
-    conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".c", 10f);
-
-    MockRM rm1 = new MockRM(conf);
-    rm1.getRMContext().setNodeLabelManager(mgr);
+    MockRM rm1 = new MockRM(conf) {
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
     rm1.start();
 
     MockNM[] mockNMs = new MockNM[5];
@@ -665,7 +690,7 @@ public class TestCapacitySchedulerSurgicalPreemption
     }
 
     // launch an app to queue, AM container should be launched in nm1
-    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
+    RMApp app1 = rm1.submitApp(1 * GB, "app", users[2], null, queues[2]);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, mockNMs[4]);
 
     am1.allocate("*", 1 * GB, 4, new ArrayList<>());
@@ -685,7 +710,7 @@ public class TestCapacitySchedulerSurgicalPreemption
     }
 
     // Submit app2 to queue-a and asks for a 0.5G container for AM (on n0)
-    RMApp app2 = rm1.submitApp(512, "app", "user", null, "a");
+    RMApp app2 = rm1.submitApp(512, "app", users[0], null, queues[0]);
     MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, mockNMs[0]);
     FiCaSchedulerApp schedulerApp2 = cs.getApplicationAttempt(
         ApplicationAttemptId.newInstance(app2.getApplicationId(), 1));
@@ -703,11 +728,11 @@ public class TestCapacitySchedulerSurgicalPreemption
       Assert.assertNotNull("Should reserve on nm-" + i,
           cs.getNode(rmNodes[i].getNodeID()).getReservedContainer());
       Assert.assertEquals(cs.getNode(rmNodes[i].getNodeID())
-          .getReservedContainer().getQueueName(), "a");
+          .getReservedContainer().getQueueName(), queues[0]);
     }
 
     // Submit app3 to queue-b and asks for a 0.5G container for AM (on n2)
-    RMApp app3 = rm1.submitApp(512, "app", "user", null, "b");
+    RMApp app3 = rm1.submitApp(512, "app", users[1], null, queues[1]);
     MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, mockNMs[2]);
     FiCaSchedulerApp schedulerApp3 = cs.getApplicationAttempt(
         ApplicationAttemptId.newInstance(app3.getApplicationId(), 1));
@@ -725,7 +750,7 @@ public class TestCapacitySchedulerSurgicalPreemption
       Assert.assertNotNull("Should reserve on nm-" + i,
           cs.getNode(rmNodes[i].getNodeID()).getReservedContainer());
       Assert.assertEquals(cs.getNode(rmNodes[i].getNodeID())
-          .getReservedContainer().getQueueName(), "b");
+          .getReservedContainer().getQueueName(), queues[1]);
     }
 
     // Sleep the timeout interval, we should be able to see 1 container selected
@@ -831,6 +856,7 @@ public class TestCapacitySchedulerSurgicalPreemption
     rm1.close();
   }
 
+
   @Test(timeout = 60000)
   public void testPreemptionForFragmentatedCluster() throws Exception {
     // Set additional_balance_queue_based_on_reserved_res to true to get


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


[14/50] [abbrv] hadoop git commit: HDFS-10706. [READ] Add tool generating FSImage from external store

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java
new file mode 100644
index 0000000..9aef106
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.junit.Assert.*;
+
+/**
+ * Validate resolver assigning all paths to a single owner/group.
+ */
+public class TestSingleUGIResolver {
+
+  @Rule public TestName name = new TestName();
+
+  private static final int TESTUID = 10101;
+  private static final int TESTGID = 10102;
+  private static final String TESTUSER = "tenaqvyybdhragqvatbf";
+  private static final String TESTGROUP = "tnyybcvatlnxf";
+
+  private SingleUGIResolver ugi = new SingleUGIResolver();
+
+  @Before
+  public void setup() {
+    Configuration conf = new Configuration(false);
+    conf.setInt(SingleUGIResolver.UID, TESTUID);
+    conf.setInt(SingleUGIResolver.GID, TESTGID);
+    conf.set(SingleUGIResolver.USER, TESTUSER);
+    conf.set(SingleUGIResolver.GROUP, TESTGROUP);
+    ugi.setConf(conf);
+    System.out.println(name.getMethodName());
+  }
+
+  @Test
+  public void testRewrite() {
+    FsPermission p1 = new FsPermission((short)0755);
+    match(ugi.resolve(file("dingo", "dingo", p1)), p1);
+    match(ugi.resolve(file(TESTUSER, "dingo", p1)), p1);
+    match(ugi.resolve(file("dingo", TESTGROUP, p1)), p1);
+    match(ugi.resolve(file(TESTUSER, TESTGROUP, p1)), p1);
+
+    FsPermission p2 = new FsPermission((short)0x8000);
+    match(ugi.resolve(file("dingo", "dingo", p2)), p2);
+    match(ugi.resolve(file(TESTUSER, "dingo", p2)), p2);
+    match(ugi.resolve(file("dingo", TESTGROUP, p2)), p2);
+    match(ugi.resolve(file(TESTUSER, TESTGROUP, p2)), p2);
+
+    Map<Integer, String> ids = ugi.ugiMap();
+    assertEquals(2, ids.size());
+    assertEquals(TESTUSER, ids.get(10101));
+    assertEquals(TESTGROUP, ids.get(10102));
+  }
+
+  @Test
+  public void testDefault() {
+    String user;
+    try {
+      user = UserGroupInformation.getCurrentUser().getShortUserName();
+    } catch (IOException e) {
+      user = "hadoop";
+    }
+    Configuration conf = new Configuration(false);
+    ugi.setConf(conf);
+    Map<Integer, String> ids = ugi.ugiMap();
+    assertEquals(2, ids.size());
+    assertEquals(user, ids.get(0));
+    assertEquals(user, ids.get(1));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testInvalidUid() {
+    Configuration conf = ugi.getConf();
+    conf.setInt(SingleUGIResolver.UID, (1 << 24) + 1);
+    ugi.setConf(conf);
+    ugi.resolve(file(TESTUSER, TESTGROUP, new FsPermission((short)0777)));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testInvalidGid() {
+    Configuration conf = ugi.getConf();
+    conf.setInt(SingleUGIResolver.GID, (1 << 24) + 1);
+    ugi.setConf(conf);
+    ugi.resolve(file(TESTUSER, TESTGROUP, new FsPermission((short)0777)));
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testDuplicateIds() {
+    Configuration conf = new Configuration(false);
+    conf.setInt(SingleUGIResolver.UID, 4344);
+    conf.setInt(SingleUGIResolver.GID, 4344);
+    conf.set(SingleUGIResolver.USER, TESTUSER);
+    conf.set(SingleUGIResolver.GROUP, TESTGROUP);
+    ugi.setConf(conf);
+    ugi.ugiMap();
+  }
+
+  static void match(long encoded, FsPermission p) {
+    assertEquals(p, new FsPermission((short)(encoded & 0xFFFF)));
+    long uid = (encoded >>> UGIResolver.USER_STRID_OFFSET);
+    uid &= UGIResolver.USER_GROUP_STRID_MASK;
+    assertEquals(TESTUID, uid);
+    long gid = (encoded >>> UGIResolver.GROUP_STRID_OFFSET);
+    gid &= UGIResolver.USER_GROUP_STRID_MASK;
+    assertEquals(TESTGID, gid);
+  }
+
+  static FileStatus file(String user, String group, FsPermission perm) {
+    Path p = new Path("foo://bar:4344/baz/dingo");
+    return new FileStatus(
+          4344 * (1 << 20),        /* long length,             */
+          false,                   /* boolean isdir,           */
+          1,                       /* int block_replication,   */
+          256 * (1 << 20),         /* long blocksize,          */
+          0L,                      /* long modification_time,  */
+          0L,                      /* long access_time,        */
+          perm,                    /* FsPermission permission, */
+          user,                    /* String owner,            */
+          group,                   /* String group,            */
+          p);                      /* Path path                */
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/resources/log4j.properties b/hadoop-tools/hadoop-fs2img/src/test/resources/log4j.properties
new file mode 100644
index 0000000..2ebf29e
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+#
+#   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.
+#
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F:%M(%L)) - %m%n
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index 28faa9f..4b90361 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -128,6 +128,12 @@
       <scope>compile</scope>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-fs2img</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index 6f95f11..c030045 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -48,6 +48,7 @@
     <module>hadoop-kafka</module>
     <module>hadoop-azure-datalake</module>
     <module>hadoop-aliyun</module>
+    <module>hadoop-fs2img</module>
   </modules>
 
   <build>


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


[10/50] [abbrv] hadoop git commit: HDFS-11663. [READ] Fix NullPointerException in ProvidedBlocksBuilder

Posted by vi...@apache.org.
HDFS-11663. [READ] Fix NullPointerException in ProvidedBlocksBuilder


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

Branch: refs/heads/HDFS-9806
Commit: 08f6b2daf4a24c5b17132e6d582b27aefe32a26c
Parents: 081e0c0
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu May 4 13:06:53 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

----------------------------------------------------------------------
 .../blockmanagement/ProvidedStorageMap.java     | 40 ++++++-----
 .../TestNameNodeProvidedImplementation.java     | 70 +++++++++++++++-----
 2 files changed, 77 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08f6b2da/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index d222344..518b7e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -134,11 +134,13 @@ public class ProvidedStorageMap {
   class ProvidedBlocksBuilder extends LocatedBlockBuilder {
 
     private ShadowDatanodeInfoWithStorage pending;
+    private boolean hasProvidedLocations;
 
     ProvidedBlocksBuilder(int maxBlocks) {
       super(maxBlocks);
       pending = new ShadowDatanodeInfoWithStorage(
           providedDescriptor, storageId);
+      hasProvidedLocations = false;
     }
 
     @Override
@@ -154,6 +156,7 @@ public class ProvidedStorageMap {
         types[i] = storages[i].getStorageType();
         if (StorageType.PROVIDED.equals(storages[i].getStorageType())) {
           locs[i] = pending;
+          hasProvidedLocations = true;
         } else {
           locs[i] = new DatanodeInfoWithStorage(
               storages[i].getDatanodeDescriptor(), sids[i], types[i]);
@@ -165,25 +168,28 @@ public class ProvidedStorageMap {
     @Override
     LocatedBlocks build(DatanodeDescriptor client) {
       // TODO: to support multiple provided storages, need to pass/maintain map
-      // set all fields of pending DatanodeInfo
-      List<String> excludedUUids = new ArrayList<String>();
-      for (LocatedBlock b: blocks) {
-        DatanodeInfo[] infos = b.getLocations();
-        StorageType[] types = b.getStorageTypes();
-
-        for (int i = 0; i < types.length; i++) {
-          if (!StorageType.PROVIDED.equals(types[i])) {
-            excludedUUids.add(infos[i].getDatanodeUuid());
+      if (hasProvidedLocations) {
+        // set all fields of pending DatanodeInfo
+        List<String> excludedUUids = new ArrayList<String>();
+        for (LocatedBlock b : blocks) {
+          DatanodeInfo[] infos = b.getLocations();
+          StorageType[] types = b.getStorageTypes();
+
+          for (int i = 0; i < types.length; i++) {
+            if (!StorageType.PROVIDED.equals(types[i])) {
+              excludedUUids.add(infos[i].getDatanodeUuid());
+            }
           }
         }
-      }
 
-      DatanodeDescriptor dn = providedDescriptor.choose(client, excludedUUids);
-      if (dn == null) {
-        dn = providedDescriptor.choose(client);
+        DatanodeDescriptor dn =
+                providedDescriptor.choose(client, excludedUUids);
+        if (dn == null) {
+          dn = providedDescriptor.choose(client);
+        }
+        pending.replaceInternal(dn);
       }
 
-      pending.replaceInternal(dn);
       return new LocatedBlocks(
           flen, isUC, blocks, last, lastComplete, feInfo, ecPolicy);
     }
@@ -278,7 +284,8 @@ public class ProvidedStorageMap {
 
     DatanodeDescriptor choose(DatanodeDescriptor client) {
       // exact match for now
-      DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
+      DatanodeDescriptor dn = client != null ?
+              dns.get(client.getDatanodeUuid()) : null;
       if (null == dn) {
         dn = chooseRandom();
       }
@@ -288,7 +295,8 @@ public class ProvidedStorageMap {
     DatanodeDescriptor choose(DatanodeDescriptor client,
         List<String> excludedUUids) {
       // exact match for now
-      DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
+      DatanodeDescriptor dn = client != null ?
+              dns.get(client.getDatanodeUuid()) : null;
 
       if (null == dn || excludedUUids.contains(client.getDatanodeUuid())) {
         dn = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08f6b2da/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 3b75806..5062439 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
@@ -69,6 +70,10 @@ public class TestNameNodeProvidedImplementation {
   final Path BLOCKFILE = new Path(NNDIRPATH, "blocks.csv");
   final String SINGLEUSER = "usr1";
   final String SINGLEGROUP = "grp1";
+  private final int numFiles = 10;
+  private final String filePrefix = "file";
+  private final String fileSuffix = ".dat";
+  private final int baseFileLen = 1024;
 
   Configuration conf;
   MiniDFSCluster cluster;
@@ -114,15 +119,16 @@ public class TestNameNodeProvidedImplementation {
     }
 
     // create 10 random files under BASE
-    for (int i=0; i < 10; i++) {
-      File newFile = new File(new Path(NAMEPATH, "file" + i).toUri());
+    for (int i=0; i < numFiles; i++) {
+      File newFile = new File(
+          new Path(NAMEPATH, filePrefix + i + fileSuffix).toUri());
       if(!newFile.exists()) {
         try {
           LOG.info("Creating " + newFile.toString());
           newFile.createNewFile();
           Writer writer = new OutputStreamWriter(
               new FileOutputStream(newFile.getAbsolutePath()), "utf-8");
-          for(int j=0; j < 10*i; j++) {
+          for(int j=0; j < baseFileLen*i; j++) {
             writer.write("0");
           }
           writer.flush();
@@ -161,29 +167,30 @@ public class TestNameNodeProvidedImplementation {
 
   void startCluster(Path nspath, int numDatanodes,
       StorageType[] storageTypes,
-      StorageType[][] storageTypesPerDatanode)
+      StorageType[][] storageTypesPerDatanode,
+      boolean doFormat)
       throws IOException {
     conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString());
 
     if (storageTypesPerDatanode != null) {
       cluster = new MiniDFSCluster.Builder(conf)
-          .format(false)
-          .manageNameDfsDirs(false)
+          .format(doFormat)
+          .manageNameDfsDirs(doFormat)
           .numDataNodes(numDatanodes)
           .storageTypes(storageTypesPerDatanode)
           .build();
     } else if (storageTypes != null) {
       cluster = new MiniDFSCluster.Builder(conf)
-          .format(false)
-          .manageNameDfsDirs(false)
+          .format(doFormat)
+          .manageNameDfsDirs(doFormat)
           .numDataNodes(numDatanodes)
           .storagesPerDatanode(storageTypes.length)
           .storageTypes(storageTypes)
           .build();
     } else {
       cluster = new MiniDFSCluster.Builder(conf)
-          .format(false)
-          .manageNameDfsDirs(false)
+          .format(doFormat)
+          .manageNameDfsDirs(doFormat)
           .numDataNodes(numDatanodes)
           .build();
     }
@@ -195,7 +202,8 @@ public class TestNameNodeProvidedImplementation {
     final long seed = r.nextLong();
     LOG.info("NAMEPATH: " + NAMEPATH);
     createImage(new RandomTreeWalk(seed), NNDIRPATH, FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 0, new StorageType[] {StorageType.PROVIDED}, null);
+    startCluster(NNDIRPATH, 0, new StorageType[] {StorageType.PROVIDED},
+        null, false);
 
     FileSystem fs = cluster.getFileSystem();
     for (TreePath e : new RandomTreeWalk(seed)) {
@@ -220,7 +228,8 @@ public class TestNameNodeProvidedImplementation {
         SingleUGIResolver.class, UGIResolver.class);
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 1, new StorageType[] {StorageType.PROVIDED}, null);
+    startCluster(NNDIRPATH, 1, new StorageType[] {StorageType.PROVIDED},
+        null, false);
   }
 
   @Test(timeout=500000)
@@ -232,10 +241,10 @@ public class TestNameNodeProvidedImplementation {
     // make the last Datanode with only DISK
     startCluster(NNDIRPATH, 3, null,
         new StorageType[][] {
-          {StorageType.PROVIDED},
-          {StorageType.PROVIDED},
-          {StorageType.DISK}}
-        );
+            {StorageType.PROVIDED},
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
+        false);
     // wait for the replication to finish
     Thread.sleep(50000);
 
@@ -290,7 +299,8 @@ public class TestNameNodeProvidedImplementation {
         FsUGIResolver.class, UGIResolver.class);
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3, new StorageType[] {StorageType.PROVIDED}, null);
+    startCluster(NNDIRPATH, 3, new StorageType[] {StorageType.PROVIDED},
+        null, false);
     FileSystem fs = cluster.getFileSystem();
     Thread.sleep(2000);
     int count = 0;
@@ -342,4 +352,30 @@ public class TestNameNodeProvidedImplementation {
       }
     }
   }
+
+  private BlockLocation[] createFile(Path path, short replication,
+      long fileLen, long blockLen) throws IOException {
+    FileSystem fs = cluster.getFileSystem();
+    //create a sample file that is not provided
+    DFSTestUtil.createFile(fs, path, false, (int) blockLen,
+        fileLen, blockLen, replication, 0, true);
+    return fs.getFileBlockLocations(path, 0, fileLen);
+  }
+
+  @Test
+  public void testClusterWithEmptyImage() throws IOException {
+    // start a cluster with 2 datanodes without any provided storage
+    startCluster(NNDIRPATH, 2, null,
+        new StorageType[][] {
+            {StorageType.DISK},
+            {StorageType.DISK}},
+        true);
+    assertTrue(cluster.isClusterUp());
+    assertTrue(cluster.isDataNodeUp());
+
+    BlockLocation[] locations = createFile(new Path("/testFile1.dat"),
+        (short) 2, 1024*1024, 1024*1024);
+    assertEquals(1, locations.length);
+    assertEquals(2, locations[0].getHosts().length);
+  }
 }


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 298fda275dc089a0ad1755f25446ff0a7cc376e4
Parents: 3b61069
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Dec 8 10:07:40 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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

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


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: c1bf2654b0e9118985b8518b0254eac4dd302a2f
Parents: eaf5f99
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Dec 7 14:21:24 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 6ba860238640d3615f1f473ff4c4668797bfb36c
Parents: 27ce813
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Dec 6 09:39:56 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:34 2017 -0800

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


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

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

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

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

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

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

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

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

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

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

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

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


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


[19/50] [abbrv] hadoop git commit: HDFS-11792. [READ] Test cases for ProvidedVolumeDF and ProviderBlockIteratorImpl

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


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

Branch: refs/heads/HDFS-9806
Commit: 8c3745db48df66e55ac617ede31f566f9f2fad74
Parents: 8ab6232
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed May 31 15:17:12 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c3745db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index a48e117..421b9cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -191,7 +191,11 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
 
   @Override
   long getBlockPoolUsed(String bpid) throws IOException {
-    return df.getBlockPoolUsed(bpid);
+    if (bpSlices.containsKey(bpid)) {
+      return df.getBlockPoolUsed(bpid);
+    } else {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
   }
 
   @Override

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


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: b0948868b805048dcd52349e805c2f209f6598a1
Parents: 68e046c
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Nov 27 17:04:20 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0948868/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 9c82967..09e8f97 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -27,8 +27,11 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Random;
+import java.util.Set;
+
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -480,16 +483,31 @@ public class TestNameNodeProvidedImplementation {
     // given the start and length in the above call,
     // only one LocatedBlock in LocatedBlocks
     assertEquals(expectedBlocks, locatedBlocks.getLocatedBlocks().size());
-    LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
-    assertEquals(expectedLocations, locatedBlock.getLocations().length);
-    return locatedBlock.getLocations();
+    DatanodeInfo[] locations =
+        locatedBlocks.getLocatedBlocks().get(0).getLocations();
+    assertEquals(expectedLocations, locations.length);
+    checkUniqueness(locations);
+    return locations;
+  }
+
+  /**
+   * verify that the given locations are all unique.
+   * @param locations
+   */
+  private void checkUniqueness(DatanodeInfo[] locations) {
+    Set<String> set = new HashSet<>();
+    for (DatanodeInfo info: locations) {
+      assertFalse("All locations should be unique",
+          set.contains(info.getDatanodeUuid()));
+      set.add(info.getDatanodeUuid());
+    }
   }
 
   /**
    * Tests setting replication of provided files.
    * @throws Exception
    */
-  @Test(timeout=30000)
+  @Test(timeout=50000)
   public void testSetReplicationForProvidedFiles() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);


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


[26/50] [abbrv] hadoop git commit: HDFS-12671. [READ] Test NameNode restarts when PROVIDED is configured

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


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

Branch: refs/heads/HDFS-9806
Commit: 78875012a5d16c613811a200f85bba06f302eb69
Parents: 84ea081
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Nov 7 12:54:27 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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


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


[34/50] [abbrv] hadoop git commit: HDFS-12789. [READ] Image generation tool does not close an opened stream

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


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

Branch: refs/heads/HDFS-9806
Commit: 94a7b68187e4bad9eb004dccdf051af711099f59
Parents: 7887501
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Nov 8 10:28:50 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a7b681/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
index ea1888a..390bb39 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
@@ -165,16 +165,23 @@ public class ImageWriter implements Closeable {
 
     // create directory and inode sections as side-files.
     // The details are written to files to avoid keeping them in memory.
-    dirsTmp = File.createTempFile("fsimg_dir", null);
-    dirsTmp.deleteOnExit();
-    dirs = beginSection(new FileOutputStream(dirsTmp));
+    FileOutputStream dirsTmpStream = null;
+    try {
+      dirsTmp = File.createTempFile("fsimg_dir", null);
+      dirsTmp.deleteOnExit();
+      dirsTmpStream = new FileOutputStream(dirsTmp);
+      dirs = beginSection(dirsTmpStream);
+    } catch (IOException e) {
+      IOUtils.cleanupWithLogger(null, raw, dirsTmpStream);
+      throw e;
+    }
+
     try {
       inodesTmp = File.createTempFile("fsimg_inode", null);
       inodesTmp.deleteOnExit();
       inodes = new FileOutputStream(inodesTmp);
     } catch (IOException e) {
-      // appropriate to close raw?
-      IOUtils.cleanup(null, raw, dirs);
+      IOUtils.cleanupWithLogger(null, raw, dirsTmpStream, dirs);
       throw e;
     }
 


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


[18/50] [abbrv] hadoop git commit: HDFS-11673. [READ] Handle failures of Datanode with PROVIDED storage

Posted by vi...@apache.org.
HDFS-11673. [READ] Handle failures of Datanode with PROVIDED storage


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

Branch: refs/heads/HDFS-9806
Commit: 6759d9fc495ac312d3dc94994daae4dcd3d92910
Parents: 8c3745d
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Jun 1 16:01:31 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6759d9fc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 0e3eab3..07502c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1514,6 +1514,7 @@ public class BlockManager implements BlockStatsMXBean {
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
+    providedStorageMap.removeDatanode(node);
     for (DatanodeStorageInfo storage : node.getStorageInfos()) {
       final Iterator<BlockInfo> it = storage.getBlockIterator();
       //add the BlockInfos to a new collection as the
@@ -2462,7 +2463,7 @@ public class BlockManager implements BlockStatsMXBean {
       // !#! Register DN with provided storage, not with storage owned by DN
       // !#! DN should still have a ref to the DNStorageInfo
       DatanodeStorageInfo storageInfo =
-          providedStorageMap.getStorage(node, storage);
+          providedStorageMap.getStorage(node, storage, context);
 
       if (storageInfo == null) {
         // We handle this for backwards compatibility.
@@ -2589,7 +2590,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
   
-  private Collection<Block> processReport(
+  Collection<Block> processReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report,
       BlockReportContext context) throws IOException {

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6759d9fc/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index e171557..60b306f 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -45,11 +45,14 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
 import org.apache.hadoop.hdfs.server.common.BlockFormat;
 import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
 import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
 import org.apache.hadoop.hdfs.server.common.TextFileRegionProvider;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 
 import org.junit.After;
@@ -406,9 +409,9 @@ public class TestNameNodeProvidedImplementation {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
     startCluster(NNDIRPATH, 2, null,
-        new StorageType[][] {
-                {StorageType.PROVIDED},
-                {StorageType.DISK}},
+        new StorageType[][]{
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
         false);
 
     String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
@@ -433,4 +436,67 @@ public class TestNameNodeProvidedImplementation {
     assertEquals(cluster.getDataNodes().get(0).getDatanodeUuid(),
         infos[0].getDatanodeUuid());
   }
+
+  @Test
+  public void testProvidedDatanodeFailures() throws Exception {
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+            FixedBlockResolver.class);
+    startCluster(NNDIRPATH, 3, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED},
+            {StorageType.PROVIDED},
+            {StorageType.DISK}},
+        false);
+
+    DataNode providedDatanode1 = cluster.getDataNodes().get(0);
+    DataNode providedDatanode2 = cluster.getDataNodes().get(1);
+
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+            cluster.getNameNodePort()), cluster.getConfiguration(0));
+
+    if (numFiles >= 1) {
+      String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
+
+      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      //the location should be one of the provided DNs available
+      assertTrue(
+          dnInfos[0].getDatanodeUuid().equals(
+              providedDatanode1.getDatanodeUuid())
+          || dnInfos[0].getDatanodeUuid().equals(
+              providedDatanode2.getDatanodeUuid()));
+
+      //stop the 1st provided datanode
+      MiniDFSCluster.DataNodeProperties providedDNProperties1 =
+          cluster.stopDataNode(0);
+
+      //make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode1.getDatanodeId().getXferAddr());
+
+      //should find the block on the 2nd provided datanode
+      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      assertEquals(providedDatanode2.getDatanodeUuid(),
+          dnInfos[0].getDatanodeUuid());
+
+      //stop the 2nd provided datanode
+      cluster.stopDataNode(1);
+      // make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode2.getDatanodeId().getXferAddr());
+
+      getAndCheckBlockLocations(client, filename, 0);
+
+      //restart the provided datanode
+      cluster.restartDataNode(providedDNProperties1, true);
+      cluster.waitActive();
+
+      //should find the block on the 1st provided datanode now
+      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      //not comparing UUIDs as the datanode can now have a different one.
+      assertEquals(providedDatanode1.getDatanodeId().getXferAddr(),
+          dnInfos[0].getXferAddr());
+    }
+  }
 }


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


[20/50] [abbrv] hadoop git commit: HDFS-12091. [READ] Check that the replicas served from a ProvidedVolumeImpl belong to the correct external storage

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


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

Branch: refs/heads/HDFS-9806
Commit: 3abe22e85b36d58d06fa5ac757ed8cb3bada7b65
Parents: 6759d9f
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Aug 7 11:35:49 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abe22e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
index 4753235..8782e71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -31,6 +31,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
@@ -174,15 +176,26 @@ public class TestProvidedImpl {
     private Configuration conf;
     private int minId;
     private int numBlocks;
+    private Iterator<FileRegion> suppliedIterator;
 
     TestFileRegionProvider() {
-      minId = MIN_BLK_ID;
-      numBlocks = NUM_PROVIDED_BLKS;
+      this(null, MIN_BLK_ID, NUM_PROVIDED_BLKS);
+    }
+
+    TestFileRegionProvider(Iterator<FileRegion> iterator, int minId,
+        int numBlocks) {
+      this.suppliedIterator = iterator;
+      this.minId = minId;
+      this.numBlocks = numBlocks;
     }
 
     @Override
     public Iterator<FileRegion> iterator() {
-      return new TestFileRegionIterator(providedBasePath, minId, numBlocks);
+      if (suppliedIterator == null) {
+        return new TestFileRegionIterator(providedBasePath, minId, numBlocks);
+      } else {
+        return suppliedIterator;
+      }
     }
 
     @Override
@@ -503,4 +516,90 @@ public class TestProvidedImpl {
       }
     }
   }
+
+  private int getBlocksInProvidedVolumes(String basePath, int numBlocks,
+      int minBlockId) throws IOException {
+    TestFileRegionIterator fileRegionIterator =
+        new TestFileRegionIterator(basePath, minBlockId, numBlocks);
+    int totalBlocks = 0;
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
+      vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID],
+          new TestFileRegionProvider(fileRegionIterator, minBlockId,
+              numBlocks));
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null);
+      totalBlocks += volumeMap.size(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+    }
+    return totalBlocks;
+  }
+
+  /**
+   * Tests if the FileRegions provided by the FileRegionProvider
+   * can belong to the Providevolume.
+   * @throws IOException
+   */
+  @Test
+  public void testProvidedVolumeContents() throws IOException {
+    int expectedBlocks = 5;
+    int minId = 0;
+    //use a path which has the same prefix as providedBasePath
+    //all these blocks can belong to the provided volume
+    int blocksFound = getBlocksInProvidedVolumes(providedBasePath + "/test1/",
+        expectedBlocks, minId);
+    assertEquals(
+        "Number of blocks in provided volumes should be " + expectedBlocks,
+        expectedBlocks, blocksFound);
+    blocksFound = getBlocksInProvidedVolumes(
+        "file:/" + providedBasePath + "/test1/", expectedBlocks, minId);
+    assertEquals(
+        "Number of blocks in provided volumes should be " + expectedBlocks,
+        expectedBlocks, blocksFound);
+    //use a path that is entirely different from the providedBasePath
+    //none of these blocks can belong to the volume
+    blocksFound =
+        getBlocksInProvidedVolumes("randomtest1/", expectedBlocks, minId);
+    assertEquals("Number of blocks in provided volumes should be 0", 0,
+        blocksFound);
+  }
+
+  @Test
+  public void testProvidedVolumeContainsBlock() throws URISyntaxException {
+    assertEquals(true, ProvidedVolumeImpl.containsBlock(null, null));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a"), null));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/c/"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c/"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c/"),
+            new URI("/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/c/"),
+            new URI("/a/b/c/d/e.file")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/e"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/e"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket1/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket2/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket1/dir1/"),
+            new URI("s3a:/bucket1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("/bucket1/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+  }
 }


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: eaf5f99bc45fb5871cab2df5ec4acbb03d274b4b
Parents: 6ba8602
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Dec 6 09:42:31 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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


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


[06/50] [abbrv] hadoop git commit: YARN-7633. Documentation for auto queue creation feature and related configurations. Contributed by Suma Shivaprasad.

Posted by vi...@apache.org.
YARN-7633. Documentation for auto queue creation feature and related configurations. Contributed by Suma Shivaprasad.


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

Branch: refs/heads/HDFS-9806
Commit: 6681dd10075f732a99e0e1f980368fc58ba45c68
Parents: 09d996f
Author: Sunil G <su...@apache.org>
Authored: Fri Dec 15 23:06:53 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Dec 15 23:07:15 2017 +0530

----------------------------------------------------------------------
 .../src/site/markdown/CapacityScheduler.md      | 88 ++++++++++++++++++++
 1 file changed, 88 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6681dd10/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index 86d9167..2598997 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -66,6 +66,8 @@ The `CapacityScheduler` supports the following features:
 
 * **Absolute Resource Configuration** - Administrators could specify absolute resources to a queue instead of providing percentage based values. This provides better control for admins to configure required amount of resources for a given queue.
 
+* **Dynamic Auto-Creation and Management of Leaf Queues** - This feature supports auto-creation of **leaf queues** in conjunction with **queue-mapping** which currently supports **user-group** based queue mappings for application placement to a queue. The scheduler also supports capacity management for these queues based on a policy configured on the parent queue.
+
 Configuration
 -------------
 
@@ -274,6 +276,92 @@ The `ReservationSystem` is integrated with the `CapacityScheduler` queue hierach
 | `yarn.scheduler.capacity.<queue-path>.reservation-planner` | *Optional* parameter: the class name that will be used to determine the implementation of the *Planner*  which will be invoked if the `Plan` capacity fall below (due to scheduled maintenance or node failuers) the user reserved resources. The default value is *org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.SimpleCapacityReplanner* which scans the `Plan` and greedily removes reservations in reversed order of acceptance (LIFO) till the reserved resources are within the `Plan` capacity |
 | `yarn.scheduler.capacity.<queue-path>.reservation-enforcement-window` | *Optional* parameter representing the time in milliseconds for which the `Planner` will validate if the constraints in the Plan are satisfied. Long value expected. The default value is one hour. |
 
+###Dynamic Auto-Creation and Management of Leaf Queues
+
+The `CapacityScheduler` supports auto-creation of **leaf queues** under parent queues which have been configured to enable this feature.
+
+  * Setup for dynamic auto-created leaf queues through queue mapping
+
+  **user-group queue mapping(s)** listed in `yarn.scheduler.capacity.queue-mappings` need to specify an additional parent queue parameter to
+  identify which parent queue the auto-created leaf queues need to be created
+   under. Refer above `Queue Mapping based on User or Group` section for more
+    details. Please note that such parent queues also need to enable
+    auto-creation of child queues as mentioned in `Parent queue configuration
+     for dynamic leaf queue creation and management` section below
+
+Example:
+
+```
+ <property>
+   <name>yarn.scheduler.capacity.queue-mappings</name>
+   <value>u:user1:queue1,g:group1:queue2,u:user2:%primary_group,u:%user:parent1.%user</value>
+   <description>
+     Here, u:%user:parent1.%user mapping allows any <user> other than user1,
+     user2 to be mapped to its own user specific leaf queue which
+     will be auto-created under <parent1>.
+   </description>
+ </property>
+```
+
+ * Parent queue configuration for dynamic leaf queue auto-creation and management
+
+The `Dynamic Queue Auto-Creation and Management` feature is integrated with the
+`CapacityScheduler` queue hierarchy and can be configured for a **ParentQueue** currently to auto-create leaf queues. Such parent queues do not
+support other pre-configured queues to co-exist along with auto-created queues. The `CapacityScheduler` supports the following parameters to enable auto-creation of queues
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.<queue-path>.auto-create-child-queue.enabled` | *Mandatory* parameter: Indicates to the `CapacityScheduler` that auto leaf queue creation needs to be enabled for the specified parent queue.  Boolean value expected. The default value is *false*, i.e. auto leaf queue creation is not enabled in *ParentQueue* by default. |
+| `yarn.scheduler.capacity.<queue-path>.auto-create-child-queue.management-policy` | *Optional* parameter: the class name that will be used to determine the implementation of the `AutoCreatedQueueManagementPolicy`  which will manage leaf queues and their capacities dynamically under this parent queue. The default value is *org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy*. Users or groups might submit applications to the auto-created leaf queues for a limited time and stop using them. Hence there could be more number of leaf queues auto-created under the parent queue than its guaranteed capacity. The current policy implementation allots either configured or zero capacity on a **best-effort** basis based on availability of capacity on the parent queue and the application submission order across leaf queues. |
+
+
+* Configuring `Auto-Created Leaf Queues` with `CapacityScheduler`
+
+The parent queue which has been enabled for auto leaf queue creation,supports
+ the configuration of template parameters for automatic configuration of the auto-created leaf queues. The auto-created queues support all of the
+ leaf queue configuration parameters except for **Queue ACL**, **Absolute
+ Resource** configurations and **Node Labels**. Queue ACLs and Node Labels are
+ currently inherited from the parent queue i.e they are not configurable on the leaf queue template
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.<queue-path>.leaf-queue-template.capacity` | *Mandatory* parameter: Specifies the minimum guaranteed capacity for the  auto-created leaf queues. Currently *Absolute Resource* configurations are not supported on auto-created leaf queues |
+| `yarn.scheduler.capacity.<queue-path>.leaf-queue-template.<leaf-queue-property>` |  *Optional* parameter: For other queue parameters that can be configured on auto-created leaf queues like maximum-capacity, user-limit-factor, maximum-am-resource-percent ...  - Refer **Queue Properties** section |
+
+Example:
+
+```
+ <property>
+   <name>yarn.scheduler.capacity.root.parent1.auto-create-child-queue.enabled</name>
+   <value>true</value>
+ </property>
+ <property>
+    <name>yarn.scheduler.capacity.root.parent1.leaf-queue-template.capacity</name>
+    <value>5</value>
+ </property>
+ <property>
+    <name>yarn.scheduler.capacity.root.parent1.leaf-queue-template.maximum-capacity</name>
+    <value>100</value>
+ </property>
+ <property>
+    <name>yarn.scheduler.capacity.root.parent1.leaf-queue-template.user-limit-factor</name>
+    <value>3.0</value>
+ </property>
+ <property>
+    <name>yarn.scheduler.capacity.root.parent1.leaf-queue-template.ordering-policy</name>
+    <value>fair</value>
+ </property>
+```
+
+* Scheduling Edit Policy configuration for auto-created queue management
+
+Admins need to specify an additional `org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueManagementDynamicEditPolicy` scheduling edit policy to the
+list of current scheduling edit policies as a comma separated string in `yarn.resourcemanager.scheduler.monitor.policies` configuration. For more details, refer `Capacity Scheduler container preemption` section above
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.resourcemanager.monitor.capacity.queue-management.monitoring-interval` | Time in milliseconds between invocations of this QueueManagementDynamicEditPolicy policy. Default value is 1500 |
+
 ###Other Properties
 
   * Resource Calculator


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: a8b1c7dc135bd5b9076f1b8c9fe8b5da352bc140
Parents: e075a61
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Dec 13 13:39:21 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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

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

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

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

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

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

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

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

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


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


[23/50] [abbrv] hadoop git commit: HDFS-11902. [READ] Merge BlockFormatProvider and FileRegionProvider.

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
index e1e85c1..2e57c9f 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
@@ -29,7 +29,7 @@ import org.apache.commons.cli.PosixParser;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -103,7 +103,7 @@ public class FileSystemImage implements Tool {
         break;
       case "b":
         opts.blocks(
-            Class.forName(o.getValue()).asSubclass(BlockFormat.class));
+            Class.forName(o.getValue()).asSubclass(BlockAliasMap.class));
         break;
       case "i":
         opts.blockIds(

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
index 14e6bed..d327363 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
@@ -24,8 +24,8 @@ import com.google.protobuf.ByteString;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
@@ -70,7 +70,7 @@ public class TreePath {
   }
 
   public INode toINode(UGIResolver ugi, BlockResolver blk,
-      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+                       BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
           throws IOException {
     if (stat.isFile()) {
       return toFile(ugi, blk, out, blockPoolID);
@@ -101,14 +101,14 @@ public class TreePath {
 
   void writeBlock(long blockId, long offset, long length,
       long genStamp, String blockPoolID,
-      BlockFormat.Writer<FileRegion> out) throws IOException {
+      BlockAliasMap.Writer<FileRegion> out) throws IOException {
     FileStatus s = getFileStatus();
     out.store(new FileRegion(blockId, s.getPath(), offset, length,
         blockPoolID, genStamp));
   }
 
   INode toFile(UGIResolver ugi, BlockResolver blk,
-      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+               BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
           throws IOException {
     final FileStatus s = getFileStatus();
     // TODO should this store resolver's user/group?

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index d622b9e..2170baa 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -44,13 +44,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
-import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionProvider;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
@@ -103,18 +99,13 @@ public class TestNameNodeProvidedImplementation {
         DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
 
-    conf.setClass(DFSConfigKeys.DFS_NAMENODE_BLOCK_PROVIDER_CLASS,
-        BlockFormatProvider.class, BlockProvider.class);
-    conf.setClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
-        TextFileRegionProvider.class, FileRegionProvider.class);
-    conf.setClass(DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-        TextFileRegionFormat.class, BlockFormat.class);
-
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_WRITE_PATH,
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TextFileRegionAliasMap.class, BlockAliasMap.class);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH,
         BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
         BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER, ",");
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, ",");
 
     conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
         new File(NAMEPATH.toUri()).toString());
@@ -167,7 +158,7 @@ public class TestNameNodeProvidedImplementation {
     ImageWriter.Options opts = ImageWriter.defaults();
     opts.setConf(conf);
     opts.output(out.toString())
-        .blocks(TextFileRegionFormat.class)
+        .blocks(TextFileRegionAliasMap.class)
         .blockIds(blockIdsClass);
     try (ImageWriter w = new ImageWriter(opts)) {
       for (TreePath e : t) {


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


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

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


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


[08/50] [abbrv] hadoop git commit: HDFS-11791. [READ] Test for increasing replication of provided files.

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


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

Branch: refs/heads/HDFS-9806
Commit: 8ab623269f9d8a813f1917181759547e2e3ba2b4
Parents: e0793a7
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed May 31 10:29:53 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

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


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


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


[13/50] [abbrv] hadoop git commit: HDFS-10675. Datanode support to read from external stores.

Posted by vi...@apache.org.
HDFS-10675. Datanode support to read from external stores.


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

Branch: refs/heads/HDFS-9806
Commit: 02a28b93ffc2669760d22184d383ae1ba3b80ccf
Parents: 1c15b17
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Mar 29 14:29:28 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/StorageType.java  |   3 +-
 .../org/apache/hadoop/fs/shell/TestCount.java   |   3 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   4 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |   4 +
 .../src/main/proto/hdfs.proto                   |   1 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  15 +
 .../hadoop/hdfs/server/common/BlockAlias.java   |  29 +
 .../hadoop/hdfs/server/common/BlockFormat.java  |  82 +++
 .../hadoop/hdfs/server/common/FileRegion.java   | 121 +++++
 .../hdfs/server/common/FileRegionProvider.java  |  37 ++
 .../hadoop/hdfs/server/common/Storage.java      |  71 ++-
 .../hadoop/hdfs/server/common/StorageInfo.java  |   6 +
 .../server/common/TextFileRegionFormat.java     | 442 ++++++++++++++++
 .../server/common/TextFileRegionProvider.java   |  88 ++++
 .../server/datanode/BlockPoolSliceStorage.java  |  21 +-
 .../hdfs/server/datanode/DataStorage.java       |  44 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |  19 +-
 .../datanode/FinalizedProvidedReplica.java      |  91 ++++
 .../hdfs/server/datanode/ProvidedReplica.java   | 248 +++++++++
 .../hdfs/server/datanode/ReplicaBuilder.java    | 100 +++-
 .../hdfs/server/datanode/ReplicaInfo.java       |  20 +-
 .../hdfs/server/datanode/StorageLocation.java   |  26 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   4 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  32 +-
 .../fsdataset/impl/DefaultProvidedVolumeDF.java |  58 ++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  40 +-
 .../datanode/fsdataset/impl/FsDatasetUtil.java  |  25 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  19 +-
 .../fsdataset/impl/FsVolumeImplBuilder.java     |   6 +
 .../fsdataset/impl/ProvidedVolumeDF.java        |  34 ++
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 526 +++++++++++++++++++
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../server/namenode/FSImageCompression.java     |   2 +-
 .../hadoop/hdfs/server/namenode/NNStorage.java  |  10 +-
 .../src/main/resources/hdfs-default.xml         |  78 +++
 .../org/apache/hadoop/hdfs/TestDFSRollback.java |   6 +-
 .../hadoop/hdfs/TestDFSStartupVersions.java     |   2 +-
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |   4 +-
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |  16 +-
 .../hdfs/server/common/TestTextBlockFormat.java | 160 ++++++
 .../server/datanode/SimulatedFSDataset.java     |   6 +-
 .../extdataset/ExternalDatasetImpl.java         |   5 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |  17 +-
 .../fsdataset/impl/TestProvidedImpl.java        | 426 +++++++++++++++
 .../hdfs/server/namenode/TestClusterId.java     |   5 +-
 45 files changed, 2873 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
index 0948801..2ecd206 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
@@ -37,7 +37,8 @@ public enum StorageType {
   RAM_DISK(true),
   SSD(false),
   DISK(false),
-  ARCHIVE(false);
+  ARCHIVE(false),
+  PROVIDED(false);
 
   private final boolean isTransient;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index a782958..b5adfcf 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -285,7 +285,7 @@ public class TestCount {
         // <----13---> <-------17------> <----13-----> <------17------->
         "    SSD_QUOTA     REM_SSD_QUOTA    DISK_QUOTA    REM_DISK_QUOTA " +
         // <----13---> <-------17------>
-        "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
+        "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA PROVIDED_QUOTA REM_PROVIDED_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);
     verifyNoMoreInteractions(out);
@@ -340,6 +340,7 @@ public class TestCount {
         "    SSD_QUOTA     REM_SSD_QUOTA " +
         "   DISK_QUOTA    REM_DISK_QUOTA " +
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
+        "PROVIDED_QUOTA REM_PROVIDED_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);
     verifyNoMoreInteractions(out);

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 73c5215..8ff9bfe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -403,6 +403,8 @@ public class PBHelperClient {
       return StorageTypeProto.ARCHIVE;
     case RAM_DISK:
       return StorageTypeProto.RAM_DISK;
+    case PROVIDED:
+      return StorageTypeProto.PROVIDED;
     default:
       throw new IllegalStateException(
           "BUG: StorageType not found, type=" + type);
@@ -419,6 +421,8 @@ public class PBHelperClient {
       return StorageType.ARCHIVE;
     case RAM_DISK:
       return StorageType.RAM_DISK;
+    case PROVIDED:
+      return StorageType.PROVIDED;
     default:
       throw new IllegalStateException(
           "BUG: StorageTypeProto not found, type=" + type);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index fca1e0e..25f1339 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -205,6 +205,7 @@ enum StorageTypeProto {
   SSD = 2;
   ARCHIVE = 3;
   RAM_DISK = 4;
+  PROVIDED = 5;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 97b8b1a..ca753ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -328,6 +328,21 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.edits.asynclogging";
   public static final boolean DFS_NAMENODE_EDITS_ASYNC_LOGGING_DEFAULT = true;
 
+  public static final String DFS_PROVIDER_CLASS = "dfs.provider.class";
+  public static final String DFS_PROVIDER_DF_CLASS = "dfs.provided.df.class";
+  public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";
+  public static final String DFS_PROVIDER_STORAGEUUID_DEFAULT =  "DS-PROVIDED";
+  public static final String DFS_PROVIDER_BLK_FORMAT_CLASS = "dfs.provided.blockformat.class";
+
+  public static final String DFS_PROVIDED_BLOCK_MAP_DELIMITER = "dfs.provided.textprovider.delimiter";
+  public static final String DFS_PROVIDED_BLOCK_MAP_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_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_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;
   public static final String  DFS_CONTENT_SUMMARY_LIMIT_KEY = "dfs.content-summary.limit";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
new file mode 100644
index 0000000..b2fac97
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Interface used to load provided blocks.
+ */
+public interface BlockAlias {
+
+  Block getBlock();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
new file mode 100644
index 0000000..66e7fdf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * An abstract class used to read and write block maps for provided blocks.
+ */
+public abstract class BlockFormat<T extends BlockAlias>  {
+
+  /**
+   * An abstract class that is used to read {@link BlockAlias}es
+   * for provided blocks.
+   */
+  public static abstract class Reader<U extends BlockAlias>
+      implements Iterable<U>, Closeable {
+
+    /**
+     * reader options.
+     */
+    public interface Options { }
+
+    public abstract U resolve(Block ident) throws IOException;
+
+  }
+
+  /**
+   * Returns the reader for the provided block map.
+   * @param opts reader options
+   * @return {@link Reader} to the block map.
+   * @throws IOException
+   */
+  public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
+
+  /**
+   * An abstract class used as a writer for the provided block map.
+   */
+  public static abstract class Writer<U extends BlockAlias>
+      implements Closeable {
+    /**
+     * writer options.
+     */
+    public interface Options { }
+
+    public abstract void store(U token) throws IOException;
+
+  }
+
+  /**
+   * Returns the writer for the provided block map.
+   * @param opts writer options.
+   * @return {@link Writer} to the block map.
+   * @throws IOException
+   */
+  public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
+
+  /**
+   * Refresh based on the underlying block map.
+   * @throws IOException
+   */
+  public abstract void refresh() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
new file mode 100644
index 0000000..c568b90
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
@@ -0,0 +1,121 @@
+/**
+ * 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 org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+
+/**
+ * This class is used to represent provided blocks that are file regions,
+ * i.e., can be described using (path, offset, length).
+ */
+public class FileRegion implements BlockAlias {
+
+  private final Path path;
+  private final long offset;
+  private final long length;
+  private final long blockId;
+  private final String bpid;
+  private final long genStamp;
+
+  public FileRegion(long blockId, Path path, long offset,
+      long length, String bpid, long genStamp) {
+    this.path = path;
+    this.offset = offset;
+    this.length = length;
+    this.blockId = blockId;
+    this.bpid = bpid;
+    this.genStamp = genStamp;
+  }
+
+  public FileRegion(long blockId, Path path, long offset,
+      long length, String bpid) {
+    this(blockId, path, offset, length, bpid,
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+
+  }
+
+  public FileRegion(long blockId, Path path, long offset,
+      long length, long genStamp) {
+    this(blockId, path, offset, length, null, genStamp);
+
+  }
+
+  public FileRegion(long blockId, Path path, long offset, long length) {
+    this(blockId, path, offset, length, null);
+  }
+
+  @Override
+  public Block getBlock() {
+    return new Block(blockId, length, genStamp);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof FileRegion)) {
+      return false;
+    }
+    FileRegion o = (FileRegion) other;
+    return blockId == o.blockId
+      && offset == o.offset
+      && length == o.length
+      && genStamp == o.genStamp
+      && path.equals(o.path);
+  }
+
+  @Override
+  public int hashCode() {
+    return (int)(blockId & Integer.MIN_VALUE);
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public long getGenerationStamp() {
+    return genStamp;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{ block=\"").append(getBlock()).append("\"");
+    sb.append(", path=\"").append(getPath()).append("\"");
+    sb.append(", off=\"").append(getOffset()).append("\"");
+    sb.append(", len=\"").append(getBlock().getNumBytes()).append("\"");
+    sb.append(", genStamp=\"").append(getBlock()
+        .getGenerationStamp()).append("\"");
+    sb.append(", bpid=\"").append(bpid).append("\"");
+    sb.append(" }");
+    return sb.toString();
+  }
+
+  public String getBlockPoolId() {
+    return this.bpid;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
new file mode 100644
index 0000000..2e94239
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.common;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+
+/**
+ * This class is a stub for reading file regions from the block map.
+ */
+public class FileRegionProvider implements Iterable<FileRegion> {
+  @Override
+  public Iterator<FileRegion> iterator() {
+    return Collections.emptyListIterator();
+  }
+
+  public void refresh() throws IOException {
+    return;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index 414d3a7..9ad61d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -40,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@@ -196,7 +197,10 @@ public abstract class Storage extends StorageInfo {
     Iterator<StorageDirectory> it =
       (dirType == null) ? dirIterator() : dirIterator(dirType);
     for ( ;it.hasNext(); ) {
-      list.add(new File(it.next().getCurrentDir(), fileName));
+      File currentDir = it.next().getCurrentDir();
+      if (currentDir != null) {
+        list.add(new File(currentDir, fileName));
+      }
     }
     return list;
   }
@@ -328,10 +332,20 @@ public abstract class Storage extends StorageInfo {
      */
     public StorageDirectory(String bpid, StorageDirType dirType,
         boolean isShared, StorageLocation location) {
-      this(new File(location.getBpURI(bpid, STORAGE_DIR_CURRENT)), dirType,
+      this(getBlockPoolCurrentDir(bpid, location), dirType,
           isShared, location);
     }
 
+    private static File getBlockPoolCurrentDir(String bpid,
+        StorageLocation location) {
+      if (location == null ||
+          location.getStorageType() == StorageType.PROVIDED) {
+        return null;
+      } else {
+        return new File(location.getBpURI(bpid, STORAGE_DIR_CURRENT));
+      }
+    }
+
     private StorageDirectory(File dir, StorageDirType dirType,
         boolean isShared, StorageLocation location) {
       this.root = dir;
@@ -347,7 +361,8 @@ public abstract class Storage extends StorageInfo {
     }
 
     private static File getStorageLocationFile(StorageLocation location) {
-      if (location == null) {
+      if (location == null ||
+          location.getStorageType() == StorageType.PROVIDED) {
         return null;
       }
       try {
@@ -406,6 +421,10 @@ public abstract class Storage extends StorageInfo {
      */
     public void clearDirectory() throws IOException {
       File curDir = this.getCurrentDir();
+      if (curDir == null) {
+        //if the directory is null, there is nothing to do.
+        return;
+      }
       if (curDir.exists()) {
         File[] files = FileUtil.listFiles(curDir);
         LOG.info("Will remove files: " + Arrays.toString(files));
@@ -423,6 +442,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getCurrentDir() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_DIR_CURRENT);
     }
 
@@ -443,6 +465,9 @@ public abstract class Storage extends StorageInfo {
      * @return the version file path
      */
     public File getVersionFile() {
+      if (root == null) {
+        return null;
+      }
       return new File(new File(root, STORAGE_DIR_CURRENT), STORAGE_FILE_VERSION);
     }
 
@@ -452,6 +477,9 @@ public abstract class Storage extends StorageInfo {
      * @return the previous version file path
      */
     public File getPreviousVersionFile() {
+      if (root == null) {
+        return null;
+      }
       return new File(new File(root, STORAGE_DIR_PREVIOUS), STORAGE_FILE_VERSION);
     }
 
@@ -462,6 +490,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getPreviousDir() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_DIR_PREVIOUS);
     }
 
@@ -476,6 +507,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getPreviousTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_PREVIOUS);
     }
 
@@ -490,6 +524,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getRemovedTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_REMOVED);
     }
 
@@ -503,6 +540,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getFinalizedTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_FINALIZED);
     }
 
@@ -517,6 +557,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getLastCheckpointTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_LAST_CKPT);
     }
 
@@ -530,6 +573,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getPreviousCheckpoint() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_PREVIOUS_CKPT);
     }
 
@@ -543,7 +589,7 @@ public abstract class Storage extends StorageInfo {
     private void checkEmptyCurrent() throws InconsistentFSStateException,
         IOException {
       File currentDir = getCurrentDir();
-      if(!currentDir.exists()) {
+      if(currentDir == null || !currentDir.exists()) {
         // if current/ does not exist, it's safe to format it.
         return;
       }
@@ -589,6 +635,13 @@ public abstract class Storage extends StorageInfo {
     public StorageState analyzeStorage(StartupOption startOpt, Storage storage,
         boolean checkCurrentIsEmpty)
         throws IOException {
+
+      if (location != null &&
+          location.getStorageType() == StorageType.PROVIDED) {
+        //currently we assume that PROVIDED storages are always NORMAL
+        return StorageState.NORMAL;
+      }
+
       assert root != null : "root is null";
       boolean hadMkdirs = false;
       String rootPath = root.getCanonicalPath();
@@ -710,6 +763,10 @@ public abstract class Storage extends StorageInfo {
      */
     public void doRecover(StorageState curState) throws IOException {
       File curDir = getCurrentDir();
+      if (curDir == null || root == null) {
+        //at this point, we do not support recovery on PROVIDED storages
+        return;
+      }
       String rootPath = root.getCanonicalPath();
       switch(curState) {
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
@@ -883,7 +940,8 @@ public abstract class Storage extends StorageInfo {
     
     @Override
     public String toString() {
-      return "Storage Directory " + this.root;
+      return "Storage Directory root= " + this.root +
+          "; location= " + this.location;
     }
 
     /**
@@ -1153,6 +1211,9 @@ public abstract class Storage extends StorageInfo {
   }
   
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
+    if (to == null) {
+      return;
+    }
     Properties props = new Properties();
     setPropertiesFromFields(props, sd);
     writeProperties(to, props);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
index 50363c9..28871e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
@@ -152,6 +152,9 @@ public class StorageInfo {
    */
   protected void setFieldsFromProperties(
       Properties props, StorageDirectory sd) throws IOException {
+    if (props == null) {
+      return;
+    }
     setLayoutVersion(props, sd);
     setNamespaceID(props, sd);
     setcTime(props, sd);
@@ -241,6 +244,9 @@ public class StorageInfo {
   }
 
   public static Properties readPropertiesFile(File from) throws IOException {
+    if (from == null) {
+      return null;
+    }
     RandomAccessFile file = new RandomAccessFile(from, "rws");
     FileInputStream in = null;
     Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
new file mode 100644
index 0000000..eacd08f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
@@ -0,0 +1,442 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.common;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is used for block maps stored as text files,
+ * with a specified delimiter.
+ */
+public class TextFileRegionFormat
+    extends BlockFormat<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private ReaderOptions readerOpts = TextReader.defaults();
+  private WriterOptions writerOpts = TextWriter.defaults();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TextFileRegionFormat.class);
+  @Override
+  public void setConf(Configuration conf) {
+    readerOpts.setConf(conf);
+    writerOpts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts)
+      throws IOException {
+    if (null == opts) {
+      opts = readerOpts;
+    }
+    if (!(opts instanceof ReaderOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    ReaderOptions o = (ReaderOptions) opts;
+    Configuration readerConf = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    return createReader(o.file, o.delim, readerConf);
+  }
+
+  @VisibleForTesting
+  TextReader createReader(Path file, String delim, Configuration cfg)
+      throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+    CompressionCodec codec = factory.getCodec(file);
+    return new TextReader(fs, file, codec, delim);
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    if (null == opts) {
+      opts = writerOpts;
+    }
+    if (!(opts instanceof WriterOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    WriterOptions o = (WriterOptions) opts;
+    Configuration cfg = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    if (o.codec != null) {
+      CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+      CompressionCodec codec = factory.getCodecByName(o.codec);
+      String name = o.file.getName() + codec.getDefaultExtension();
+      o.filename(new Path(o.file.getParent(), name));
+      return createWriter(o.file, codec, o.delim, cfg);
+    }
+    return createWriter(o.file, null, o.delim, conf);
+  }
+
+  @VisibleForTesting
+  TextWriter createWriter(Path file, CompressionCodec codec, String delim,
+      Configuration cfg) throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    OutputStream tmp = fs.create(file);
+    java.io.Writer out = new BufferedWriter(new OutputStreamWriter(
+          (null == codec) ? tmp : codec.createOutputStream(tmp), "UTF-8"));
+    return new TextWriter(out, delim);
+  }
+
+  /**
+   * Class specifying reader options for the {@link TextFileRegionFormat}.
+   */
+  public static class ReaderOptions
+      implements TextReader.Options, Configurable {
+
+    private Configuration conf;
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
+    private Path file = new Path(
+        new File(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT)
+        .toURI().toString());
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
+          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
+      file = new Path(tmpfile);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT);
+      LOG.info("TextFileRegionFormat: read path " + tmpfile.toString());
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public ReaderOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    @Override
+    public ReaderOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+  }
+
+  /**
+   * Class specifying writer options for the {@link TextFileRegionFormat}.
+   */
+  public static class WriterOptions
+      implements TextWriter.Options, Configurable {
+
+    private Configuration conf;
+    private String codec = null;
+    private Path file =
+        new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile = conf.get(
+          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_WRITE_PATH, file.toString());
+      file = new Path(tmpfile);
+      codec = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_CODEC);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public WriterOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    public String getCodec() {
+      return codec;
+    }
+
+    public Path getFile() {
+      return file;
+    }
+
+    @Override
+    public WriterOptions codec(String codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    @Override
+    public WriterOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link TextReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    static ReaderOptions defaults() {
+      return new ReaderOptions();
+    }
+
+    private final Path file;
+    private final String delim;
+    private final FileSystem fs;
+    private final CompressionCodec codec;
+    private final Map<FRIterator, BufferedReader> iterators;
+
+    protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
+        String delim) {
+      this(fs, file, codec, delim,
+          new IdentityHashMap<FRIterator, BufferedReader>());
+    }
+
+    TextReader(FileSystem fs, Path file, CompressionCodec codec, String delim,
+        Map<FRIterator, BufferedReader> iterators) {
+      this.fs = fs;
+      this.file = file;
+      this.codec = codec;
+      this.delim = delim;
+      this.iterators = Collections.synchronizedMap(iterators);
+    }
+
+    @Override
+    public FileRegion resolve(Block ident) throws IOException {
+      // consider layering index w/ composable format
+      Iterator<FileRegion> i = iterator();
+      try {
+        while (i.hasNext()) {
+          FileRegion f = i.next();
+          if (f.getBlock().equals(ident)) {
+            return f;
+          }
+        }
+      } finally {
+        BufferedReader r = iterators.remove(i);
+        if (r != null) {
+          // null on last element
+          r.close();
+        }
+      }
+      return null;
+    }
+
+    class FRIterator implements Iterator<FileRegion> {
+
+      private FileRegion pending;
+
+      @Override
+      public boolean hasNext() {
+        return pending != null;
+      }
+
+      @Override
+      public FileRegion next() {
+        if (null == pending) {
+          throw new NoSuchElementException();
+        }
+        FileRegion ret = pending;
+        try {
+          pending = nextInternal(this);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return ret;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    private FileRegion nextInternal(Iterator<FileRegion> i) throws IOException {
+      BufferedReader r = iterators.get(i);
+      if (null == r) {
+        throw new IllegalStateException();
+      }
+      String line = r.readLine();
+      if (null == line) {
+        iterators.remove(i);
+        return null;
+      }
+      String[] f = line.split(delim);
+      if (f.length != 6) {
+        throw new IOException("Invalid line: " + line);
+      }
+      return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
+          Long.parseLong(f[2]), Long.parseLong(f[3]), f[5],
+          Long.parseLong(f[4]));
+    }
+
+    public InputStream createStream() throws IOException {
+      InputStream i = fs.open(file);
+      if (codec != null) {
+        i = codec.createInputStream(i);
+      }
+      return i;
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      FRIterator i = new FRIterator();
+      try {
+        BufferedReader r =
+            new BufferedReader(new InputStreamReader(createStream(), "UTF-8"));
+        iterators.put(i, r);
+        i.pending = nextInternal(i);
+      } catch (IOException e) {
+        iterators.remove(i);
+        throw new RuntimeException(e);
+      }
+      return i;
+    }
+
+    @Override
+    public void close() throws IOException {
+      ArrayList<IOException> ex = new ArrayList<>();
+      synchronized (iterators) {
+        for (Iterator<BufferedReader> i = iterators.values().iterator();
+             i.hasNext();) {
+          try {
+            BufferedReader r = i.next();
+            r.close();
+          } catch (IOException e) {
+            ex.add(e);
+          } finally {
+            i.remove();
+          }
+        }
+        iterators.clear();
+      }
+      if (!ex.isEmpty()) {
+        throw MultipleIOException.createIOException(ex);
+      }
+    }
+
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options codec(String codec);
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    public static WriterOptions defaults() {
+      return new WriterOptions();
+    }
+
+    private final String delim;
+    private final java.io.Writer out;
+
+    public TextWriter(java.io.Writer out, String delim) {
+      this.out = out;
+      this.delim = delim;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      out.append(String.valueOf(token.getBlock().getBlockId())).append(delim);
+      out.append(token.getPath().toString()).append(delim);
+      out.append(Long.toString(token.getOffset())).append(delim);
+      out.append(Long.toString(token.getLength())).append(delim);
+      out.append(Long.toString(token.getGenerationStamp())).append(delim);
+      out.append(token.getBlockPoolId()).append("\n");
+    }
+
+    @Override
+    public void close() throws IOException {
+      out.close();
+    }
+
+  }
+
+  @Override
+  public void refresh() throws IOException {
+    //nothing to do;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
new file mode 100644
index 0000000..0fa667e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.common;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This class is used to read file regions from block maps
+ * specified using delimited text.
+ */
+public class TextFileRegionProvider
+    extends FileRegionProvider implements Configurable {
+
+  private Configuration conf;
+  private BlockFormat<FileRegion> fmt;
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void setConf(Configuration conf) {
+    fmt = ReflectionUtils.newInstance(
+        conf.getClass(DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
+            TextFileRegionFormat.class,
+            BlockFormat.class),
+        conf);
+    ((Configurable)fmt).setConf(conf); //redundant?
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Iterator<FileRegion> iterator() {
+    try {
+      final BlockFormat.Reader<FileRegion> r = fmt.getReader(null);
+      return new Iterator<FileRegion>() {
+
+        private final Iterator<FileRegion> inner = r.iterator();
+
+        @Override
+        public boolean hasNext() {
+          return inner.hasNext();
+        }
+
+        @Override
+        public FileRegion next() {
+          return inner.next();
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to read provided blocks", e);
+    }
+  }
+
+  @Override
+  public void refresh() throws IOException {
+    fmt.refresh();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
index bc41715..012d1f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -360,6 +361,9 @@ public class BlockPoolSliceStorage extends Storage {
   private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
       StartupOption startOpt, List<Callable<StorageDirectory>> callables,
       Configuration conf) throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      return false; // regular startup for PROVIDED storage directories
+    }
     if (startOpt == StartupOption.ROLLBACK && sd.getPreviousDir().exists()) {
       Preconditions.checkState(!getTrashRootDir(sd).exists(),
           sd.getPreviousDir() + " and " + getTrashRootDir(sd) + " should not " +
@@ -439,6 +443,10 @@ public class BlockPoolSliceStorage extends Storage {
         LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       return;
     }
+    //no upgrades for storage directories that are PROVIDED
+    if (bpSd.getRoot() == null) {
+      return;
+    }
     final int oldLV = getLayoutVersion();
     LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
         + ".\n   old LV = " + oldLV
@@ -589,8 +597,9 @@ public class BlockPoolSliceStorage extends Storage {
       throws IOException {
     File prevDir = bpSd.getPreviousDir();
     // regular startup if previous dir does not exist
-    if (!prevDir.exists())
+    if (prevDir == null || !prevDir.exists()) {
       return;
+    }
     // read attributes out of the VERSION file of previous directory
     BlockPoolSliceStorage prevInfo = new BlockPoolSliceStorage();
     prevInfo.readPreviousVersionProperties(bpSd);
@@ -631,6 +640,10 @@ public class BlockPoolSliceStorage extends Storage {
    * that holds the snapshot.
    */
   void doFinalize(File dnCurDir) throws IOException {
+    LOG.info("doFinalize: " + dnCurDir);
+    if (dnCurDir == null) {
+      return; //we do nothing if the directory is null
+    }
     File bpRoot = getBpRoot(blockpoolID, dnCurDir);
     StorageDirectory bpSd = new StorageDirectory(bpRoot);
     // block pool level previous directory
@@ -841,6 +854,9 @@ public class BlockPoolSliceStorage extends Storage {
   public void setRollingUpgradeMarkers(List<StorageDirectory> dnStorageDirs)
       throws IOException {
     for (StorageDirectory sd : dnStorageDirs) {
+      if (sd.getCurrentDir() == null) {
+        return;
+      }
       File bpRoot = getBpRoot(blockpoolID, sd.getCurrentDir());
       File markerFile = new File(bpRoot, ROLLING_UPGRADE_MARKER_FILE);
       if (!storagesWithRollingUpgradeMarker.contains(bpRoot.toString())) {
@@ -863,6 +879,9 @@ public class BlockPoolSliceStorage extends Storage {
   public void clearRollingUpgradeMarkers(List<StorageDirectory> dnStorageDirs)
       throws IOException {
     for (StorageDirectory sd : dnStorageDirs) {
+      if (sd.getCurrentDir() == null) {
+        continue;
+      }
       File bpRoot = getBpRoot(blockpoolID, sd.getCurrentDir());
       File markerFile = new File(bpRoot, ROLLING_UPGRADE_MARKER_FILE);
       if (!storagesWithoutRollingUpgradeMarker.contains(bpRoot.toString())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 6d6e96a..a1bde31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -129,22 +130,31 @@ public class DataStorage extends Storage {
     this.datanodeUuid = newDatanodeUuid;
   }
 
-  private static boolean createStorageID(StorageDirectory sd, int lv) {
+  private static boolean createStorageID(StorageDirectory sd, int lv,
+      Configuration conf) {
     // Clusters previously upgraded from layout versions earlier than
     // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
     // new storage ID. We check for that and fix it now.
     final boolean haveValidStorageId = DataNodeLayoutVersion.supports(
         LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, lv)
         && DatanodeStorage.isValidStorageId(sd.getStorageUuid());
-    return createStorageID(sd, !haveValidStorageId);
+    return createStorageID(sd, !haveValidStorageId, conf);
   }
 
   /** Create an ID for this storage.
    * @return true if a new storage ID was generated.
    * */
   public static boolean createStorageID(
-      StorageDirectory sd, boolean regenerateStorageIds) {
+      StorageDirectory sd, boolean regenerateStorageIds, Configuration conf) {
     final String oldStorageID = sd.getStorageUuid();
+    if (sd.getStorageLocation() != null &&
+        sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      // We only support one provided storage per datanode for now.
+      // TODO support multiple provided storage ids per datanode.
+      sd.setStorageUuid(conf.get(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+          DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT));
+      return false;
+    }
     if (oldStorageID == null || regenerateStorageIds) {
       sd.setStorageUuid(DatanodeStorage.generateUuid());
       LOG.info("Generated new storageID " + sd.getStorageUuid() +
@@ -273,7 +283,7 @@ public class DataStorage extends Storage {
         LOG.info("Storage directory with location " + location
             + " is not formatted for namespace " + nsInfo.getNamespaceID()
             + ". Formatting...");
-        format(sd, nsInfo, datanode.getDatanodeUuid());
+        format(sd, nsInfo, datanode.getDatanodeUuid(), datanode.getConf());
         break;
       default:  // recovery part is common
         sd.doRecover(curState);
@@ -547,15 +557,15 @@ public class DataStorage extends Storage {
   }
 
   void format(StorageDirectory sd, NamespaceInfo nsInfo,
-              String datanodeUuid) throws IOException {
+              String newDatanodeUuid, Configuration conf) throws IOException {
     sd.clearDirectory(); // create directory
     this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
-    setDatanodeUuid(datanodeUuid);
+    setDatanodeUuid(newDatanodeUuid);
 
-    createStorageID(sd, false);
+    createStorageID(sd, false, conf);
     writeProperties(sd);
   }
 
@@ -600,6 +610,9 @@ public class DataStorage extends Storage {
 
   private void setFieldsFromProperties(Properties props, StorageDirectory sd,
       boolean overrideLayoutVersion, int toLayoutVersion) throws IOException {
+    if (props == null) {
+      return;
+    }
     if (overrideLayoutVersion) {
       this.layoutVersion = toLayoutVersion;
     } else {
@@ -694,6 +707,10 @@ public class DataStorage extends Storage {
   private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
       StartupOption startOpt, List<Callable<StorageDirectory>> callables,
       Configuration conf) throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      createStorageID(sd, layoutVersion, conf);
+      return false; // regular start up for PROVIDED storage directories
+    }
     if (startOpt == StartupOption.ROLLBACK) {
       doRollback(sd, nsInfo); // rollback if applicable
     }
@@ -724,7 +741,7 @@ public class DataStorage extends Storage {
 
     // regular start up.
     if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
-      createStorageID(sd, layoutVersion);
+      createStorageID(sd, layoutVersion, conf);
       return false; // need to write properties
     }
 
@@ -733,7 +750,7 @@ public class DataStorage extends Storage {
       if (federationSupported) {
         // If the existing on-disk layout version supports federation,
         // simply update the properties.
-        upgradeProperties(sd);
+        upgradeProperties(sd, conf);
       } else {
         doUpgradePreFederation(sd, nsInfo, callables, conf);
       }
@@ -829,15 +846,16 @@ public class DataStorage extends Storage {
 
     // 4. Write version file under <SD>/current
     clusterID = nsInfo.getClusterID();
-    upgradeProperties(sd);
+    upgradeProperties(sd, conf);
     
     // 5. Rename <SD>/previous.tmp to <SD>/previous
     rename(tmpDir, prevDir);
     LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
   }
 
-  void upgradeProperties(StorageDirectory sd) throws IOException {
-    createStorageID(sd, layoutVersion);
+  void upgradeProperties(StorageDirectory sd, Configuration conf)
+      throws IOException {
+    createStorageID(sd, layoutVersion, conf);
     LOG.info("Updating layout version from " + layoutVersion
         + " to " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
         + " for storage " + sd.getRoot());
@@ -989,7 +1007,7 @@ public class DataStorage extends Storage {
     // then finalize it. Else finalize the corresponding BP.
     for (StorageDirectory sd : getStorageDirs()) {
       File prevDir = sd.getPreviousDir();
-      if (prevDir.exists()) {
+      if (prevDir != null && prevDir.exists()) {
         // data node level storage finalize
         doFinalize(sd);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 966bcb0..3b6d06c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -44,6 +44,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -105,7 +106,7 @@ public class DirectoryScanner implements Runnable {
    * @param b whether to retain diffs
    */
   @VisibleForTesting
-  void setRetainDiffs(boolean b) {
+  public void setRetainDiffs(boolean b) {
     retainDiffs = b;
   }
 
@@ -215,7 +216,8 @@ public class DirectoryScanner implements Runnable {
    * @param dataset the dataset to scan
    * @param conf the Configuration object
    */
-  DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset, Configuration conf) {
+  public DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset,
+      Configuration conf) {
     this.datanode = datanode;
     this.dataset = dataset;
     int interval = (int) conf.getTimeDuration(
@@ -369,15 +371,14 @@ public class DirectoryScanner implements Runnable {
    * Reconcile differences between disk and in-memory blocks
    */
   @VisibleForTesting
-  void reconcile() throws IOException {
+  public void reconcile() throws IOException {
     scan();
     for (Entry<String, LinkedList<ScanInfo>> entry : diffs.entrySet()) {
       String bpid = entry.getKey();
       LinkedList<ScanInfo> diff = entry.getValue();
       
       for (ScanInfo info : diff) {
-        dataset.checkAndUpdate(bpid, info.getBlockId(), info.getBlockFile(),
-            info.getMetaFile(), info.getVolume());
+        dataset.checkAndUpdate(bpid, info);
       }
     }
     if (!retainDiffs) clear();
@@ -429,11 +430,12 @@ public class DirectoryScanner implements Runnable {
           }
           // Block file and/or metadata file exists on the disk
           // Block exists in memory
-          if (info.getBlockFile() == null) {
+          if (info.getVolume().getStorageType() != StorageType.PROVIDED &&
+              info.getBlockFile() == null) {
             // Block metadata file exits and block file is missing
             addDifference(diffRecord, statsRecord, info);
           } else if (info.getGenStamp() != memBlock.getGenerationStamp()
-              || info.getBlockFileLength() != memBlock.getNumBytes()) {
+              || info.getBlockLength() != memBlock.getNumBytes()) {
             // Block metadata file is missing or has wrong generation stamp,
             // or block file length is different than expected
             statsRecord.mismatchBlocks++;
@@ -611,6 +613,9 @@ public class DirectoryScanner implements Runnable {
       for (String bpid : bpList) {
         LinkedList<ScanInfo> report = new LinkedList<>();
 
+        perfTimer.reset().start();
+        throttleTimer.reset().start();
+
         try {
           result.put(bpid, volume.compileReport(bpid, report, this));
         } catch (InterruptedException ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
new file mode 100644
index 0000000..722d573
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+
+/**
+ * This class is used for provided replicas that are finalized.
+ */
+public class FinalizedProvidedReplica extends ProvidedReplica {
+
+  public FinalizedProvidedReplica(long blockId, URI fileURI,
+      long fileOffset, long blockLen, long genStamp,
+      FsVolumeSpi volume, Configuration conf) {
+    super(blockId, fileURI, fileOffset, blockLen, genStamp, volume, conf);
+  }
+
+  @Override
+  public ReplicaState getState() {
+    return ReplicaState.FINALIZED;
+  }
+
+  @Override
+  public long getBytesOnDisk() {
+    return getNumBytes();
+  }
+
+  @Override
+  public long getVisibleLength() {
+    return getNumBytes(); //all bytes are visible
+  }
+
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return super.toString();
+  }
+
+  @Override
+  public ReplicaInfo getOriginalReplica() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getOriginalReplica");
+  }
+
+  @Override
+  public long getRecoveryID() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getRecoveryID");
+  }
+
+  @Override
+  public void setRecoveryID(long recoveryId) {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support setRecoveryID");
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support createInfo");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
new file mode 100644
index 0000000..b021ea2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
@@ -0,0 +1,248 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This abstract class is used as a base class for provided replicas.
+ */
+public abstract class ProvidedReplica extends ReplicaInfo {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ProvidedReplica.class);
+
+  // Null checksum information for provided replicas.
+  // Shared across all replicas.
+  static final byte[] NULL_CHECKSUM_ARRAY =
+      FsDatasetUtil.createNullChecksumByteArray();
+  private URI fileURI;
+  private long fileOffset;
+  private Configuration conf;
+  private FileSystem remoteFS;
+
+  /**
+   * Constructor.
+   * @param blockId block id
+   * @param fileURI remote URI this block is to be read from
+   * @param fileOffset the offset in the remote URI
+   * @param blockLen the length of the block
+   * @param genStamp the generation stamp of the block
+   * @param volume the volume this block belongs to
+   */
+  public ProvidedReplica(long blockId, URI fileURI, long fileOffset,
+      long blockLen, long genStamp, FsVolumeSpi volume, Configuration conf) {
+    super(volume, blockId, blockLen, genStamp);
+    this.fileURI = fileURI;
+    this.fileOffset = fileOffset;
+    this.conf = conf;
+    try {
+      this.remoteFS = FileSystem.get(fileURI, this.conf);
+    } catch (IOException e) {
+      LOG.warn("Failed to obtain filesystem for " + fileURI);
+      this.remoteFS = null;
+    }
+  }
+
+  public ProvidedReplica(ProvidedReplica r) {
+    super(r);
+    this.fileURI = r.fileURI;
+    this.fileOffset = r.fileOffset;
+    this.conf = r.conf;
+    try {
+      this.remoteFS = FileSystem.newInstance(fileURI, this.conf);
+    } catch (IOException e) {
+      this.remoteFS = null;
+    }
+  }
+
+  @Override
+  public URI getBlockURI() {
+    return this.fileURI;
+  }
+
+  @Override
+  public InputStream getDataInputStream(long seekOffset) throws IOException {
+    if (remoteFS != null) {
+      FSDataInputStream ins = remoteFS.open(new Path(fileURI));
+      ins.seek(fileOffset + seekOffset);
+      return new FSDataInputStream(ins);
+    } else {
+      throw new IOException("Remote filesystem for provided replica " + this +
+          " does not exist");
+    }
+  }
+
+  @Override
+  public OutputStream getDataOutputStream(boolean append) throws IOException {
+    throw new UnsupportedOperationException(
+        "OutputDataStream is not implemented for ProvidedReplica");
+  }
+
+  @Override
+  public URI getMetadataURI() {
+    return null;
+  }
+
+  @Override
+  public OutputStream getMetadataOutputStream(boolean append)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean blockDataExists() {
+    if(remoteFS != null) {
+      try {
+        return remoteFS.exists(new Path(fileURI));
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public boolean deleteBlockData() {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support deleting block data");
+  }
+
+  @Override
+  public long getBlockDataLength() {
+    return this.getNumBytes();
+  }
+
+  @Override
+  public LengthInputStream getMetadataInputStream(long offset)
+      throws IOException {
+    return new LengthInputStream(new ByteArrayInputStream(NULL_CHECKSUM_ARRAY),
+        NULL_CHECKSUM_ARRAY.length);
+  }
+
+  @Override
+  public boolean metadataExists() {
+    return NULL_CHECKSUM_ARRAY == null ? false : true;
+  }
+
+  @Override
+  public boolean deleteMetadata() {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support deleting metadata");
+  }
+
+  @Override
+  public long getMetadataLength() {
+    return NULL_CHECKSUM_ARRAY == null ? 0 : NULL_CHECKSUM_ARRAY.length;
+  }
+
+  @Override
+  public boolean renameMeta(URI destURI) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support renaming metadata");
+  }
+
+  @Override
+  public boolean renameData(URI destURI) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support renaming data");
+  }
+
+  @Override
+  public boolean getPinning(LocalFileSystem localFS) throws IOException {
+    return false;
+  }
+
+  @Override
+  public void setPinning(LocalFileSystem localFS) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support pinning");
+  }
+
+  @Override
+  public void bumpReplicaGS(long newGS) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support writes");
+  }
+
+  @Override
+  public boolean breakHardLinksIfNeeded() throws IOException {
+    return false;
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo()
+      throws UnsupportedOperationException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support writes");
+  }
+
+  @Override
+  public int compareWith(ScanInfo info) {
+    //local scanning cannot find any provided blocks.
+    if (info.getFileRegion().equals(
+        new FileRegion(this.getBlockId(), new Path(fileURI),
+            fileOffset, this.getNumBytes(), this.getGenerationStamp()))) {
+      return 0;
+    } else {
+      return (int) (info.getBlockLength() - getNumBytes());
+    }
+  }
+
+  @Override
+  public void truncateBlock(long newLength) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support truncate");
+  }
+
+  @Override
+  public void updateWithReplica(StorageLocation replicaLocation) {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support update");
+  }
+
+  @Override
+  public void copyMetadata(URI destination) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support copy metadata");
+  }
+
+  @Override
+  public void copyBlockdata(URI destination) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support copy data");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
index 280aaa0..639467f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
@@ -18,9 +18,13 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
+import java.net.URI;
 
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 
 /**
@@ -42,11 +46,20 @@ public class ReplicaBuilder {
 
   private ReplicaInfo fromReplica;
 
+  private URI uri;
+  private long offset;
+  private Configuration conf;
+  private FileRegion fileRegion;
+
   public ReplicaBuilder(ReplicaState state) {
     volume = null;
     writer = null;
     block = null;
     length = -1;
+    fileRegion = null;
+    conf = null;
+    fromReplica = null;
+    uri = null;
     this.state = state;
   }
 
@@ -105,6 +118,26 @@ public class ReplicaBuilder {
     return this;
   }
 
+  public ReplicaBuilder setURI(URI uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  public ReplicaBuilder setConf(Configuration conf) {
+    this.conf = conf;
+    return this;
+  }
+
+  public ReplicaBuilder setOffset(long offset) {
+    this.offset = offset;
+    return this;
+  }
+
+  public ReplicaBuilder setFileRegion(FileRegion fileRegion) {
+    this.fileRegion = fileRegion;
+    return this;
+  }
+
   public LocalReplicaInPipeline buildLocalReplicaInPipeline()
       throws IllegalArgumentException {
     LocalReplicaInPipeline info = null;
@@ -176,7 +209,7 @@ public class ReplicaBuilder {
     }
   }
 
-  private ReplicaInfo buildFinalizedReplica() throws IllegalArgumentException {
+  private LocalReplica buildFinalizedReplica() throws IllegalArgumentException {
     if (null != fromReplica &&
         fromReplica.getState() == ReplicaState.FINALIZED) {
       return new FinalizedReplica((FinalizedReplica)fromReplica);
@@ -193,7 +226,7 @@ public class ReplicaBuilder {
     }
   }
 
-  private ReplicaInfo buildRWR() throws IllegalArgumentException {
+  private LocalReplica buildRWR() throws IllegalArgumentException {
 
     if (null != fromReplica && fromReplica.getState() == ReplicaState.RWR) {
       return new ReplicaWaitingToBeRecovered(
@@ -211,7 +244,7 @@ public class ReplicaBuilder {
     }
   }
 
-  private ReplicaInfo buildRUR() throws IllegalArgumentException {
+  private LocalReplica buildRUR() throws IllegalArgumentException {
     if (null == fromReplica) {
       throw new IllegalArgumentException(
           "Missing a valid replica to recover from");
@@ -228,8 +261,53 @@ public class ReplicaBuilder {
     }
   }
 
-  public ReplicaInfo build() throws IllegalArgumentException {
-    ReplicaInfo info = null;
+  private ProvidedReplica buildProvidedFinalizedReplica()
+      throws IllegalArgumentException {
+    ProvidedReplica info = null;
+    if (fromReplica != null) {
+      throw new IllegalArgumentException("Finalized PROVIDED replica " +
+          "cannot be constructed from another replica");
+    }
+    if (fileRegion == null && uri == null) {
+      throw new IllegalArgumentException(
+          "Trying to construct a provided replica on " + volume +
+          " without enough information");
+    }
+    if (fileRegion == null) {
+      info = new FinalizedProvidedReplica(blockId, uri, offset,
+          length, genStamp, volume, conf);
+    } else {
+      info = new FinalizedProvidedReplica(fileRegion.getBlock().getBlockId(),
+          fileRegion.getPath().toUri(),
+          fileRegion.getOffset(),
+          fileRegion.getBlock().getNumBytes(),
+          fileRegion.getBlock().getGenerationStamp(),
+          volume, conf);
+    }
+    return info;
+  }
+
+  private ProvidedReplica buildProvidedReplica()
+      throws IllegalArgumentException {
+    ProvidedReplica info = null;
+    switch(this.state) {
+    case FINALIZED:
+      info = buildProvidedFinalizedReplica();
+      break;
+    case RWR:
+    case RUR:
+    case RBW:
+    case TEMPORARY:
+    default:
+      throw new IllegalArgumentException("Unknown replica state " +
+          state + " for PROVIDED replica");
+    }
+    return info;
+  }
+
+  private LocalReplica buildLocalReplica()
+      throws IllegalArgumentException {
+    LocalReplica info = null;
     switch(this.state) {
     case FINALIZED:
       info = buildFinalizedReplica();
@@ -249,4 +327,16 @@ public class ReplicaBuilder {
     }
     return info;
   }
+
+  public ReplicaInfo build() throws IllegalArgumentException {
+
+    ReplicaInfo info = null;
+    if(volume != null && volume.getStorageType() == StorageType.PROVIDED) {
+      info = buildProvidedReplica();
+    } else {
+      info = buildLocalReplica();
+    }
+
+    return info;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
index 65e9ba7..3718799 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
@@ -50,6 +50,17 @@ abstract public class ReplicaInfo extends Block
       new FileIoProvider(null, null);
 
   /**
+   * Constructor.
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaInfo(Block block, FsVolumeSpi vol) {
+    this(vol, block.getBlockId(), block.getNumBytes(),
+        block.getGenerationStamp());
+  }
+
+  /**
   * Constructor
   * @param vol volume where replica is located
   * @param blockId block id
@@ -62,7 +73,14 @@ abstract public class ReplicaInfo extends Block
   }
   
   /**
-   * Get the volume where this replica is located on disk.
+   * Copy constructor.
+   * @param from where to copy from
+   */
+  ReplicaInfo(ReplicaInfo from) {
+    this(from, from.getVolume());
+  }
+
+  /**
    * @return the volume where this replica is located on disk
    */
   public FsVolumeSpi getVolume() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
index b4d5794..fb7acfd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
@@ -98,6 +98,16 @@ public class StorageLocation
 
   public boolean matchesStorageDirectory(StorageDirectory sd,
       String bpid) throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED &&
+        storageType == StorageType.PROVIDED) {
+      return matchesStorageDirectory(sd);
+    }
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED ||
+        storageType == StorageType.PROVIDED) {
+      //only one of these is PROVIDED; so it cannot be a match!
+      return false;
+    }
+    //both storage directories are local
     return this.getBpURI(bpid, Storage.STORAGE_DIR_CURRENT).normalize()
         .equals(sd.getRoot().toURI().normalize());
   }
@@ -197,6 +207,10 @@ public class StorageLocation
     if (conf == null) {
       conf = new HdfsConfiguration();
     }
+    if (storageType == StorageType.PROVIDED) {
+      //skip creation if the storage type is PROVIDED
+      return;
+    }
 
     LocalFileSystem localFS = FileSystem.getLocal(conf);
     FsPermission permission = new FsPermission(conf.get(
@@ -213,10 +227,14 @@ public class StorageLocation
 
   @Override  // Checkable
   public VolumeCheckResult check(CheckContext context) throws IOException {
-    DiskChecker.checkDir(
-        context.localFileSystem,
-        new Path(baseURI),
-        context.expectedPermission);
+    //we assume provided storage locations are always healthy,
+    //and check only for local storages.
+    if (storageType != StorageType.PROVIDED) {
+      DiskChecker.checkDir(
+          context.localFileSystem,
+          new Path(baseURI),
+          context.expectedPermission);
+    }
     return VolumeCheckResult.HEALTHY;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 7be42e8..f4bf839 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
@@ -252,8 +253,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * and, in case that they are not matched, update the record or mark it
    * as corrupted.
    */
-  void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) throws IOException;
+  void checkAndUpdate(String bpid, ScanInfo info) throws IOException;
 
   /**
    * @param b - the block


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


[09/50] [abbrv] hadoop git commit: HDFS-11703. [READ] Tests for ProvidedStorageMap

Posted by vi...@apache.org.
HDFS-11703. [READ] Tests for ProvidedStorageMap


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

Branch: refs/heads/HDFS-9806
Commit: e0793a78212aea18ffb6ec55ea287982cfe7cb8b
Parents: 08f6b2d
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu May 4 13:14:41 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

----------------------------------------------------------------------
 .../blockmanagement/ProvidedStorageMap.java     |   6 +
 .../blockmanagement/TestProvidedStorageMap.java | 153 +++++++++++++++++++
 2 files changed, 159 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0793a78/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index 518b7e9..0faf16d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -28,6 +28,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentSkipListMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -121,6 +122,11 @@ public class ProvidedStorageMap {
     return dn.getStorageInfo(s.getStorageID());
   }
 
+  @VisibleForTesting
+  public DatanodeStorageInfo getProvidedStorageInfo() {
+    return providedStorageInfo;
+  }
+
   public LocatedBlockBuilder newLocatedBlocks(int maxValue) {
     if (!providedEnabled) {
       return new LocatedBlockBuilder(maxValue);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0793a78/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
new file mode 100644
index 0000000..50e2fed
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
@@ -0,0 +1,153 @@
+/**
+ * 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 org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.util.RwLock;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * This class tests the {@link ProvidedStorageMap}.
+ */
+public class TestProvidedStorageMap {
+
+  private Configuration conf;
+  private BlockManager bm;
+  private RwLock nameSystemLock;
+  private String providedStorageID;
+
+  static class TestBlockProvider extends BlockProvider
+          implements Configurable {
+
+    @Override
+    public void setConf(Configuration conf) {
+    }
+
+    @Override
+    public Configuration getConf() {
+      return null;
+    }
+
+    @Override
+    public Iterator<Block> iterator() {
+      return new Iterator<Block>() {
+        @Override
+        public boolean hasNext() {
+          return false;
+        }
+        @Override
+        public Block next() {
+          return null;
+        }
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+  }
+
+  @Before
+  public void setup() {
+    providedStorageID = DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT;
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+            providedStorageID);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
+    conf.setClass(DFSConfigKeys.DFS_NAMENODE_BLOCK_PROVIDER_CLASS,
+            TestBlockProvider.class, BlockProvider.class);
+
+    bm = mock(BlockManager.class);
+    nameSystemLock = mock(RwLock.class);
+  }
+
+  private DatanodeDescriptor createDatanodeDescriptor(int port) {
+    return DFSTestUtil.getDatanodeDescriptor("127.0.0.1", port, "defaultRack",
+            "localhost");
+  }
+
+  @Test
+  public void testProvidedStorageMap() throws IOException {
+    ProvidedStorageMap providedMap = new ProvidedStorageMap(
+            nameSystemLock, bm, conf);
+    DatanodeStorageInfo providedMapStorage =
+            providedMap.getProvidedStorageInfo();
+    //the provided storage cannot be null
+    assertNotNull(providedMapStorage);
+
+    //create a datanode
+    DatanodeDescriptor dn1 = createDatanodeDescriptor(5000);
+
+    //associate two storages to the datanode
+    DatanodeStorage dn1ProvidedStorage = new DatanodeStorage(
+            providedStorageID,
+            DatanodeStorage.State.NORMAL,
+            StorageType.PROVIDED);
+    DatanodeStorage dn1DiskStorage = new DatanodeStorage(
+            "sid-1", DatanodeStorage.State.NORMAL, StorageType.DISK);
+
+    when(nameSystemLock.hasWriteLock()).thenReturn(true);
+    DatanodeStorageInfo dns1Provided = providedMap.getStorage(dn1,
+            dn1ProvidedStorage);
+    DatanodeStorageInfo dns1Disk = providedMap.getStorage(dn1,
+            dn1DiskStorage);
+
+    assertTrue("The provided storages should be equal",
+            dns1Provided == providedMapStorage);
+    assertTrue("Disk storage has not yet been registered with block manager",
+            dns1Disk == null);
+    //add the disk storage to the datanode.
+    DatanodeStorageInfo dnsDisk = new DatanodeStorageInfo(dn1, dn1DiskStorage);
+    dn1.injectStorage(dnsDisk);
+    assertTrue("Disk storage must match the injected storage info",
+            dnsDisk == providedMap.getStorage(dn1, dn1DiskStorage));
+
+    //create a 2nd datanode
+    DatanodeDescriptor dn2 = createDatanodeDescriptor(5010);
+    //associate a provided storage with the datanode
+    DatanodeStorage dn2ProvidedStorage = new DatanodeStorage(
+            providedStorageID,
+            DatanodeStorage.State.NORMAL,
+            StorageType.PROVIDED);
+
+    DatanodeStorageInfo dns2Provided = providedMap.getStorage(
+            dn2, dn2ProvidedStorage);
+    assertTrue("The provided storages should be equal",
+            dns2Provided == providedMapStorage);
+    assertTrue("The DatanodeDescriptor should contain the provided storage",
+            dn2.getStorageInfo(providedStorageID) == providedMapStorage);
+
+
+  }
+}


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


[16/50] [abbrv] hadoop git commit: HDFS-11653. [READ] ProvidedReplica should return an InputStream that is bounded by its length

Posted by vi...@apache.org.
HDFS-11653. [READ] ProvidedReplica should return an InputStream that is bounded by its length


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

Branch: refs/heads/HDFS-9806
Commit: 081e0c056f529bb539ba7b7341a3ae40776d4ebe
Parents: 3fee8ad
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu May 4 12:43:48 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/ProvidedReplica.java   |   5 +-
 .../datanode/TestProvidedReplicaImpl.java       | 163 +++++++++++++++++++
 2 files changed, 167 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/081e0c05/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
index b021ea2..946ab5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
+
+import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -98,7 +100,8 @@ public abstract class ProvidedReplica extends ReplicaInfo {
     if (remoteFS != null) {
       FSDataInputStream ins = remoteFS.open(new Path(fileURI));
       ins.seek(fileOffset + seekOffset);
-      return new FSDataInputStream(ins);
+      return new BoundedInputStream(
+          new FSDataInputStream(ins), getBlockDataLength());
     } else {
       throw new IOException("Remote filesystem for provided replica " + this +
           " does not exist");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/081e0c05/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
new file mode 100644
index 0000000..8258c21
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.input.BoundedInputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests the implementation of {@link ProvidedReplica}.
+ */
+public class TestProvidedReplicaImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestProvidedReplicaImpl.class);
+  private static final String BASE_DIR =
+      new FileSystemTestHelper().getTestRootDir();
+  private static final String FILE_NAME = "provided-test";
+  //length of the file that is associated with the provided blocks.
+  private static final long FILE_LEN = 128 * 1024 * 10L + 64 * 1024;
+  //length of each provided block.
+  private static final long BLK_LEN = 128 * 1024L;
+
+  private static List<ProvidedReplica> replicas;
+
+  private static void createFileIfNotExists(String baseDir) throws IOException {
+    File newFile = new File(baseDir, FILE_NAME);
+    newFile.getParentFile().mkdirs();
+    if(!newFile.exists()) {
+      newFile.createNewFile();
+      OutputStream writer = new FileOutputStream(newFile.getAbsolutePath());
+      //FILE_LEN is length in bytes.
+      byte[] bytes = new byte[1];
+      bytes[0] = (byte) 0;
+      for(int i=0; i< FILE_LEN; i++) {
+        writer.write(bytes);
+      }
+      writer.flush();
+      writer.close();
+      LOG.info("Created provided file " + newFile +
+          " of length " + newFile.length());
+    }
+  }
+
+  private static void createProvidedReplicas(Configuration conf) {
+    long numReplicas = (long) Math.ceil((double) FILE_LEN/BLK_LEN);
+    File providedFile = new File(BASE_DIR, FILE_NAME);
+    replicas = new ArrayList<ProvidedReplica>();
+
+    LOG.info("Creating " + numReplicas + " provided replicas");
+    for (int i=0; i<numReplicas; i++) {
+      long currentReplicaLength =
+          FILE_LEN >= (i+1)*BLK_LEN ? BLK_LEN : FILE_LEN - i*BLK_LEN;
+      replicas.add(
+          new FinalizedProvidedReplica(i, providedFile.toURI(), i*BLK_LEN,
+          currentReplicaLength, 0, null, conf));
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    createFileIfNotExists(new File(BASE_DIR).getAbsolutePath());
+    createProvidedReplicas(new Configuration());
+  }
+
+  /**
+   * Checks if {@code ins} matches the provided file from offset
+   * {@code fileOffset} for length {@ dataLength}.
+   * @param file the local file
+   * @param ins input stream to compare against
+   * @param fileOffset offset
+   * @param dataLength length
+   * @throws IOException
+   */
+  private void verifyReplicaContents(File file,
+      InputStream ins, long fileOffset, long dataLength)
+          throws IOException {
+
+    InputStream fileIns = new FileInputStream(file);
+    fileIns.skip(fileOffset);
+
+    try (ReadableByteChannel i =
+        Channels.newChannel(new BoundedInputStream(fileIns, dataLength))) {
+      try (ReadableByteChannel j = Channels.newChannel(ins)) {
+        ByteBuffer ib = ByteBuffer.allocate(4096);
+        ByteBuffer jb = ByteBuffer.allocate(4096);
+        while (true) {
+          int il = i.read(ib);
+          int jl = j.read(jb);
+          if (il < 0 || jl < 0) {
+            assertEquals(il, jl);
+            break;
+          }
+          ib.flip();
+          jb.flip();
+          int cmp = Math.min(ib.remaining(), jb.remaining());
+          for (int k = 0; k < cmp; ++k) {
+            assertEquals(ib.get(), jb.get());
+          }
+          ib.compact();
+          jb.compact();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testProvidedReplicaRead() throws IOException {
+
+    File providedFile = new File(BASE_DIR, FILE_NAME);
+    for(int i=0; i < replicas.size(); i++) {
+      ProvidedReplica replica = replicas.get(i);
+      //block data should exist!
+      assertTrue(replica.blockDataExists());
+      assertEquals(providedFile.toURI(), replica.getBlockURI());
+      verifyReplicaContents(providedFile, replica.getDataInputStream(0),
+          BLK_LEN*i, replica.getBlockDataLength());
+    }
+    LOG.info("All replica contents verified");
+
+    providedFile.delete();
+    //the block data should no longer be found!
+    for(int i=0; i < replicas.size(); i++) {
+      ProvidedReplica replica = replicas.get(i);
+      assertTrue(!replica.blockDataExists());
+    }
+  }
+
+}


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


[42/50] [abbrv] hadoop git commit: HDFS-12894. [READ] Skip setting block count of ProvidedDatanodeStorageInfo on DN registration update

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


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

Branch: refs/heads/HDFS-9806
Commit: 27ce8136032c1ece21f2f36cc21cd110ae703bdf
Parents: 31b56cd
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Dec 5 17:55:32 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:34 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/27ce8136/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index f92c4e8..916cbaa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -4943,4 +4943,9 @@ public class BlockManager implements BlockStatsMXBean {
   public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
     pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
   }
+
+  @VisibleForTesting
+  public ProvidedStorageMap getProvidedStorageMap() {
+    return providedStorageMap;
+  }
 }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27ce8136/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index deaf9d5..d057247 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -559,7 +559,9 @@ public class TestNameNodeProvidedImplementation {
     DataNode providedDatanode2 = cluster.getDataNodes().get(1);
 
     DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-            cluster.getNameNodePort()), cluster.getConfiguration(0));
+        cluster.getNameNodePort()), cluster.getConfiguration(0));
+
+    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
 
     if (numFiles >= 1) {
       String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
@@ -596,10 +598,15 @@ public class TestNameNodeProvidedImplementation {
           providedDatanode2.getDatanodeId().getXferAddr());
       getAndCheckBlockLocations(client, filename, baseFileLen, 1, 0);
 
+      // BR count for the provided ProvidedDatanodeStorageInfo should reset to
+      // 0, when all DNs with PROVIDED storage fail.
+      assertEquals(0, providedDNInfo.getBlockReportCount());
       //restart the provided datanode
       cluster.restartDataNode(providedDNProperties1, true);
       cluster.waitActive();
 
+      assertEquals(1, providedDNInfo.getBlockReportCount());
+
       //should find the block on the 1st provided datanode now
       dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
       //not comparing UUIDs as the datanode can now have a different one.
@@ -621,6 +628,8 @@ public class TestNameNodeProvidedImplementation {
         false);
 
     DataNode providedDatanode = cluster.getDataNodes().get(0);
+    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
+    int initialBRCount = providedDNInfo.getBlockReportCount();
     for (int i= 0; i < numFiles; i++) {
       // expect to have 2 locations as we have 2 provided Datanodes.
       verifyFileLocation(i, 2);
@@ -631,10 +640,19 @@ public class TestNameNodeProvidedImplementation {
       cluster.waitActive();
       cluster.triggerHeartbeats();
       Thread.sleep(1000);
+      // the report count should just continue to increase.
+      assertEquals(initialBRCount + i + 1,
+          providedDNInfo.getBlockReportCount());
       verifyFileLocation(i, 2);
     }
   }
 
+  private DatanodeStorageInfo getProvidedDatanodeStorageInfo() {
+    ProvidedStorageMap providedStorageMap =
+        cluster.getNamesystem().getBlockManager().getProvidedStorageMap();
+    return providedStorageMap.getProvidedStorageInfo();
+  }
+
   @Test(timeout=30000)
   public void testNamenodeRestart() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: fa33176064fd9fb5768434957f2a1ffe317bb157
Parents: 6c094a7
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Aug 14 10:29:47 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa331760/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 60b306f..3f937c4 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -74,7 +74,7 @@ public class TestNameNodeProvidedImplementation {
   final Random r = new Random();
   final File fBASE = new File(MiniDFSCluster.getBaseDirectory());
   final Path BASE = new Path(fBASE.toURI().toString());
-  final Path NAMEPATH = new Path(BASE, "providedDir");;
+  final Path NAMEPATH = new Path(BASE, "providedDir");
   final Path NNDIRPATH = new Path(BASE, "nnDir");
   final Path BLOCKFILE = new Path(NNDIRPATH, "blocks.csv");
   final String SINGLEUSER = "usr1";
@@ -116,6 +116,8 @@ public class TestNameNodeProvidedImplementation {
         BLOCKFILE.toString());
     conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER, ",");
 
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
+        new File(NAMEPATH.toUri()).toString());
     File imageDir = new File(NAMEPATH.toUri());
     if (!imageDir.exists()) {
       LOG.info("Creating directory: " + imageDir);


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


[38/50] [abbrv] hadoop git commit: HDFS-12713. [READ] Refactor FileRegion and BlockAliasMap to separate out HDFS metadata and PROVIDED storage metadata. Contributed by Ewan Higgs

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


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

Branch: refs/heads/HDFS-9806
Commit: 31b56cd6050007fd16ec6f29fc2b65bda098198c
Parents: 63a2e7c
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Dec 5 13:46:30 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:29 2017 -0800

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


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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
index f7be111..3d9eeea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
@@ -121,7 +121,7 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
         Block block = fromBlockBytes(entry.getKey());
         ProvidedStorageLocation providedStorageLocation =
             fromProvidedStorageLocationBytes(entry.getValue());
-        batch.add(new FileRegion(block, providedStorageLocation, null));
+        batch.add(new FileRegion(block, providedStorageLocation));
         ++i;
       }
       if (iterator.hasNext()) {
@@ -157,6 +157,11 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     levelDb.put(extendedBlockDbFormat, providedStorageLocationDbFormat);
   }
 
+  @Override
+  public String getBlockPoolId() {
+    return null;
+  }
+
   public void close() throws IOException {
     levelDb.close();
   }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
index 197e663..a229ae7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
@@ -55,11 +55,13 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   private RPC.Server aliasMapServer;
   private Configuration conf;
   private InMemoryAliasMap aliasMap;
+  private String blockPoolId;
 
   public InMemoryLevelDBAliasMapServer(
-      CheckedFunction<Configuration, InMemoryAliasMap> initFun) {
+      CheckedFunction<Configuration, InMemoryAliasMap> initFun,
+      String blockPoolId) {
     this.initFun = initFun;
-
+    this.blockPoolId = blockPoolId;
   }
 
   public void start() throws IOException {
@@ -92,7 +94,7 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
         .setVerbose(true)
         .build();
 
-    LOG.info("Starting InMemoryLevelDBAliasMapServer on ", rpcAddress);
+    LOG.info("Starting InMemoryLevelDBAliasMapServer on {}", rpcAddress);
     aliasMapServer.start();
   }
 
@@ -117,6 +119,11 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   }
 
   @Override
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+
+  @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index f6b0c0b..7fbc71a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -152,8 +152,12 @@ public class ProvidedStorageMap {
       LOG.info("Calling process first blk report from storage: "
           + providedStorageInfo);
       // first pass; periodic refresh should call bm.processReport
-      bm.processFirstBlockReport(providedStorageInfo,
-          new ProvidedBlockList(aliasMap.getReader(null).iterator()));
+      BlockAliasMap.Reader<BlockAlias> reader =
+          aliasMap.getReader(null, bm.getBlockPoolId());
+      if (reader != null) {
+        bm.processFirstBlockReport(providedStorageInfo,
+                new ProvidedBlockList(reader.iterator()));
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
index c2697c8..e6f0d0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
@@ -34,39 +34,21 @@ import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 public class FileRegion implements BlockAlias {
 
   private final Pair<Block, ProvidedStorageLocation> pair;
-  private final String bpid;
 
   public FileRegion(long blockId, Path path, long offset,
-      long length, String bpid, long genStamp) {
+      long length, long genStamp) {
     this(new Block(blockId, length, genStamp),
-        new ProvidedStorageLocation(path, offset, length, new byte[0]), bpid);
+        new ProvidedStorageLocation(path, offset, length, new byte[0]));
   }
 
-  public FileRegion(long blockId, Path path, long offset,
-      long length, String bpid) {
-    this(blockId, path, offset, length, bpid,
+  public FileRegion(long blockId, Path path, long offset, long length) {
+    this(blockId, path, offset, length,
         HdfsConstants.GRANDFATHER_GENERATION_STAMP);
   }
 
-  public FileRegion(long blockId, Path path, long offset,
-      long length, long genStamp) {
-    this(blockId, path, offset, length, null, genStamp);
-  }
-
   public FileRegion(Block block,
       ProvidedStorageLocation providedStorageLocation) {
     this.pair  = Pair.of(block, providedStorageLocation);
-    this.bpid = null;
-  }
-
-  public FileRegion(Block block,
-      ProvidedStorageLocation providedStorageLocation, String bpid) {
-    this.pair  = Pair.of(block, providedStorageLocation);
-    this.bpid = bpid;
-  }
-
-  public FileRegion(long blockId, Path path, long offset, long length) {
-    this(blockId, path, offset, length, null);
   }
 
   public Block getBlock() {
@@ -77,10 +59,6 @@ public class FileRegion implements BlockAlias {
     return pair.getValue();
   }
 
-  public String getBlockPoolId() {
-    return this.bpid;
-  }
-
   @Override
   public boolean equals(Object o) {
     if (this == o) {

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
index faf1f83..b419c38 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
@@ -44,6 +44,7 @@ public class TestProvidedStorageMap {
   private BlockManager bm;
   private RwLock nameSystemLock;
   private String providedStorageID;
+  private String blockPoolID;
 
   @Before
   public void setup() {
@@ -55,8 +56,9 @@ public class TestProvidedStorageMap {
     conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
         TestProvidedImpl.TestFileRegionBlockAliasMap.class,
         BlockAliasMap.class);
-
+    blockPoolID = "BP-12344-10.1.1.2-12344";
     bm = mock(BlockManager.class);
+    when(bm.getBlockPoolId()).thenReturn(blockPoolID);
     nameSystemLock = mock(RwLock.class);
   }
 

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
index 8bdbaa4..1a89f76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -150,7 +150,7 @@ public class TestProvidedImpl {
           }
         }
         region = new FileRegion(currentCount, new Path(newFile.toString()),
-            0, BLK_LEN, BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+            0, BLK_LEN);
         currentCount++;
       }
       return region;
@@ -194,9 +194,12 @@ public class TestProvidedImpl {
     }
 
     @Override
-    public Reader<FileRegion> getReader(Reader.Options opts)
+    public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolId)
         throws IOException {
 
+      if (!blockPoolId.equals(BLOCK_POOL_IDS[CHOSEN_BP_ID])) {
+        return null;
+      }
       BlockAliasMap.Reader<FileRegion> reader =
           new BlockAliasMap.Reader<FileRegion>() {
             @Override
@@ -224,7 +227,7 @@ public class TestProvidedImpl {
     }
 
     @Override
-    public Writer<FileRegion> getWriter(Writer.Options opts)
+    public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolId)
         throws IOException {
       // not implemented
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
index 4598e9c..80bbaf9 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
@@ -73,6 +73,7 @@ public class FileSystemImage implements Tool {
     options.addOption("i", "blockidclass", true, "Block resolver class");
     options.addOption("c", "cachedirs", true, "Max active dirents");
     options.addOption("cid", "clusterID", true, "Cluster ID");
+    options.addOption("bpid", "blockPoolID", true, "Block Pool ID");
     options.addOption("h", "help", false, "Print usage");
     return options;
   }
@@ -120,6 +121,9 @@ public class FileSystemImage implements Tool {
       case "cid":
         opts.clusterID(o.getValue());
         break;
+      case "bpid":
+        opts.blockPoolID(o.getValue());
+        break;
       default:
         throw new UnsupportedOperationException("Internal error");
       }

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
index 8f1382a..aca1220 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
@@ -107,8 +107,7 @@ public class TreePath {
       long genStamp, String blockPoolID,
       BlockAliasMap.Writer<FileRegion> out) throws IOException {
     FileStatus s = getFileStatus();
-    out.store(new FileRegion(blockId, s.getPath(), offset, length,
-        blockPoolID, genStamp));
+    out.store(new FileRegion(blockId, s.getPath(), offset, length, genStamp));
   }
 
   INode toFile(UGIResolver ugi, BlockResolver blk,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b56cd6/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 70e4c33..deaf9d5 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.BlockMissingException;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -79,6 +78,7 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
 import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
 import static org.junit.Assert.*;
 
@@ -93,7 +93,6 @@ public class TestNameNodeProvidedImplementation {
   final Path BASE = new Path(fBASE.toURI().toString());
   final Path NAMEPATH = new Path(BASE, "providedDir");
   final Path NNDIRPATH = new Path(BASE, "nnDir");
-  final Path BLOCKFILE = new Path(NNDIRPATH, "blocks.csv");
   final String SINGLEUSER = "usr1";
   final String SINGLEGROUP = "grp1";
   private final int numFiles = 10;
@@ -101,6 +100,7 @@ public class TestNameNodeProvidedImplementation {
   private final String fileSuffix = ".dat";
   private final int baseFileLen = 1024;
   private long providedDataSize = 0;
+  private final String bpid = "BP-1234-10.1.1.1-1224";
 
   Configuration conf;
   MiniDFSCluster cluster;
@@ -123,10 +123,10 @@ public class TestNameNodeProvidedImplementation {
 
     conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
         TextFileRegionAliasMap.class, BlockAliasMap.class);
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH,
-        BLOCKFILE.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
-        BLOCKFILE.toString());
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR,
+        NNDIRPATH.toString());
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE,
+        new Path(NNDIRPATH, fileNameFromBlockPoolID(bpid)).toString());
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, ",");
 
     conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
@@ -189,14 +189,14 @@ public class TestNameNodeProvidedImplementation {
     opts.output(out.toString())
         .blocks(aliasMapClass)
         .blockIds(blockIdsClass)
-        .clusterID(clusterID);
+        .clusterID(clusterID)
+        .blockPoolID(bpid);
     try (ImageWriter w = new ImageWriter(opts)) {
       for (TreePath e : t) {
         w.accept(e);
       }
     }
   }
-
   void startCluster(Path nspath, int numDatanodes,
       StorageType[] storageTypes,
       StorageType[][] storageTypesPerDatanode,
@@ -743,9 +743,7 @@ public class TestNameNodeProvidedImplementation {
   }
 
 
-  // This test will fail until there is a refactoring of the FileRegion
-  // (HDFS-12713).
-  @Test(expected=BlockMissingException.class)
+  @Test
   public void testInMemoryAliasMap() throws Exception {
     conf.setClass(ImageWriter.Options.UGI_CLASS,
         FsUGIResolver.class, UGIResolver.class);
@@ -758,9 +756,9 @@ public class TestNameNodeProvidedImplementation {
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
         tempDirectory.getAbsolutePath());
     conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
-
+    conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
     InMemoryLevelDBAliasMapServer levelDBAliasMapServer =
-        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init);
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, bpid);
     levelDBAliasMapServer.setConf(conf);
     levelDBAliasMapServer.start();
 


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


[15/50] [abbrv] hadoop git commit: HDFS-10706. [READ] Add tool generating FSImage from external store

Posted by vi...@apache.org.
HDFS-10706. [READ] Add tool generating FSImage from external store


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

Branch: refs/heads/HDFS-9806
Commit: fc7f2d37b16e6c1e176ebafe628bd216689c7e26
Parents: 02a28b9
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Sat Apr 15 12:15:08 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-fs2img/pom.xml              |  87 +++
 .../hdfs/server/namenode/BlockResolver.java     |  95 +++
 .../hadoop/hdfs/server/namenode/FSTreeWalk.java | 105 ++++
 .../hdfs/server/namenode/FileSystemImage.java   | 139 +++++
 .../FixedBlockMultiReplicaResolver.java         |  44 ++
 .../server/namenode/FixedBlockResolver.java     |  93 +++
 .../hdfs/server/namenode/FsUGIResolver.java     |  58 ++
 .../hdfs/server/namenode/ImageWriter.java       | 600 +++++++++++++++++++
 .../hdfs/server/namenode/NullBlockFormat.java   |  87 +++
 .../hdfs/server/namenode/SingleUGIResolver.java |  90 +++
 .../hadoop/hdfs/server/namenode/TreePath.java   | 167 ++++++
 .../hadoop/hdfs/server/namenode/TreeWalk.java   | 103 ++++
 .../hdfs/server/namenode/UGIResolver.java       | 131 ++++
 .../hdfs/server/namenode/package-info.java      |  23 +
 .../hdfs/server/namenode/RandomTreeWalk.java    | 186 ++++++
 .../server/namenode/TestFixedBlockResolver.java | 121 ++++
 .../server/namenode/TestRandomTreeWalk.java     | 130 ++++
 .../server/namenode/TestSingleUGIResolver.java  | 148 +++++
 .../src/test/resources/log4j.properties         |  24 +
 hadoop-tools/hadoop-tools-dist/pom.xml          |   6 +
 hadoop-tools/pom.xml                            |   1 +
 21 files changed, 2438 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/pom.xml b/hadoop-tools/hadoop-fs2img/pom.xml
new file mode 100644
index 0000000..36096b7
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/pom.xml
@@ -0,0 +1,87 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha3-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-fs2img</artifactId>
+  <version>3.0.0-alpha3-SNAPSHOT</version>
+  <description>fs2img</description>
+  <name>fs2img</name>
+  <packaging>jar</packaging>
+
+  <properties>
+    <hadoop.log.dir>${project.build.directory}/log</hadoop.log.dir>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+         <configuration>
+          <archive>
+           <manifest>
+            <mainClass>org.apache.hadoop.hdfs.server.namenode.FileSystemImage</mainClass>
+           </manifest>
+         </archive>
+        </configuration>
+       </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
new file mode 100644
index 0000000..94b92b8
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+
+/**
+ * Given an external reference, create a sequence of blocks and associated
+ * metadata.
+ */
+public abstract class BlockResolver {
+
+  protected BlockProto buildBlock(long blockId, long bytes) {
+    return buildBlock(blockId, bytes, 1001);
+  }
+
+  protected BlockProto buildBlock(long blockId, long bytes, long genstamp) {
+    BlockProto.Builder b = BlockProto.newBuilder()
+        .setBlockId(blockId)
+        .setNumBytes(bytes)
+        .setGenStamp(genstamp);
+    return b.build();
+  }
+
+  /**
+   * @param s the external reference.
+   * @return sequence of blocks that make up the reference.
+   */
+  public Iterable<BlockProto> resolve(FileStatus s) {
+    List<Long> lengths = blockLengths(s);
+    ArrayList<BlockProto> ret = new ArrayList<>(lengths.size());
+    long tot = 0;
+    for (long l : lengths) {
+      tot += l;
+      ret.add(buildBlock(nextId(), l));
+    }
+    if (tot != s.getLen()) {
+      // log a warning?
+      throw new IllegalStateException(
+          "Expected " + s.getLen() + " found " + tot);
+    }
+    return ret;
+  }
+
+  /**
+   * @return the next block id.
+   */
+  public abstract long nextId();
+
+  /**
+   * @return the maximum sequentially allocated block ID for this filesystem.
+   */
+  protected abstract long lastId();
+
+  /**
+   * @param status the external reference.
+   * @return the lengths of the resultant blocks.
+   */
+  protected abstract List<Long> blockLengths(FileStatus status);
+
+
+  /**
+   * @param status the external reference.
+   * @return the block size to assign to this external reference.
+   */
+  public long preferredBlockSize(FileStatus status) {
+    return status.getBlockSize();
+  }
+
+  /**
+   * @param status the external reference.
+   * @return the replication to assign to this external reference.
+   */
+  public abstract int getReplication(FileStatus status);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
new file mode 100644
index 0000000..f736112
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.ConcurrentModificationException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Traversal of an external FileSystem.
+ */
+public class FSTreeWalk extends TreeWalk {
+
+  private final Path root;
+  private final FileSystem fs;
+
+  public FSTreeWalk(Path root, Configuration conf) throws IOException {
+    this.root = root;
+    fs = root.getFileSystem(conf);
+  }
+
+  @Override
+  protected Iterable<TreePath> getChildren(TreePath path, long id,
+      TreeIterator i) {
+    // TODO symlinks
+    if (!path.getFileStatus().isDirectory()) {
+      return Collections.emptyList();
+    }
+    try {
+      ArrayList<TreePath> ret = new ArrayList<>();
+      for (FileStatus s : fs.listStatus(path.getFileStatus().getPath())) {
+        ret.add(new TreePath(s, id, i));
+      }
+      return ret;
+    } catch (FileNotFoundException e) {
+      throw new ConcurrentModificationException("FS modified");
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  class FSTreeIterator extends TreeIterator {
+
+    private FSTreeIterator() {
+    }
+
+    FSTreeIterator(TreePath p) {
+      getPendingQueue().addFirst(
+          new TreePath(p.getFileStatus(), p.getParentId(), this));
+    }
+
+    FSTreeIterator(Path p) throws IOException {
+      try {
+        FileStatus s = fs.getFileStatus(root);
+        getPendingQueue().addFirst(new TreePath(s, -1L, this));
+      } catch (FileNotFoundException e) {
+        if (p.equals(root)) {
+          throw e;
+        }
+        throw new ConcurrentModificationException("FS modified");
+      }
+    }
+
+    @Override
+    public TreeIterator fork() {
+      if (getPendingQueue().isEmpty()) {
+        return new FSTreeIterator();
+      }
+      return new FSTreeIterator(getPendingQueue().removeFirst());
+    }
+
+  }
+
+  @Override
+  public TreeIterator iterator() {
+    try {
+      return new FSTreeIterator(root);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
new file mode 100644
index 0000000..e1e85c1
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Create FSImage from an external namespace.
+ */
+public class FileSystemImage implements Tool {
+
+  private Configuration conf;
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    // require absolute URI to write anywhere but local
+    FileSystem.setDefaultUri(conf, new File(".").toURI().toString());
+  }
+
+  protected void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("fs2img [OPTIONS] URI", new Options());
+    formatter.setSyntaxPrefix("");
+    formatter.printHelp("Options", options());
+    ToolRunner.printGenericCommandUsage(System.out);
+  }
+
+  static Options options() {
+    Options options = new Options();
+    options.addOption("o", "outdir", true, "Output directory");
+    options.addOption("u", "ugiclass", true, "UGI resolver class");
+    options.addOption("b", "blockclass", true, "Block output class");
+    options.addOption("i", "blockidclass", true, "Block resolver class");
+    options.addOption("c", "cachedirs", true, "Max active dirents");
+    options.addOption("h", "help", false, "Print usage");
+    return options;
+  }
+
+  @Override
+  public int run(String[] argv) throws Exception {
+    Options options = options();
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd;
+    try {
+      cmd = parser.parse(options, argv);
+    } catch (ParseException e) {
+      System.out.println(
+          "Error parsing command-line options: " + e.getMessage());
+      printUsage();
+      return -1;
+    }
+
+    if (cmd.hasOption("h")) {
+      printUsage();
+      return -1;
+    }
+
+    ImageWriter.Options opts =
+        ReflectionUtils.newInstance(ImageWriter.Options.class, getConf());
+    for (Option o : cmd.getOptions()) {
+      switch (o.getOpt()) {
+      case "o":
+        opts.output(o.getValue());
+        break;
+      case "u":
+        opts.ugi(Class.forName(o.getValue()).asSubclass(UGIResolver.class));
+        break;
+      case "b":
+        opts.blocks(
+            Class.forName(o.getValue()).asSubclass(BlockFormat.class));
+        break;
+      case "i":
+        opts.blockIds(
+            Class.forName(o.getValue()).asSubclass(BlockResolver.class));
+        break;
+      case "c":
+        opts.cache(Integer.parseInt(o.getValue()));
+        break;
+      default:
+        throw new UnsupportedOperationException("Internal error");
+      }
+    }
+
+    String[] rem = cmd.getArgs();
+    if (rem.length != 1) {
+      printUsage();
+      return -1;
+    }
+
+    try (ImageWriter w = new ImageWriter(opts)) {
+      for (TreePath e : new FSTreeWalk(new Path(rem[0]), getConf())) {
+        w.accept(e); // add and continue
+      }
+    }
+    return 0;
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int ret = ToolRunner.run(new FileSystemImage(), argv);
+    System.exit(ret);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
new file mode 100644
index 0000000..0c8ce6e
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockMultiReplicaResolver.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+
+/**
+ * Resolver mapping all files to a configurable, uniform blocksize
+ * and replication.
+ */
+public class FixedBlockMultiReplicaResolver extends FixedBlockResolver {
+
+  public static final String REPLICATION =
+      "hdfs.image.writer.resolver.fixed.block.replication";
+
+  private int replication;
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    replication = conf.getInt(REPLICATION, 1);
+  }
+
+  public int getReplication(FileStatus s) {
+    return replication;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
new file mode 100644
index 0000000..8ff9695
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FixedBlockResolver.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+
+/**
+ * Resolver mapping all files to a configurable, uniform blocksize.
+ */
+public class FixedBlockResolver extends BlockResolver implements Configurable {
+
+  public static final String BLOCKSIZE =
+      "hdfs.image.writer.resolver.fixed.block.size";
+  public static final String START_BLOCK =
+      "hdfs.image.writer.resolver.fixed.block.start";
+
+  private Configuration conf;
+  private long blocksize = 256 * (1L << 20);
+  private final AtomicLong blockIds = new AtomicLong(0);
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    blocksize = conf.getLong(BLOCKSIZE, 256 * (1L << 20));
+    blockIds.set(conf.getLong(START_BLOCK, (1L << 30)));
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  protected List<Long> blockLengths(FileStatus s) {
+    ArrayList<Long> ret = new ArrayList<>();
+    if (!s.isFile()) {
+      return ret;
+    }
+    if (0 == s.getLen()) {
+      // the file has length 0; so we will have one block of size 0
+      ret.add(0L);
+      return ret;
+    }
+    int nblocks = (int)((s.getLen() - 1) / blocksize) + 1;
+    for (int i = 0; i < nblocks - 1; ++i) {
+      ret.add(blocksize);
+    }
+    long rem = s.getLen() % blocksize;
+    ret.add(0 == (rem % blocksize) ? blocksize : rem);
+    return ret;
+  }
+
+  @Override
+  public long nextId() {
+    return blockIds.incrementAndGet();
+  }
+
+  @Override
+  public long lastId() {
+    return blockIds.get();
+  }
+
+  @Override
+  public long preferredBlockSize(FileStatus s) {
+    return blocksize;
+  }
+
+  @Override
+  public int getReplication(FileStatus s) {
+    return 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
new file mode 100644
index 0000000..ca16d96
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsUGIResolver.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Dynamically assign ids to users/groups as they appear in the external
+ * filesystem.
+ */
+public class FsUGIResolver extends UGIResolver {
+
+  private int id;
+  private final Set<String> usernames;
+  private final Set<String> groupnames;
+
+  FsUGIResolver() {
+    super();
+    id = 0;
+    usernames = new HashSet<String>();
+    groupnames = new HashSet<String>();
+  }
+
+  @Override
+  public synchronized void addUser(String name) {
+    if (!usernames.contains(name)) {
+      addUser(name, id);
+      id++;
+      usernames.add(name);
+    }
+  }
+
+  @Override
+  public synchronized void addGroup(String name) {
+    if (!groupnames.contains(name)) {
+      addGroup(name, id);
+      id++;
+      groupnames.add(name);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
new file mode 100644
index 0000000..a3603a1
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
@@ -0,0 +1,600 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.OutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.DigestOutputStream;
+import java.security.MessageDigest;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.base.Charsets;
+import com.google.protobuf.CodedOutputStream;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
+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.FilesUnderConstructionSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection.DirEntry;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressorStream;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+
+import static org.apache.hadoop.hdfs.server.namenode.FSImageUtil.MAGIC_HEADER;
+
+/**
+ * Utility crawling an existing hierarchical FileSystem and emitting
+ * a valid FSImage/NN storage.
+ */
+// TODO: generalize to types beyond FileRegion
+public class ImageWriter implements Closeable {
+
+  private static final int ONDISK_VERSION = 1;
+  private static final int LAYOUT_VERSION = -64; // see NameNodeLayoutVersion
+
+  private final Path outdir;
+  private final FileSystem outfs;
+  private final File dirsTmp;
+  private final OutputStream dirs;
+  private final File inodesTmp;
+  private final OutputStream inodes;
+  private final MessageDigest digest;
+  private final FSImageCompression compress;
+  private final long startBlock;
+  private final long startInode;
+  private final UGIResolver ugis;
+  private final BlockFormat.Writer<FileRegion> blocks;
+  private final BlockResolver blockIds;
+  private final Map<Long, DirEntry.Builder> dircache;
+  private final TrackedOutputStream<DigestOutputStream> raw;
+
+  private boolean closed = false;
+  private long curSec;
+  private long curBlock;
+  private final AtomicLong curInode;
+  private final FileSummary.Builder summary = FileSummary.newBuilder()
+      .setOndiskVersion(ONDISK_VERSION)
+      .setLayoutVersion(LAYOUT_VERSION);
+
+  private final String blockPoolID;
+
+  public static Options defaults() {
+    return new Options();
+  }
+
+  @SuppressWarnings("unchecked")
+  public ImageWriter(Options opts) throws IOException {
+    final OutputStream out;
+    if (null == opts.outStream) {
+      FileSystem fs = opts.outdir.getFileSystem(opts.getConf());
+      outfs = (fs instanceof LocalFileSystem)
+          ? ((LocalFileSystem)fs).getRaw()
+          : fs;
+      Path tmp = opts.outdir;
+      if (!outfs.mkdirs(tmp)) {
+        throw new IOException("Failed to create output dir: " + tmp);
+      }
+      try (NNStorage stor = new NNStorage(opts.getConf(),
+          Arrays.asList(tmp.toUri()), Arrays.asList(tmp.toUri()))) {
+        NamespaceInfo info = NNStorage.newNamespaceInfo();
+        if (info.getLayoutVersion() != LAYOUT_VERSION) {
+          throw new IllegalStateException("Incompatible layout " +
+              info.getLayoutVersion() + " (expected " + LAYOUT_VERSION);
+        }
+        stor.format(info);
+        blockPoolID = info.getBlockPoolID();
+      }
+      outdir = new Path(tmp, "current");
+      out = outfs.create(new Path(outdir, "fsimage_0000000000000000000"));
+    } else {
+      // XXX necessary? writing a NNStorage now...
+      outdir = null;
+      outfs = null;
+      out = opts.outStream;
+      blockPoolID = "";
+    }
+    digest = MD5Hash.getDigester();
+    raw = new TrackedOutputStream<>(new DigestOutputStream(
+            new BufferedOutputStream(out), digest));
+    compress = opts.compress;
+    CompressionCodec codec = compress.getImageCodec();
+    if (codec != null) {
+      summary.setCodec(codec.getClass().getCanonicalName());
+    }
+    startBlock = opts.startBlock;
+    curBlock = startBlock;
+    startInode = opts.startInode;
+    curInode = new AtomicLong(startInode);
+    dircache = Collections.synchronizedMap(new DirEntryCache(opts.maxdircache));
+
+    ugis = null == opts.ugis
+        ? ReflectionUtils.newInstance(opts.ugisClass, opts.getConf())
+        : opts.ugis;
+    BlockFormat<FileRegion> fmt = null == opts.blocks
+        ? ReflectionUtils.newInstance(opts.blockFormatClass, opts.getConf())
+        : opts.blocks;
+    blocks = fmt.getWriter(null);
+    blockIds = null == opts.blockIds
+        ? ReflectionUtils.newInstance(opts.blockIdsClass, opts.getConf())
+        : opts.blockIds;
+
+    // create directory and inode sections as side-files.
+    // The details are written to files to avoid keeping them in memory.
+    dirsTmp = File.createTempFile("fsimg_dir", null);
+    dirsTmp.deleteOnExit();
+    dirs = beginSection(new FileOutputStream(dirsTmp));
+    try {
+      inodesTmp = File.createTempFile("fsimg_inode", null);
+      inodesTmp.deleteOnExit();
+      inodes = new FileOutputStream(inodesTmp);
+    } catch (IOException e) {
+      // appropriate to close raw?
+      IOUtils.cleanup(null, raw, dirs);
+      throw e;
+    }
+
+    raw.write(MAGIC_HEADER);
+    curSec = raw.pos;
+    assert raw.pos == MAGIC_HEADER.length;
+  }
+
+  public void accept(TreePath e) throws IOException {
+    assert e.getParentId() < curInode.get();
+    // allocate ID
+    long id = curInode.getAndIncrement();
+    e.accept(id);
+    assert e.getId() < curInode.get();
+    INode n = e.toINode(ugis, blockIds, blocks, blockPoolID);
+    writeInode(n);
+
+    if (e.getParentId() > 0) {
+      // add DirEntry to map, which may page out entries
+      DirEntry.Builder de = DirEntry.newBuilder()
+          .setParent(e.getParentId())
+          .addChildren(e.getId());
+      dircache.put(e.getParentId(), de);
+    }
+  }
+
+  @SuppressWarnings("serial")
+  class DirEntryCache extends LinkedHashMap<Long, DirEntry.Builder> {
+
+    // should cache path to root, not evict LRCached
+    private final int nEntries;
+
+    DirEntryCache(int nEntries) {
+      this.nEntries = nEntries;
+    }
+
+    @Override
+    public DirEntry.Builder put(Long p, DirEntry.Builder b) {
+      DirEntry.Builder e = get(p);
+      if (null == e) {
+        return super.put(p, b);
+      }
+      //merge
+      e.addAllChildren(b.getChildrenList());
+      // not strictly conforming
+      return e;
+    }
+
+    @Override
+    protected boolean removeEldestEntry(Entry<Long, DirEntry.Builder> be) {
+      if (size() > nEntries) {
+        DirEntry d = be.getValue().build();
+        try {
+          writeDirEntry(d);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return true;
+      }
+      return false;
+    }
+  }
+
+  synchronized void writeInode(INode n) throws IOException {
+    n.writeDelimitedTo(inodes);
+  }
+
+  synchronized void writeDirEntry(DirEntry e) throws IOException {
+    e.writeDelimitedTo(dirs);
+  }
+
+  // from FSImageFormatProtobuf... why not just read position from the stream?
+  private static int getOndiskSize(com.google.protobuf.GeneratedMessage s) {
+    return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize())
+        + s.getSerializedSize();
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    for (DirEntry.Builder b : dircache.values()) {
+      DirEntry e = b.build();
+      writeDirEntry(e);
+    }
+    dircache.clear();
+
+    // close side files
+    IOUtils.cleanup(null, dirs, inodes, blocks);
+    if (null == dirs || null == inodes) {
+      // init failed
+      if (raw != null) {
+        raw.close();
+      }
+      return;
+    }
+    try {
+      writeNameSystemSection();
+      writeINodeSection();
+      writeDirSection();
+      writeStringTableSection();
+
+      // write summary directly to raw
+      FileSummary s = summary.build();
+      s.writeDelimitedTo(raw);
+      int length = getOndiskSize(s);
+      byte[] lengthBytes = new byte[4];
+      ByteBuffer.wrap(lengthBytes).asIntBuffer().put(length);
+      raw.write(lengthBytes);
+    } finally {
+      raw.close();
+    }
+    writeMD5("fsimage_0000000000000000000");
+    closed = true;
+  }
+
+  /**
+   * Write checksum for image file. Pulled from MD5Utils/internals. Awkward to
+   * reuse existing tools/utils.
+   */
+  void writeMD5(String imagename) throws IOException {
+    if (null == outdir) {
+      //LOG.warn("Not writing MD5");
+      return;
+    }
+    MD5Hash md5 = new MD5Hash(digest.digest());
+    String digestString = StringUtils.byteToHexString(md5.getDigest());
+    Path chk = new Path(outdir, imagename + ".md5");
+    try (OutputStream out = outfs.create(chk)) {
+      String md5Line = digestString + " *" + imagename + "\n";
+      out.write(md5Line.getBytes(Charsets.UTF_8));
+    }
+  }
+
+  OutputStream beginSection(OutputStream out) throws IOException {
+    CompressionCodec codec = compress.getImageCodec();
+    if (null == codec) {
+      return out;
+    }
+    return codec.createOutputStream(out);
+  }
+
+  void endSection(OutputStream out, SectionName name) throws IOException {
+    CompressionCodec codec = compress.getImageCodec();
+    if (codec != null) {
+      ((CompressorStream)out).finish();
+    }
+    out.flush();
+    long length = raw.pos - curSec;
+    summary.addSections(FileSummary.Section.newBuilder()
+        .setName(name.toString()) // not strictly correct, but name not visible
+        .setOffset(curSec).setLength(length));
+    curSec += length;
+  }
+
+  void writeNameSystemSection() throws IOException {
+    NameSystemSection.Builder b = NameSystemSection.newBuilder()
+        .setGenstampV1(1000)
+        .setGenstampV1Limit(0)
+        .setGenstampV2(1001)
+        .setLastAllocatedBlockId(blockIds.lastId())
+        .setTransactionId(0);
+    NameSystemSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    endSection(sec, SectionName.NS_INFO);
+  }
+
+  void writeINodeSection() throws IOException {
+    // could reset dict to avoid compression cost in close
+    INodeSection.Builder b = INodeSection.newBuilder()
+        .setNumInodes(curInode.get() - startInode)
+        .setLastInodeId(curInode.get());
+    INodeSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    // copy inodes
+    try (FileInputStream in = new FileInputStream(inodesTmp)) {
+      IOUtils.copyBytes(in, sec, 4096, false);
+    }
+    endSection(sec, SectionName.INODE);
+  }
+
+  void writeDirSection() throws IOException {
+    // No header, so dirs can be written/compressed independently
+    //INodeDirectorySection.Builder b = INodeDirectorySection.newBuilder();
+    OutputStream sec = raw;
+    // copy dirs
+    try (FileInputStream in = new FileInputStream(dirsTmp)) {
+      IOUtils.copyBytes(in, sec, 4096, false);
+    }
+    endSection(sec, SectionName.INODE_DIR);
+  }
+
+  void writeFilesUCSection() throws IOException {
+    FilesUnderConstructionSection.Builder b =
+        FilesUnderConstructionSection.newBuilder();
+    FilesUnderConstructionSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    endSection(sec, SectionName.FILES_UNDERCONSTRUCTION);
+  }
+
+  void writeSnapshotDiffSection() throws IOException {
+    SnapshotDiffSection.Builder b = SnapshotDiffSection.newBuilder();
+    SnapshotDiffSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    endSection(sec, SectionName.SNAPSHOT_DIFF);
+  }
+
+  void writeSecretManagerSection() throws IOException {
+    SecretManagerSection.Builder b = SecretManagerSection.newBuilder()
+        .setCurrentId(0)
+        .setTokenSequenceNumber(0);
+    SecretManagerSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    endSection(sec, SectionName.SECRET_MANAGER);
+  }
+
+  void writeCacheManagerSection() throws IOException {
+    CacheManagerSection.Builder b = CacheManagerSection.newBuilder()
+        .setNumPools(0)
+        .setNumDirectives(0)
+        .setNextDirectiveId(1);
+    CacheManagerSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    endSection(sec, SectionName.CACHE_MANAGER);
+  }
+
+  void writeStringTableSection() throws IOException {
+    StringTableSection.Builder b = StringTableSection.newBuilder();
+    Map<Integer, String> u = ugis.ugiMap();
+    b.setNumEntry(u.size());
+    StringTableSection s = b.build();
+
+    OutputStream sec = beginSection(raw);
+    s.writeDelimitedTo(sec);
+    for (Map.Entry<Integer, String> e : u.entrySet()) {
+      StringTableSection.Entry.Builder x =
+          StringTableSection.Entry.newBuilder()
+              .setId(e.getKey())
+              .setStr(e.getValue());
+      x.build().writeDelimitedTo(sec);
+    }
+    endSection(sec, SectionName.STRING_TABLE);
+  }
+
+  @Override
+  public synchronized String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{ codec=\"").append(compress.getImageCodec());
+    sb.append("\", startBlock=").append(startBlock);
+    sb.append(", curBlock=").append(curBlock);
+    sb.append(", startInode=").append(startInode);
+    sb.append(", curInode=").append(curInode);
+    sb.append(", ugi=").append(ugis);
+    sb.append(", blockIds=").append(blockIds);
+    sb.append(", offset=").append(raw.pos);
+    sb.append(" }");
+    return sb.toString();
+  }
+
+  static class TrackedOutputStream<T extends OutputStream>
+      extends FilterOutputStream {
+
+    private long pos = 0L;
+
+    TrackedOutputStream(T out) {
+      super(out);
+    }
+
+    @SuppressWarnings("unchecked")
+    public T getInner() {
+      return (T) out;
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      out.write(b);
+      ++pos;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+      write(b, 0, b.length);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      out.write(b, off, len);
+      pos += len;
+    }
+
+    @Override
+    public void flush() throws IOException {
+      super.flush();
+    }
+
+    @Override
+    public void close() throws IOException {
+      super.close();
+    }
+
+  }
+
+  /**
+   * Configurable options for image generation mapping pluggable components.
+   */
+  public static class Options implements Configurable {
+
+    public static final String START_INODE = "hdfs.image.writer.start.inode";
+    public static final String CACHE_ENTRY = "hdfs.image.writer.cache.entries";
+    public static final String UGI_CLASS   = "hdfs.image.writer.ugi.class";
+    public static final String BLOCK_RESOLVER_CLASS =
+        "hdfs.image.writer.blockresolver.class";
+
+    private Path outdir;
+    private Configuration conf;
+    private OutputStream outStream;
+    private int maxdircache;
+    private long startBlock;
+    private long startInode;
+    private UGIResolver ugis;
+    private Class<? extends UGIResolver> ugisClass;
+    private BlockFormat<FileRegion> blocks;
+
+    @SuppressWarnings("rawtypes")
+    private Class<? extends BlockFormat> blockFormatClass;
+    private BlockResolver blockIds;
+    private Class<? extends BlockResolver> blockIdsClass;
+    private FSImageCompression compress =
+        FSImageCompression.createNoopCompression();
+
+    protected Options() {
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      //long lastTxn = conf.getLong(LAST_TXN, 0L);
+      String def = new File("hdfs/name").toURI().toString();
+      outdir = new Path(conf.get(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, def));
+      startBlock = conf.getLong(FixedBlockResolver.START_BLOCK, (1L << 30) + 1);
+      startInode = conf.getLong(START_INODE, (1L << 14) + 1);
+      maxdircache = conf.getInt(CACHE_ENTRY, 100);
+      ugisClass = conf.getClass(UGI_CLASS,
+          SingleUGIResolver.class, UGIResolver.class);
+      blockFormatClass = conf.getClass(
+          DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
+          NullBlockFormat.class, BlockFormat.class);
+      blockIdsClass = conf.getClass(BLOCK_RESOLVER_CLASS,
+          FixedBlockResolver.class, BlockResolver.class);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    public Options output(String out) {
+      this.outdir = new Path(out);
+      return this;
+    }
+
+    public Options outStream(OutputStream outStream) {
+      this.outStream = outStream;
+      return this;
+    }
+
+    public Options codec(String codec) throws IOException {
+      this.compress = FSImageCompression.createCompression(getConf(), codec);
+      return this;
+    }
+
+    public Options cache(int nDirEntries) {
+      this.maxdircache = nDirEntries;
+      return this;
+    }
+
+    public Options ugi(UGIResolver ugis) {
+      this.ugis = ugis;
+      return this;
+    }
+
+    public Options ugi(Class<? extends UGIResolver> ugisClass) {
+      this.ugisClass = ugisClass;
+      return this;
+    }
+
+    public Options blockIds(BlockResolver blockIds) {
+      this.blockIds = blockIds;
+      return this;
+    }
+
+    public Options blockIds(Class<? extends BlockResolver> blockIdsClass) {
+      this.blockIdsClass = blockIdsClass;
+      return this;
+    }
+
+    public Options blocks(BlockFormat<FileRegion> blocks) {
+      this.blocks = blocks;
+      return this;
+    }
+
+    @SuppressWarnings("rawtypes")
+    public Options blocks(Class<? extends BlockFormat> blocksClass) {
+      this.blockFormatClass = blocksClass;
+      return this;
+    }
+
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
new file mode 100644
index 0000000..0fd3f2b
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Map all owners/groups in external system to a single user in FSImage.
+ */
+public class SingleUGIResolver extends UGIResolver implements Configurable {
+
+  public static final String UID   = "hdfs.image.writer.ugi.single.uid";
+  public static final String USER  = "hdfs.image.writer.ugi.single.user";
+  public static final String GID   = "hdfs.image.writer.ugi.single.gid";
+  public static final String GROUP = "hdfs.image.writer.ugi.single.group";
+
+  private int uid;
+  private int gid;
+  private String user;
+  private String group;
+  private Configuration conf;
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    uid = conf.getInt(UID, 0);
+    user = conf.get(USER);
+    if (null == user) {
+      try {
+        user = UserGroupInformation.getCurrentUser().getShortUserName();
+      } catch (IOException e) {
+        user = "hadoop";
+      }
+    }
+    gid = conf.getInt(GID, 1);
+    group = conf.get(GROUP);
+    if (null == group) {
+      group = user;
+    }
+
+    resetUGInfo();
+    addUser(user, uid);
+    addGroup(group, gid);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public String user(FileStatus s) {
+    return user;
+  }
+
+  @Override
+  public String group(FileStatus s) {
+    return group;
+  }
+
+  @Override
+  public void addUser(String name) {
+    //do nothing
+  }
+
+  @Override
+  public void addGroup(String name) {
+    //do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
new file mode 100644
index 0000000..14e6bed
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import com.google.protobuf.ByteString;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
+import static org.apache.hadoop.hdfs.DFSUtil.string2Bytes;
+import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.DEFAULT_NAMESPACE_QUOTA;
+import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.DEFAULT_STORAGE_SPACE_QUOTA;
+
+/**
+ * Traversal cursor in external filesystem.
+ * TODO: generalize, move FS/FileRegion to FSTreePath
+ */
+public class TreePath {
+  private long id = -1;
+  private final long parentId;
+  private final FileStatus stat;
+  private final TreeWalk.TreeIterator i;
+
+  protected TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i) {
+    this.i = i;
+    this.stat = stat;
+    this.parentId = parentId;
+  }
+
+  public FileStatus getFileStatus() {
+    return stat;
+  }
+
+  public long getParentId() {
+    return parentId;
+  }
+
+  public long getId() {
+    if (id < 0) {
+      throw new IllegalStateException();
+    }
+    return id;
+  }
+
+  void accept(long id) {
+    this.id = id;
+    i.onAccept(this, id);
+  }
+
+  public INode toINode(UGIResolver ugi, BlockResolver blk,
+      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+          throws IOException {
+    if (stat.isFile()) {
+      return toFile(ugi, blk, out, blockPoolID);
+    } else if (stat.isDirectory()) {
+      return toDirectory(ugi);
+    } else if (stat.isSymlink()) {
+      throw new UnsupportedOperationException("symlinks not supported");
+    } else {
+      throw new UnsupportedOperationException("Unknown type: " + stat);
+    }
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof TreePath)) {
+      return false;
+    }
+    TreePath o = (TreePath) other;
+    return getParentId() == o.getParentId()
+      && getFileStatus().equals(o.getFileStatus());
+  }
+
+  @Override
+  public int hashCode() {
+    long pId = getParentId() * getFileStatus().hashCode();
+    return (int)(pId ^ (pId >>> 32));
+  }
+
+  void writeBlock(long blockId, long offset, long length,
+      long genStamp, String blockPoolID,
+      BlockFormat.Writer<FileRegion> out) throws IOException {
+    FileStatus s = getFileStatus();
+    out.store(new FileRegion(blockId, s.getPath(), offset, length,
+        blockPoolID, genStamp));
+  }
+
+  INode toFile(UGIResolver ugi, BlockResolver blk,
+      BlockFormat.Writer<FileRegion> out, String blockPoolID)
+          throws IOException {
+    final FileStatus s = getFileStatus();
+    // TODO should this store resolver's user/group?
+    ugi.addUser(s.getOwner());
+    ugi.addGroup(s.getGroup());
+    INodeFile.Builder b = INodeFile.newBuilder()
+        .setReplication(blk.getReplication(s))
+        .setModificationTime(s.getModificationTime())
+        .setAccessTime(s.getAccessTime())
+        .setPreferredBlockSize(blk.preferredBlockSize(s))
+        .setPermission(ugi.resolve(s))
+        .setStoragePolicyID(HdfsConstants.PROVIDED_STORAGE_POLICY_ID);
+    //TODO: storage policy should be configurable per path; use BlockResolver
+    long off = 0L;
+    for (BlockProto block : blk.resolve(s)) {
+      b.addBlocks(block);
+      writeBlock(block.getBlockId(), off, block.getNumBytes(),
+          block.getGenStamp(), blockPoolID, out);
+      off += block.getNumBytes();
+    }
+    INode.Builder ib = INode.newBuilder()
+        .setType(INode.Type.FILE)
+        .setId(id)
+        .setName(ByteString.copyFrom(string2Bytes(s.getPath().getName())))
+        .setFile(b);
+    return ib.build();
+  }
+
+  INode toDirectory(UGIResolver ugi) {
+    final FileStatus s = getFileStatus();
+    ugi.addUser(s.getOwner());
+    ugi.addGroup(s.getGroup());
+    INodeDirectory.Builder b = INodeDirectory.newBuilder()
+        .setModificationTime(s.getModificationTime())
+        .setNsQuota(DEFAULT_NAMESPACE_QUOTA)
+        .setDsQuota(DEFAULT_STORAGE_SPACE_QUOTA)
+        .setPermission(ugi.resolve(s));
+    INode.Builder ib = INode.newBuilder()
+        .setType(INode.Type.DIRECTORY)
+        .setId(id)
+        .setName(ByteString.copyFrom(string2Bytes(s.getPath().getName())))
+        .setDirectory(b);
+    return ib.build();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{ stat=\"").append(getFileStatus()).append("\"");
+    sb.append(", id=").append(getId());
+    sb.append(", parentId=").append(getParentId());
+    sb.append(", iterObjId=").append(System.identityHashCode(i));
+    sb.append(" }");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
new file mode 100644
index 0000000..7fd26f9
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreeWalk.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Iterator;
+
+/**
+ * Traversal yielding a hierarchical sequence of paths.
+ */
+public abstract class TreeWalk implements Iterable<TreePath> {
+
+  /**
+   * @param path path to the node being explored.
+   * @param id the id of the node.
+   * @param iterator the {@link TreeIterator} to use.
+   * @return paths representing the children of the current node.
+   */
+  protected abstract Iterable<TreePath> getChildren(
+      TreePath path, long id, TreeWalk.TreeIterator iterator);
+
+  public abstract TreeIterator iterator();
+
+  /**
+   * Enumerator class for hierarchies. Implementations SHOULD support a fork()
+   * operation yielding a subtree of the current cursor.
+   */
+  public abstract class TreeIterator implements Iterator<TreePath> {
+
+    private final Deque<TreePath> pending;
+
+    TreeIterator() {
+      this(new ArrayDeque<TreePath>());
+    }
+
+    protected TreeIterator(Deque<TreePath> pending) {
+      this.pending = pending;
+    }
+
+    public abstract TreeIterator fork();
+
+    @Override
+    public boolean hasNext() {
+      return !pending.isEmpty();
+    }
+
+    @Override
+    public TreePath next() {
+      return pending.removeFirst();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
+    protected void onAccept(TreePath p, long id) {
+      for (TreePath k : getChildren(p, id, this)) {
+        pending.addFirst(k);
+      }
+    }
+
+    /**
+     * @return the Deque containing the pending paths.
+     */
+    protected Deque<TreePath> getPendingQueue() {
+      return pending;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("{ Treewalk=\"").append(TreeWalk.this.toString());
+      sb.append(", pending=[");
+      Iterator<TreePath> i = pending.iterator();
+      if (i.hasNext()) {
+        sb.append("\"").append(i.next()).append("\"");
+      }
+      while (i.hasNext()) {
+        sb.append(", \"").append(i.next()).append("\"");
+      }
+      sb.append("]");
+      sb.append(" }");
+      return sb.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
new file mode 100644
index 0000000..2d50668
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/UGIResolver.java
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Pluggable class for mapping ownership and permissions from an external
+ * store to an FSImage.
+ */
+public abstract class UGIResolver {
+
+  static final int USER_STRID_OFFSET = 40;
+  static final int GROUP_STRID_OFFSET = 16;
+  static final long USER_GROUP_STRID_MASK = (1 << 24) - 1;
+
+  /**
+   * Permission is serialized as a 64-bit long. [0:24):[25:48):[48:64) (in Big
+   * Endian).
+   * The first and the second parts are the string ids of the user and
+   * group name, and the last 16 bits are the permission bits.
+   * @param owner name of owner
+   * @param group name of group
+   * @param permission Permission octects
+   * @return FSImage encoding of permissions
+   */
+  protected final long buildPermissionStatus(
+      String owner, String group, short permission) {
+
+    long userId = users.get(owner);
+    if (0L != ((~USER_GROUP_STRID_MASK) & userId)) {
+      throw new IllegalArgumentException("UID must fit in 24 bits");
+    }
+
+    long groupId = groups.get(group);
+    if (0L != ((~USER_GROUP_STRID_MASK) & groupId)) {
+      throw new IllegalArgumentException("GID must fit in 24 bits");
+    }
+    return ((userId & USER_GROUP_STRID_MASK) << USER_STRID_OFFSET)
+        | ((groupId & USER_GROUP_STRID_MASK) << GROUP_STRID_OFFSET)
+        | permission;
+  }
+
+  private final Map<String, Integer> users;
+  private final Map<String, Integer> groups;
+
+  public UGIResolver() {
+    this(new HashMap<String, Integer>(), new HashMap<String, Integer>());
+  }
+
+  UGIResolver(Map<String, Integer> users, Map<String, Integer> groups) {
+    this.users = users;
+    this.groups = groups;
+  }
+
+  public Map<Integer, String> ugiMap() {
+    Map<Integer, String> ret = new HashMap<>();
+    for (Map<String, Integer> m : Arrays.asList(users, groups)) {
+      for (Map.Entry<String, Integer> e : m.entrySet()) {
+        String s = ret.put(e.getValue(), e.getKey());
+        if (s != null) {
+          throw new IllegalStateException("Duplicate mapping: " +
+              e.getValue() + " " + s + " " + e.getKey());
+        }
+      }
+    }
+    return ret;
+  }
+
+  public abstract void addUser(String name);
+
+  protected void addUser(String name, int id) {
+    Integer uid = users.put(name, id);
+    if (uid != null) {
+      throw new IllegalArgumentException("Duplicate mapping: " + name +
+          " " + uid + " " + id);
+    }
+  }
+
+  public abstract void addGroup(String name);
+
+  protected void addGroup(String name, int id) {
+    Integer gid = groups.put(name, id);
+    if (gid != null) {
+      throw new IllegalArgumentException("Duplicate mapping: " + name +
+          " " + gid + " " + id);
+    }
+  }
+
+  protected void resetUGInfo() {
+    users.clear();
+    groups.clear();
+  }
+
+  public long resolve(FileStatus s) {
+    return buildPermissionStatus(user(s), group(s), permission(s).toShort());
+  }
+
+  public String user(FileStatus s) {
+    return s.getOwner();
+  }
+
+  public String group(FileStatus s) {
+    return s.getGroup();
+  }
+
+  public FsPermission permission(FileStatus s) {
+    return s.getPermission();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/package-info.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/package-info.java
new file mode 100644
index 0000000..956292e
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
new file mode 100644
index 0000000..c82c489
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Random, repeatable hierarchy generator.
+ */
+public class RandomTreeWalk extends TreeWalk {
+
+  private final Path root;
+  private final long seed;
+  private final float depth;
+  private final int children;
+  private final Map<Long, Long> mSeed;
+  //private final AtomicLong blockIds = new AtomicLong(1L << 30);
+
+  RandomTreeWalk(long seed) {
+    this(seed, 10);
+  }
+
+  RandomTreeWalk(long seed, int children) {
+    this(seed, children, 0.15f);
+  }
+
+  RandomTreeWalk(long seed, int children, float depth) {
+    this(randomRoot(seed), seed, children, 0.15f);
+  }
+
+  RandomTreeWalk(Path root, long seed, int children, float depth) {
+    this.seed = seed;
+    this.depth = depth;
+    this.children = children;
+    mSeed = Collections.synchronizedMap(new HashMap<Long, Long>());
+    mSeed.put(-1L, seed);
+    this.root = root;
+  }
+
+  static Path randomRoot(long seed) {
+    Random r = new Random(seed);
+    String scheme;
+    do {
+      scheme = genName(r, 3, 5).toLowerCase();
+    } while (Character.isDigit(scheme.charAt(0)));
+    String authority = genName(r, 3, 15).toLowerCase();
+    int port = r.nextInt(1 << 13) + 1000;
+    return new Path(scheme, authority + ":" + port, "/");
+  }
+
+  @Override
+  public TreeIterator iterator() {
+    return new RandomTreeIterator(seed);
+  }
+
+  @Override
+  protected Iterable<TreePath> getChildren(TreePath p, long id,
+      TreeIterator walk) {
+    final FileStatus pFs = p.getFileStatus();
+    if (pFs.isFile()) {
+      return Collections.emptyList();
+    }
+    // seed is f(parent seed, attrib)
+    long cseed = mSeed.get(p.getParentId()) * p.getFileStatus().hashCode();
+    mSeed.put(p.getId(), cseed);
+    Random r = new Random(cseed);
+
+    int nChildren = r.nextInt(children);
+    ArrayList<TreePath> ret = new ArrayList<TreePath>();
+    for (int i = 0; i < nChildren; ++i) {
+      ret.add(new TreePath(genFileStatus(p, r), p.getId(), walk));
+    }
+    return ret;
+  }
+
+  FileStatus genFileStatus(TreePath parent, Random r) {
+    final int blocksize = 128 * (1 << 20);
+    final Path name;
+    final boolean isDir;
+    if (null == parent) {
+      name = root;
+      isDir = true;
+    } else {
+      Path p = parent.getFileStatus().getPath();
+      name = new Path(p, genName(r, 3, 10));
+      isDir = r.nextFloat() < depth;
+    }
+    final long len = isDir ? 0 : r.nextInt(Integer.MAX_VALUE);
+    final int nblocks = 0 == len ? 0 : (((int)((len - 1) / blocksize)) + 1);
+    BlockLocation[] blocks = genBlocks(r, nblocks, blocksize, len);
+    try {
+      return new LocatedFileStatus(new FileStatus(
+          len,              /* long length,             */
+          isDir,            /* boolean isdir,           */
+          1,                /* int block_replication,   */
+          blocksize,        /* long blocksize,          */
+          0L,               /* long modification_time,  */
+          0L,               /* long access_time,        */
+          null,             /* FsPermission permission, */
+          "hadoop",         /* String owner,            */
+          "hadoop",         /* String group,            */
+          name),            /* Path path                */
+          blocks);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  BlockLocation[] genBlocks(Random r, int nblocks, int blocksize, long len) {
+    BlockLocation[] blocks = new BlockLocation[nblocks];
+    if (0 == nblocks) {
+      return blocks;
+    }
+    for (int i = 0; i < nblocks - 1; ++i) {
+      blocks[i] = new BlockLocation(null, null, i * blocksize, blocksize);
+    }
+    blocks[nblocks - 1] = new BlockLocation(null, null,
+        (nblocks - 1) * blocksize,
+        0 == (len % blocksize) ? blocksize : len % blocksize);
+    return blocks;
+  }
+
+  static String genName(Random r, int min, int max) {
+    int len = r.nextInt(max - min + 1) + min;
+    char[] ret = new char[len];
+    while (len > 0) {
+      int c = r.nextInt() & 0x7F; // restrict to ASCII
+      if (Character.isLetterOrDigit(c)) {
+        ret[--len] = (char) c;
+      }
+    }
+    return new String(ret);
+  }
+
+  class RandomTreeIterator extends TreeIterator {
+
+    RandomTreeIterator() {
+    }
+
+    RandomTreeIterator(long seed) {
+      Random r = new Random(seed);
+      FileStatus iroot = genFileStatus(null, r);
+      getPendingQueue().addFirst(new TreePath(iroot, -1, this));
+    }
+
+    RandomTreeIterator(TreePath p) {
+      getPendingQueue().addFirst(
+          new TreePath(p.getFileStatus(), p.getParentId(), this));
+    }
+
+    @Override
+    public TreeIterator fork() {
+      if (getPendingQueue().isEmpty()) {
+        return new RandomTreeIterator();
+      }
+      return new RandomTreeIterator(getPendingQueue().removeFirst());
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java
new file mode 100644
index 0000000..8b52ffd
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.junit.Assert.*;
+
+/**
+ * Validate fixed-size block partitioning.
+ */
+public class TestFixedBlockResolver {
+
+  @Rule public TestName name = new TestName();
+
+  private final FixedBlockResolver blockId = new FixedBlockResolver();
+
+  @Before
+  public void setup() {
+    Configuration conf = new Configuration(false);
+    conf.setLong(FixedBlockResolver.BLOCKSIZE, 512L * (1L << 20));
+    conf.setLong(FixedBlockResolver.START_BLOCK, 512L * (1L << 20));
+    blockId.setConf(conf);
+    System.out.println(name.getMethodName());
+  }
+
+  @Test
+  public void testExactBlock() throws Exception {
+    FileStatus f = file(512, 256);
+    int nblocks = 0;
+    for (BlockProto b : blockId.resolve(f)) {
+      ++nblocks;
+      assertEquals(512L * (1L << 20), b.getNumBytes());
+    }
+    assertEquals(1, nblocks);
+
+    FileStatus g = file(1024, 256);
+    nblocks = 0;
+    for (BlockProto b : blockId.resolve(g)) {
+      ++nblocks;
+      assertEquals(512L * (1L << 20), b.getNumBytes());
+    }
+    assertEquals(2, nblocks);
+
+    FileStatus h = file(5120, 256);
+    nblocks = 0;
+    for (BlockProto b : blockId.resolve(h)) {
+      ++nblocks;
+      assertEquals(512L * (1L << 20), b.getNumBytes());
+    }
+    assertEquals(10, nblocks);
+  }
+
+  @Test
+  public void testEmpty() throws Exception {
+    FileStatus f = file(0, 100);
+    Iterator<BlockProto> b = blockId.resolve(f).iterator();
+    assertTrue(b.hasNext());
+    assertEquals(0, b.next().getNumBytes());
+    assertFalse(b.hasNext());
+  }
+
+  @Test
+  public void testRandomFile() throws Exception {
+    Random r = new Random();
+    long seed = r.nextLong();
+    System.out.println("seed: " + seed);
+    r.setSeed(seed);
+
+    int len = r.nextInt(4096) + 512;
+    int blk = r.nextInt(len - 128) + 128;
+    FileStatus s = file(len, blk);
+    long nbytes = 0;
+    for (BlockProto b : blockId.resolve(s)) {
+      nbytes += b.getNumBytes();
+      assertTrue(512L * (1L << 20) >= b.getNumBytes());
+    }
+    assertEquals(s.getLen(), nbytes);
+  }
+
+  FileStatus file(long lenMB, long blocksizeMB) {
+    Path p = new Path("foo://bar:4344/baz/dingo");
+    return new FileStatus(
+          lenMB * (1 << 20),       /* long length,             */
+          false,                   /* boolean isdir,           */
+          1,                       /* int block_replication,   */
+          blocksizeMB * (1 << 20), /* long blocksize,          */
+          0L,                      /* long modification_time,  */
+          0L,                      /* long access_time,        */
+          null,                    /* FsPermission permission, */
+          "hadoop",                /* String owner,            */
+          "hadoop",                /* String group,            */
+          p);                      /* Path path                */
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc7f2d37/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java
new file mode 100644
index 0000000..b8e6ac9
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.junit.Assert.*;
+
+/**
+ * Validate randomly generated hierarchies, including fork() support in
+ * base class.
+ */
+public class TestRandomTreeWalk {
+
+  @Rule public TestName name = new TestName();
+
+  private Random r = new Random();
+
+  @Before
+  public void setSeed() {
+    long seed = r.nextLong();
+    r.setSeed(seed);
+    System.out.println(name.getMethodName() + " seed: " + seed);
+  }
+
+  @Test
+  public void testRandomTreeWalkRepeat() throws Exception {
+    Set<TreePath> ns = new HashSet<>();
+    final long seed = r.nextLong();
+    RandomTreeWalk t1 = new RandomTreeWalk(seed, 10, .1f);
+    int i = 0;
+    for (TreePath p : t1) {
+      p.accept(i++);
+      assertTrue(ns.add(p));
+    }
+
+    RandomTreeWalk t2 = new RandomTreeWalk(seed, 10, .1f);
+    int j = 0;
+    for (TreePath p : t2) {
+      p.accept(j++);
+      assertTrue(ns.remove(p));
+    }
+    assertTrue(ns.isEmpty());
+  }
+
+  @Test
+  public void testRandomTreeWalkFork() throws Exception {
+    Set<FileStatus> ns = new HashSet<>();
+
+    final long seed = r.nextLong();
+    RandomTreeWalk t1 = new RandomTreeWalk(seed, 10, .15f);
+    int i = 0;
+    for (TreePath p : t1) {
+      p.accept(i++);
+      assertTrue(ns.add(p.getFileStatus()));
+    }
+
+    RandomTreeWalk t2 = new RandomTreeWalk(seed, 10, .15f);
+    int j = 0;
+    ArrayList<TreeWalk.TreeIterator> iters = new ArrayList<>();
+    iters.add(t2.iterator());
+    while (!iters.isEmpty()) {
+      for (TreeWalk.TreeIterator sub = iters.remove(iters.size() - 1);
+           sub.hasNext();) {
+        TreePath p = sub.next();
+        if (0 == (r.nextInt() % 4)) {
+          iters.add(sub.fork());
+          Collections.shuffle(iters, r);
+        }
+        p.accept(j++);
+        assertTrue(ns.remove(p.getFileStatus()));
+      }
+    }
+    assertTrue(ns.isEmpty());
+  }
+
+  @Test
+  public void testRandomRootWalk() throws Exception {
+    Set<FileStatus> ns = new HashSet<>();
+    final long seed = r.nextLong();
+    Path root = new Path("foo://bar:4344/dingos");
+    String sroot = root.toString();
+    int nroot = sroot.length();
+    RandomTreeWalk t1 = new RandomTreeWalk(root, seed, 10, .1f);
+    int i = 0;
+    for (TreePath p : t1) {
+      p.accept(i++);
+      FileStatus stat = p.getFileStatus();
+      assertTrue(ns.add(stat));
+      assertEquals(sroot, stat.getPath().toString().substring(0, nroot));
+    }
+
+    RandomTreeWalk t2 = new RandomTreeWalk(root, seed, 10, .1f);
+    int j = 0;
+    for (TreePath p : t2) {
+      p.accept(j++);
+      FileStatus stat = p.getFileStatus();
+      assertTrue(ns.remove(stat));
+      assertEquals(sroot, stat.getPath().toString().substring(0, nroot));
+    }
+    assertTrue(ns.isEmpty());
+  }
+
+}


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: be27bcb9088288be512efe728700176fa80e91ca
Parents: a8b1c7d
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Dec 15 10:15:15 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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


[03/50] [abbrv] hadoop git commit: HDFS-12895. RBF: Add ACL support for mount table. Contributed by Yiqun Lin.

Posted by vi...@apache.org.
HDFS-12895. RBF: Add ACL support for mount table. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-9806
Commit: ee028bfdf1c88a27cd925bed93ebb599a164dd2e
Parents: 89b6c48
Author: Yiqun Lin <yq...@apache.org>
Authored: Fri Dec 15 14:09:24 2017 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Fri Dec 15 14:09:24 2017 +0800

----------------------------------------------------------------------
 .../federation/router/RouterAdminServer.java    |  71 ++++++++
 .../router/RouterPermissionChecker.java         |  82 +++++++++
 .../store/impl/MountTableStoreImpl.java         |  52 +++++-
 .../federation/store/records/MountTable.java    |  68 +++++++
 .../store/records/impl/pb/MountTablePBImpl.java |  61 +++++++
 .../server/namenode/FSPermissionChecker.java    |   4 +-
 .../hdfs/tools/federation/RouterAdmin.java      |  88 ++++++++-
 .../src/main/proto/FederationProtocol.proto     |   4 +
 .../main/webapps/router/federationhealth.html   |   6 +
 .../src/site/markdown/HDFSCommands.md           |   2 +-
 .../src/site/markdown/HDFSRouterFederation.md   |   8 +
 .../metrics/TestFederationMetrics.java          |   3 +
 .../federation/router/TestRouterAdminCLI.java   | 182 +++++++++++++++++++
 13 files changed, 615 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
index 7687216..5fad0c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -35,9 +38,12 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableE
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,6 +71,14 @@ public class RouterAdminServer extends AbstractService
   private final Server adminServer;
   private final InetSocketAddress adminAddress;
 
+  /**
+   * Permission related info used for constructing new router permission
+   * checker instance.
+   */
+  private static String routerOwner;
+  private static String superGroup;
+  private static boolean isPermissionEnabled;
+
   public RouterAdminServer(Configuration conf, Router router)
       throws IOException {
     super(RouterAdminServer.class.getName());
@@ -96,6 +110,7 @@ public class RouterAdminServer extends AbstractService
     LOG.info("Admin server binding to {}:{}",
         bindHost, confRpcAddress.getPort());
 
+    initializePermissionSettings(this.conf);
     this.adminServer = new RPC.Builder(this.conf)
         .setProtocol(RouterAdminProtocolPB.class)
         .setInstance(clientNNPbService)
@@ -112,6 +127,22 @@ public class RouterAdminServer extends AbstractService
     router.setAdminServerAddress(this.adminAddress);
   }
 
+  /**
+   * Initialize permission related settings.
+   *
+   * @param routerConf
+   * @throws IOException
+   */
+  private static void initializePermissionSettings(Configuration routerConf)
+      throws IOException {
+    routerOwner = UserGroupInformation.getCurrentUser().getShortUserName();
+    superGroup = routerConf.get(
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
+    isPermissionEnabled = routerConf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
+        DFS_PERMISSIONS_ENABLED_DEFAULT);
+  }
+
   /** Allow access to the client RPC server for testing. */
   @VisibleForTesting
   Server getAdminServer() {
@@ -180,4 +211,44 @@ public class RouterAdminServer extends AbstractService
       GetMountTableEntriesRequest request) throws IOException {
     return getMountTableStore().getMountTableEntries(request);
   }
+
+  /**
+   * Get a new permission checker used for making mount table access
+   * control. This method will be invoked during each RPC call in router
+   * admin server.
+   *
+   * @return Router permission checker
+   * @throws AccessControlException
+   */
+  public static RouterPermissionChecker getPermissionChecker()
+      throws AccessControlException {
+    if (!isPermissionEnabled) {
+      return null;
+    }
+
+    try {
+      return new RouterPermissionChecker(routerOwner, superGroup,
+          NameNode.getRemoteUser());
+    } catch (IOException e) {
+      throw new AccessControlException(e);
+    }
+  }
+
+  /**
+   * Get super user name.
+   *
+   * @return String super user name.
+   */
+  public static String getSuperUser() {
+    return routerOwner;
+  }
+
+  /**
+   * Get super group name.
+   *
+   * @return String super group name.
+   */
+  public static String getSuperGroup(){
+    return superGroup;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterPermissionChecker.java
new file mode 100644
index 0000000..9d81dce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterPermissionChecker.java
@@ -0,0 +1,82 @@
+/**
+ * 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.federation.router;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Class that helps in checking permissions in Router-based federation.
+ */
+public class RouterPermissionChecker extends FSPermissionChecker {
+  static final Log LOG = LogFactory.getLog(RouterPermissionChecker.class);
+
+  /** Mount table default permission. */
+  public static final short MOUNT_TABLE_PERMISSION_DEFAULT = 00755;
+
+  public RouterPermissionChecker(String routerOwner, String supergroup,
+      UserGroupInformation callerUgi) {
+    super(routerOwner, supergroup, callerUgi, null);
+  }
+
+  /**
+   * Whether a mount table entry can be accessed by the current context.
+   *
+   * @param mountTable
+   *          MountTable being accessed
+   * @param access
+   *          type of action being performed on the cache pool
+   * @throws AccessControlException
+   *           if mount table cannot be accessed
+   */
+  public void checkPermission(MountTable mountTable, FsAction access)
+      throws AccessControlException {
+    if (isSuperUser()) {
+      return;
+    }
+
+    FsPermission mode = mountTable.getMode();
+    if (getUser().equals(mountTable.getOwnerName())
+        && mode.getUserAction().implies(access)) {
+      return;
+    }
+
+    if (isMemberOfGroup(mountTable.getGroupName())
+        && mode.getGroupAction().implies(access)) {
+      return;
+    }
+
+    if (!getUser().equals(mountTable.getOwnerName())
+        && !isMemberOfGroup(mountTable.getGroupName())
+        && mode.getOtherAction().implies(access)) {
+      return;
+    }
+
+    throw new AccessControlException(
+        "Permission denied while accessing mount table "
+            + mountTable.getSourcePath()
+            + ": user " + getUser() + " does not have " + access.toString()
+            + " permissions.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java
index e6affb2..eb117d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java
@@ -24,6 +24,9 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
+import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
@@ -36,6 +39,7 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableE
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Time;
 
 /**
@@ -52,7 +56,15 @@ public class MountTableStoreImpl extends MountTableStore {
   @Override
   public AddMountTableEntryResponse addMountTableEntry(
       AddMountTableEntryRequest request) throws IOException {
-    boolean status = getDriver().put(request.getEntry(), false, true);
+    MountTable mountTable = request.getEntry();
+    if (mountTable != null) {
+      RouterPermissionChecker pc = RouterAdminServer.getPermissionChecker();
+      if (pc != null) {
+        pc.checkPermission(mountTable, FsAction.WRITE);
+      }
+    }
+
+    boolean status = getDriver().put(mountTable, false, true);
     AddMountTableEntryResponse response =
         AddMountTableEntryResponse.newInstance();
     response.setStatus(status);
@@ -62,8 +74,15 @@ public class MountTableStoreImpl extends MountTableStore {
   @Override
   public UpdateMountTableEntryResponse updateMountTableEntry(
       UpdateMountTableEntryRequest request) throws IOException {
-    MountTable entry = request.getEntry();
-    boolean status = getDriver().put(entry, true, true);
+    MountTable mountTable = request.getEntry();
+    if (mountTable != null) {
+      RouterPermissionChecker pc = RouterAdminServer.getPermissionChecker();
+      if (pc != null) {
+        pc.checkPermission(mountTable, FsAction.WRITE);
+      }
+    }
+
+    boolean status = getDriver().put(mountTable, true, true);
     UpdateMountTableEntryResponse response =
         UpdateMountTableEntryResponse.newInstance();
     response.setStatus(status);
@@ -77,8 +96,17 @@ public class MountTableStoreImpl extends MountTableStore {
     final MountTable partial = MountTable.newInstance();
     partial.setSourcePath(srcPath);
     final Query<MountTable> query = new Query<>(partial);
-    int removedRecords = getDriver().remove(getRecordClass(), query);
-    boolean status = (removedRecords == 1);
+    final MountTable deleteEntry = getDriver().get(getRecordClass(), query);
+
+    boolean status = false;
+    if (deleteEntry != null) {
+      RouterPermissionChecker pc = RouterAdminServer.getPermissionChecker();
+      if (pc != null) {
+        pc.checkPermission(deleteEntry, FsAction.WRITE);
+      }
+      status = getDriver().remove(deleteEntry);
+    }
+
     RemoveMountTableEntryResponse response =
         RemoveMountTableEntryResponse.newInstance();
     response.setStatus(status);
@@ -88,12 +116,13 @@ public class MountTableStoreImpl extends MountTableStore {
   @Override
   public GetMountTableEntriesResponse getMountTableEntries(
       GetMountTableEntriesRequest request) throws IOException {
-
+    RouterPermissionChecker pc =
+        RouterAdminServer.getPermissionChecker();
     // Get all values from the cache
     List<MountTable> records = getCachedRecords();
 
     // Sort and filter
-    Collections.sort(records);
+    Collections.sort(records, MountTable.SOURCE_COMPARATOR);
     String reqSrcPath = request.getSrcPath();
     if (reqSrcPath != null && !reqSrcPath.isEmpty()) {
       // Return only entries beneath this path
@@ -103,6 +132,15 @@ public class MountTableStoreImpl extends MountTableStore {
         String srcPath = record.getSourcePath();
         if (!srcPath.startsWith(reqSrcPath)) {
           it.remove();
+        } else if (pc != null) {
+          // do the READ permission check
+          try {
+            pc.checkPermission(record, FsAction.READ);
+          } catch (AccessControlException ignored) {
+            // Remove this mount table entry if it cannot
+            // be accessed by current user.
+            it.remove();
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
index 0a3f19d..1b5d2d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
@@ -28,9 +28,13 @@ import java.util.TreeMap;
 
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -127,6 +131,15 @@ public abstract class MountTable extends BaseRecord {
     // Set the serialized dest string
     record.setDestinations(locations);
 
+    // Set permission fields
+    UserGroupInformation ugi = NameNode.getRemoteUser();
+    record.setOwnerName(ugi.getShortUserName());
+    String group = ugi.getGroups().isEmpty() ? ugi.getShortUserName()
+        : ugi.getPrimaryGroupName();
+    record.setGroupName(group);
+    record.setMode(new FsPermission(
+        RouterPermissionChecker.MOUNT_TABLE_PERMISSION_DEFAULT));
+
     // Validate
     record.validate();
     return record;
@@ -194,6 +207,48 @@ public abstract class MountTable extends BaseRecord {
   public abstract void setDestOrder(DestinationOrder order);
 
   /**
+   * Get owner name of this mount table entry.
+   *
+   * @return Owner name
+   */
+  public abstract String getOwnerName();
+
+  /**
+   * Set owner name of this mount table entry.
+   *
+   * @param owner Owner name for mount table entry
+   */
+  public abstract void setOwnerName(String owner);
+
+  /**
+   * Get group name of this mount table entry.
+   *
+   * @return Group name
+   */
+  public abstract String getGroupName();
+
+  /**
+   * Set group name of this mount table entry.
+   *
+   * @param group Group name for mount table entry
+   */
+  public abstract void setGroupName(String group);
+
+  /**
+   * Get permission of this mount table entry.
+   *
+   * @return FsPermission permission mode
+   */
+  public abstract FsPermission getMode();
+
+  /**
+   * Set permission for this mount table entry.
+   *
+   * @param mode Permission for mount table entry
+   */
+  public abstract void setMode(FsPermission mode);
+
+  /**
    * Get the default location.
    * @return The default location.
    */
@@ -235,6 +290,19 @@ public abstract class MountTable extends BaseRecord {
     if (this.isReadOnly()) {
       sb.append("[RO]");
     }
+
+    if (this.getOwnerName() != null) {
+      sb.append("[owner:").append(this.getOwnerName()).append("]");
+    }
+
+    if (this.getGroupName() != null) {
+      sb.append("[group:").append(this.getGroupName()).append("]");
+    }
+
+    if (this.getMode() != null) {
+      sb.append("[mode:").append(this.getMode()).append("]");
+    }
+
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
index d2870bd..372f209 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto.Builder;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto.DestOrder;
@@ -28,6 +29,8 @@ import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProt
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoteLocationProto;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
+import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 
@@ -189,6 +192,64 @@ public class MountTablePBImpl extends MountTable implements PBRecord {
     }
   }
 
+  @Override
+  public String getOwnerName() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasOwnerName()) {
+      return RouterAdminServer.getSuperUser();
+    }
+    return proto.getOwnerName();
+  }
+
+  @Override
+  public void setOwnerName(String owner) {
+    Builder builder = this.translator.getBuilder();
+    if (owner == null) {
+      builder.clearOwnerName();
+    } else {
+      builder.setOwnerName(owner);
+    }
+  }
+
+  @Override
+  public String getGroupName() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasGroupName()) {
+      return RouterAdminServer.getSuperGroup();
+    }
+    return proto.getGroupName();
+  }
+
+  @Override
+  public void setGroupName(String group) {
+    Builder builder = this.translator.getBuilder();
+    if (group == null) {
+      builder.clearGroupName();
+    } else {
+      builder.setGroupName(group);
+    }
+  }
+
+  @Override
+  public FsPermission getMode() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    short mode = RouterPermissionChecker.MOUNT_TABLE_PERMISSION_DEFAULT;
+    if (proto.hasMode()) {
+      mode = (short) proto.getMode();
+    }
+    return new FsPermission(mode);
+  }
+
+  @Override
+  public void setMode(FsPermission mode) {
+    Builder builder = this.translator.getBuilder();
+    if (mode == null) {
+      builder.clearMode();
+    } else {
+      builder.setMode(mode.toShort());
+    }
+  }
+
   private DestinationOrder convert(DestOrder order) {
     switch (order) {
     case LOCAL:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index c854b49..45876a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -46,7 +46,7 @@ import org.apache.hadoop.security.UserGroupInformation;
  * 
  * Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
  */
-class FSPermissionChecker implements AccessControlEnforcer {
+public class FSPermissionChecker implements AccessControlEnforcer {
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
 
   private static String getPath(byte[][] components, int start, int end) {
@@ -86,7 +86,7 @@ class FSPermissionChecker implements AccessControlEnforcer {
   private final INodeAttributeProvider attributeProvider;
 
 
-  FSPermissionChecker(String fsOwner, String supergroup,
+  protected FSPermissionChecker(String fsOwner, String supergroup,
       UserGroupInformation callerUgi,
       INodeAttributeProvider attributeProvider) {
     this.fsOwner = fsOwner;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
index 897432f..a91a602 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
@@ -77,7 +78,7 @@ public class RouterAdmin extends Configured implements Tool {
   public void printUsage() {
     String usage = "Federation Admin Tools:\n"
         + "\t[-add <source> <nameservice> <destination> "
-        + "[-readonly]\n"
+        + "[-readonly] -owner <owner> -group <group> -mode <mode>]\n"
         + "\t[-rm <source>]\n"
         + "\t[-ls <path>]\n";
     System.out.println(usage);
@@ -193,6 +194,9 @@ public class RouterAdmin extends Configured implements Tool {
 
     // Optional parameters
     boolean readOnly = false;
+    String owner = null;
+    String group = null;
+    FsPermission mode = null;
     DestinationOrder order = DestinationOrder.HASH;
     while (i < parameters.length) {
       if (parameters[i].equals("-readonly")) {
@@ -204,11 +208,23 @@ public class RouterAdmin extends Configured implements Tool {
         } catch(Exception e) {
           System.err.println("Cannot parse order: " + parameters[i]);
         }
+      } else if (parameters[i].equals("-owner")) {
+        i++;
+        owner = parameters[i];
+      } else if (parameters[i].equals("-group")) {
+        i++;
+        group = parameters[i];
+      } else if (parameters[i].equals("-mode")) {
+        i++;
+        short modeValue = Short.parseShort(parameters[i], 8);
+        mode = new FsPermission(modeValue);
       }
+
       i++;
     }
 
-    return addMount(mount, nss, dest, readOnly, order);
+    return addMount(mount, nss, dest, readOnly, order,
+        new ACLEntity(owner, group, mode));
   }
 
   /**
@@ -219,11 +235,13 @@ public class RouterAdmin extends Configured implements Tool {
    * @param dest Destination path.
    * @param readonly If the mount point is read only.
    * @param order Order of the destination locations.
+   * @param aclInfo the ACL info for mount point.
    * @return If the mount point was added.
    * @throws IOException Error adding the mount point.
    */
   public boolean addMount(String mount, String[] nss, String dest,
-      boolean readonly, DestinationOrder order) throws IOException {
+      boolean readonly, DestinationOrder order, ACLEntity aclInfo)
+      throws IOException {
     // Get the existing entry
     MountTableManager mountTable = client.getMountTableManager();
     GetMountTableEntriesRequest getRequest =
@@ -251,6 +269,20 @@ public class RouterAdmin extends Configured implements Tool {
       if (order != null) {
         newEntry.setDestOrder(order);
       }
+
+      // Set ACL info for mount table entry
+      if (aclInfo.getOwner() != null) {
+        newEntry.setOwnerName(aclInfo.getOwner());
+      }
+
+      if (aclInfo.getGroup() != null) {
+        newEntry.setGroupName(aclInfo.getGroup());
+      }
+
+      if (aclInfo.getMode() != null) {
+        newEntry.setMode(aclInfo.getMode());
+      }
+
       AddMountTableEntryRequest request =
           AddMountTableEntryRequest.newInstance(newEntry);
       AddMountTableEntryResponse addResponse =
@@ -273,6 +305,20 @@ public class RouterAdmin extends Configured implements Tool {
       if (order != null) {
         existingEntry.setDestOrder(order);
       }
+
+      // Update ACL info of mount table entry
+      if (aclInfo.getOwner() != null) {
+        existingEntry.setOwnerName(aclInfo.getOwner());
+      }
+
+      if (aclInfo.getGroup() != null) {
+        existingEntry.setGroupName(aclInfo.getGroup());
+      }
+
+      if (aclInfo.getMode() != null) {
+        existingEntry.setMode(aclInfo.getMode());
+      }
+
       UpdateMountTableEntryRequest updateRequest =
           UpdateMountTableEntryRequest.newInstance(existingEntry);
       UpdateMountTableEntryResponse updateResponse =
@@ -323,8 +369,8 @@ public class RouterAdmin extends Configured implements Tool {
   private static void printMounts(List<MountTable> entries) {
     System.out.println("Mount Table Entries:");
     System.out.println(String.format(
-        "%-25s %-25s",
-        "Source", "Destinations"));
+        "%-25s %-25s %-25s %-25s %-25s",
+        "Source", "Destinations", "Owner", "Group", "Mode"));
     for (MountTable entry : entries) {
       StringBuilder destBuilder = new StringBuilder();
       for (RemoteLocation location : entry.getDestinations()) {
@@ -334,8 +380,38 @@ public class RouterAdmin extends Configured implements Tool {
         destBuilder.append(String.format("%s->%s", location.getNameserviceId(),
             location.getDest()));
       }
-      System.out.println(String.format("%-25s %-25s", entry.getSourcePath(),
+      System.out.print(String.format("%-25s %-25s", entry.getSourcePath(),
           destBuilder.toString()));
+
+      System.out.println(String.format(" %-25s %-25s %-25s",
+          entry.getOwnerName(), entry.getGroupName(), entry.getMode()));
+    }
+  }
+
+  /**
+   * Inner class that stores ACL info of mount table.
+   */
+  static class ACLEntity {
+    private final String owner;
+    private final String group;
+    private final FsPermission mode;
+
+    ACLEntity(String owner, String group, FsPermission mode) {
+      this.owner = owner;
+      this.group = group;
+      this.mode = mode;
+    }
+
+    public String getOwner() {
+      return owner;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+
+    public FsPermission getMode() {
+      return mode;
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
index 32a6250..88acd08 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
@@ -129,6 +129,10 @@ message MountTableRecordProto {
     RANDOM = 2;
   }
   optional DestOrder destOrder = 6 [default = HASH];
+
+  optional string ownerName = 10;
+  optional string groupName = 11;
+  optional int32 mode = 12;
 }
 
 message AddMountTableEntryRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html
index 3da5283..9c28975 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html
@@ -335,6 +335,9 @@
       <th>Target path</th>
       <th>Order</th>
       <th>Read only</th>
+      <th>Owner</th>
+      <th>Group</th>
+      <th>Permission</th>
       <th>Date Modified</th>
       <th>Date Created</th>
     </tr>
@@ -347,6 +350,9 @@
       <td>{path}</td>
       <td>{order}</td>
       <td class="dfshealth-node-icon dfshealth-mount-read-only-{readonly}"/>
+      <td>{ownerName}</td>
+      <td>{groupName}</td>
+      <td>{mode}</td>
       <td>{dateModified}</td>
       <td>{dateCreated}</td>
     </tr>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 16a47fc..316b955 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -425,7 +425,7 @@ Runs the DFS router. See [Router](./HDFSRouterFederation.html#Router) for more i
 Usage:
 
       hdfs dfsrouteradmin
-          [-add <source> <nameservice> <destination> [-readonly]]
+          [-add <source> <nameservice> <destination> [-readonly] -owner <owner> -group <group> -mode <mode>]
           [-rm <source>]
           [-ls <path>]
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
index 5075a22..cd3f437 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
@@ -190,6 +190,14 @@ It also supports mount points that disallow writes:
 
 If a mount point is not set, the Router will map it to the default namespace `dfs.federation.router.default.nameserviceId`.
 
+Mount table have UNIX-like *permissions*, which restrict which users and groups have access to the mount point. Write permissions allow users to add
+, update or remove mount point. Read permissions allow users to list mount point. Execute permissions are unused.
+
+Mount table permission can be set by following command:
+
+    [hdfs]$ $HADOOP_HOME/bin/hdfs dfsrouteradmin -add /tmp ns1 /tmp -owner root -group supergroup -mode 0755
+
+The option mode is UNIX-style permissions for the mount table. Permissions are specified in octal, e.g. 0755. By default, this is set to 0755.
 
 Client configuration
 --------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
index d6a194f..61fda0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
@@ -84,6 +84,9 @@ public class TestFederationMetrics extends TestMetricsBase {
               json.getString("nameserviceId"));
           assertEquals(entry.getDefaultLocation().getDest(),
               json.getString("path"));
+          assertEquals(entry.getOwnerName(), json.getString("ownerName"));
+          assertEquals(entry.getGroupName(), json.getString("groupName"));
+          assertEquals(entry.getMode().toString(), json.getString("mode"));
           assertNotNullAndNotEmpty(json.getString("dateCreated"));
           assertNotNullAndNotEmpty(json.getString("dateModified"));
           match++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee028bfd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
new file mode 100644
index 0000000..3882b8b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
@@ -0,0 +1,182 @@
+/**
+ * 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.federation.router;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.impl.MountTableStoreImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.tools.federation.RouterAdmin;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+/**
+ * Tests Router admin commands.
+ */
+public class TestRouterAdminCLI {
+  private static StateStoreDFSCluster cluster;
+  private static RouterContext routerContext;
+  private static StateStoreService stateStore;
+
+  private static RouterAdmin admin;
+  private static RouterClient client;
+
+  private static final String TEST_USER = "test-user";
+
+  private final ByteArrayOutputStream out = new ByteArrayOutputStream();
+  private static final PrintStream OLD_OUT = System.out;
+
+  @BeforeClass
+  public static void globalSetUp() throws Exception {
+    cluster = new StateStoreDFSCluster(false, 1);
+    // Build and start a router with State Store + admin + RPC
+    Configuration conf = new RouterConfigBuilder()
+        .stateStore()
+        .admin()
+        .rpc()
+        .build();
+    cluster.addRouterOverrides(conf);
+
+    // Start routers
+    cluster.startRouters();
+
+    routerContext = cluster.getRandomRouter();
+    Router router = routerContext.getRouter();
+    stateStore = router.getStateStore();
+
+    Configuration routerConf = new Configuration();
+    InetSocketAddress routerSocket = router.getAdminServerAddress();
+    routerConf.setSocketAddr(DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY,
+        routerSocket);
+    admin = new RouterAdmin(routerConf);
+    client = routerContext.getAdminClient();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    cluster.stopRouter(routerContext);
+    cluster.shutdown();
+    cluster = null;
+  }
+
+  @Test
+  public void testMountTableDefaultACL() throws Exception {
+    String[] argv = new String[] {"-add", "/testpath0", "ns0", "/testdir0"};
+    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance("/testpath0");
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    MountTable mountTable = getResponse.getEntries().get(0);
+
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    String group = ugi.getGroups().isEmpty() ? ugi.getShortUserName()
+        : ugi.getPrimaryGroupName();
+    assertEquals(ugi.getShortUserName(), mountTable.getOwnerName());
+    assertEquals(group, mountTable.getGroupName());
+    assertEquals((short) 0755, mountTable.getMode().toShort());
+  }
+
+  @Test
+  public void testMountTablePermissions() throws Exception {
+    // re-set system out for testing
+    System.setOut(new PrintStream(out));
+    // use superuser to add new mount table with only read permission
+    String[] argv = new String[] {"-add", "/testpath2-1", "ns0", "/testdir2-1",
+        "-owner", TEST_USER, "-group", TEST_USER, "-mode", "0455"};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    String superUser = UserGroupInformation.
+        getCurrentUser().getShortUserName();
+    // use normal user as current user to test
+    UserGroupInformation remoteUser = UserGroupInformation
+        .createRemoteUser(TEST_USER);
+    UserGroupInformation.setLoginUser(remoteUser);
+
+    // verify read permission by executing other commands
+    verifyExecutionResult("/testpath2-1", true, -1, -1);
+
+    // add new mount table with only write permission
+    argv = new String[] {"-add", "/testpath2-2", "ns0", "/testdir2-2",
+        "-owner", TEST_USER, "-group", TEST_USER, "-mode", "0255"};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    verifyExecutionResult("/testpath2-2", false, 0, 0);
+
+    // set mount table entry with read and write permission
+    argv = new String[] {"-add", "/testpath2-3", "ns0", "/testdir2-3",
+        "-owner", TEST_USER, "-group", TEST_USER, "-mode", "0755"};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    verifyExecutionResult("/testpath2-3", true, 0, 0);
+
+    // set back system out and login user
+    System.setOut(OLD_OUT);
+    remoteUser = UserGroupInformation.createRemoteUser(superUser);
+    UserGroupInformation.setLoginUser(remoteUser);
+  }
+
+  /**
+   * Verify router admin commands execution result.
+   *
+   * @param mount
+   *          target mount table
+   * @param canRead
+   *          whether can list mount tables under specified mount
+   * @param addCommandCode
+   *          expected return code of add command executed for specified mount
+   * @param rmCommandCode
+   *          expected return code of rm command executed for specified mount
+   * @throws Exception
+   */
+  private void verifyExecutionResult(String mount, boolean canRead,
+      int addCommandCode, int rmCommandCode) throws Exception {
+    String[] argv = null;
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+
+    out.reset();
+    // execute ls command
+    argv = new String[] {"-ls", mount};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertEquals(canRead, out.toString().contains(mount));
+
+    // execute add/update command
+    argv = new String[] {"-add", mount, "ns0", mount + "newdir"};
+    assertEquals(addCommandCode, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    // execute remove command
+    argv = new String[] {"-rm", mount};
+    assertEquals(rmCommandCode, ToolRunner.run(admin, argv));
+  }
+}
\ No newline at end of file


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


[11/50] [abbrv] hadoop git commit: HDFS-11190. [READ] Namenode support for data stored in external stores.

Posted by vi...@apache.org.
HDFS-11190. [READ] Namenode support for data stored in external stores.


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

Branch: refs/heads/HDFS-9806
Commit: 3fee8ad04f3077e5271a48a0069f4508e3761c92
Parents: fc7f2d3
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Apr 21 11:12:36 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:26 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/LocatedBlock.java      |  96 ++++-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 .../blockmanagement/BlockFormatProvider.java    |  91 ++++
 .../server/blockmanagement/BlockManager.java    |  95 +++--
 .../server/blockmanagement/BlockProvider.java   |  65 +++
 .../BlockStoragePolicySuite.java                |   6 +
 .../blockmanagement/DatanodeDescriptor.java     |  34 +-
 .../server/blockmanagement/DatanodeManager.java |   2 +
 .../blockmanagement/DatanodeStorageInfo.java    |   4 +
 .../blockmanagement/LocatedBlockBuilder.java    | 109 +++++
 .../blockmanagement/ProvidedStorageMap.java     | 427 +++++++++++++++++++
 .../src/main/resources/hdfs-default.xml         |  30 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   4 +
 .../blockmanagement/TestDatanodeManager.java    |  65 ++-
 .../TestNameNodeProvidedImplementation.java     | 345 +++++++++++++++
 15 files changed, 1292 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index 85bec92..5ad0bca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.protocol;
 
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
@@ -62,40 +63,50 @@ public class LocatedBlock {
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
     // By default, startOffset is unknown(-1) and corrupt is false.
-    this(b, locs, null, null, -1, false, EMPTY_LOCS);
+    this(b, convert(locs, null, null), null, null, -1, false, EMPTY_LOCS);
   }
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
       String[] storageIDs, StorageType[] storageTypes) {
-    this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
+    this(b, convert(locs, storageIDs, storageTypes),
+         storageIDs, storageTypes, -1, false, EMPTY_LOCS);
   }
 
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
-      StorageType[] storageTypes, long startOffset,
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+      String[] storageIDs, StorageType[] storageTypes, long startOffset,
+      boolean corrupt, DatanodeInfo[] cachedLocs) {
+    this(b, convert(locs, storageIDs, storageTypes),
+        storageIDs, storageTypes, startOffset, corrupt,
+        null == cachedLocs || 0 == cachedLocs.length ? EMPTY_LOCS : cachedLocs);
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeInfoWithStorage[] locs,
+      String[] storageIDs, StorageType[] storageTypes, long startOffset,
       boolean corrupt, DatanodeInfo[] cachedLocs) {
     this.b = b;
     this.offset = startOffset;
     this.corrupt = corrupt;
-    if (locs==null) {
-      this.locs = EMPTY_LOCS;
-    } else {
-      this.locs = new DatanodeInfoWithStorage[locs.length];
-      for(int i = 0; i < locs.length; i++) {
-        DatanodeInfo di = locs[i];
-        DatanodeInfoWithStorage storage = new DatanodeInfoWithStorage(di,
-            storageIDs != null ? storageIDs[i] : null,
-            storageTypes != null ? storageTypes[i] : null);
-        this.locs[i] = storage;
-      }
-    }
+    this.locs = null == locs ? EMPTY_LOCS : locs;
     this.storageIDs = storageIDs;
     this.storageTypes = storageTypes;
+    this.cachedLocs = null == cachedLocs || 0 == cachedLocs.length
+      ? EMPTY_LOCS
+      : cachedLocs;
+  }
+
+  private static DatanodeInfoWithStorage[] convert(
+      DatanodeInfo[] infos, String[] storageIDs, StorageType[] storageTypes) {
+    if (null == infos) {
+      return EMPTY_LOCS;
+    }
 
-    if (cachedLocs == null || cachedLocs.length == 0) {
-      this.cachedLocs = EMPTY_LOCS;
-    } else {
-      this.cachedLocs = cachedLocs;
+    DatanodeInfoWithStorage[] ret = new DatanodeInfoWithStorage[infos.length];
+    for(int i = 0; i < infos.length; i++) {
+      ret[i] = new DatanodeInfoWithStorage(infos[i],
+          storageIDs   != null ? storageIDs[i]   : null,
+          storageTypes != null ? storageTypes[i] : null);
     }
+    return ret;
   }
 
   public Token<BlockTokenIdentifier> getBlockToken() {
@@ -145,6 +156,51 @@ public class LocatedBlock {
     }
   }
 
+  /**
+   * Comparator that ensures that a PROVIDED storage type is greater than
+   * any other storage type. Any other storage types are considered equal.
+   */
+  private class ProvidedLastComparator
+    implements Comparator<DatanodeInfoWithStorage> {
+    @Override
+    public int compare(DatanodeInfoWithStorage dns1,
+        DatanodeInfoWithStorage dns2) {
+      if (StorageType.PROVIDED.equals(dns1.getStorageType())
+          && !StorageType.PROVIDED.equals(dns2.getStorageType())) {
+        return 1;
+      }
+      if (!StorageType.PROVIDED.equals(dns1.getStorageType())
+          && StorageType.PROVIDED.equals(dns2.getStorageType())) {
+        return -1;
+      }
+      // Storage types of dns1 and dns2 are now both provided or not provided;
+      // thus, are essentially equal for the purpose of this comparator.
+      return 0;
+    }
+  }
+
+  /**
+   * Moves all locations that have {@link StorageType}
+   * {@code PROVIDED} to the end of the locations array without
+   * changing the relative ordering of the remaining locations
+   * Only the first {@code activeLen} locations are considered.
+   * The caller must immediately invoke {@link
+   * org.apache.hadoop.hdfs.protocol.LocatedBlock#updateCachedStorageInfo}
+   * to update the cached Storage ID/Type arrays.
+   * @param activeLen
+   */
+  public void moveProvidedToEnd(int activeLen) {
+
+    if (activeLen <= 0) {
+      return;
+    }
+    // as this is a stable sort, for elements that are equal,
+    // the current order of the elements is maintained
+    Arrays.sort(locs, 0,
+        (activeLen < locs.length) ? activeLen : locs.length,
+        new ProvidedLastComparator());
+  }
+
   public long getStartOffset() {
     return offset;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index ca753ce..7449987 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -328,6 +328,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.edits.asynclogging";
   public static final boolean DFS_NAMENODE_EDITS_ASYNC_LOGGING_DEFAULT = true;
 
+  public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
+  public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;
+
+  public static final String DFS_NAMENODE_BLOCK_PROVIDER_CLASS = "dfs.namenode.block.provider.class";
+
   public static final String DFS_PROVIDER_CLASS = "dfs.provider.class";
   public static final String DFS_PROVIDER_DF_CLASS = "dfs.provided.df.class";
   public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
new file mode 100644
index 0000000..930263d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.BlockAlias;
+import org.apache.hadoop.hdfs.server.common.BlockFormat;
+import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Loads provided blocks from a {@link BlockFormat}.
+ */
+public class BlockFormatProvider extends BlockProvider
+    implements Configurable {
+
+  private Configuration conf;
+  private BlockFormat<? extends BlockAlias> blockFormat;
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockFormatProvider.class);
+
+  @Override
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public void setConf(Configuration conf) {
+    Class<? extends BlockFormat> c = conf.getClass(
+        DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
+        TextFileRegionFormat.class, BlockFormat.class);
+    blockFormat = ReflectionUtils.newInstance(c, conf);
+    LOG.info("Loaded BlockFormat class : " + c.getClass().getName());
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Iterator<Block> iterator() {
+    try {
+      final BlockFormat.Reader<? extends BlockAlias> reader =
+          blockFormat.getReader(null);
+
+      return new Iterator<Block>() {
+
+        private final Iterator<? extends BlockAlias> inner = reader.iterator();
+
+        @Override
+        public boolean hasNext() {
+          return inner.hasNext();
+        }
+
+        @Override
+        public Block next() {
+          return inner.next().getBlock();
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to read provided blocks", e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index ae04aac..0e3eab3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -435,6 +435,9 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private final short minReplicationToBeInMaintenance;
 
+  /** Storages accessible from multiple DNs. */
+  private final ProvidedStorageMap providedStorageMap;
+
   public BlockManager(final Namesystem namesystem, boolean haEnabled,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -467,6 +470,8 @@ public class BlockManager implements BlockStatsMXBean {
 
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
+    providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
+
     this.maxCorruptFilesReturned = conf.getInt(
       DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
       DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
@@ -1144,7 +1149,7 @@ public class BlockManager implements BlockStatsMXBean {
     final long fileLength = bc.computeContentSummary(
         getStoragePolicySuite()).getLength();
     final long pos = fileLength - lastBlock.getNumBytes();
-    return createLocatedBlock(lastBlock, pos,
+    return createLocatedBlock(null, lastBlock, pos,
         BlockTokenIdentifier.AccessMode.WRITE);
   }
 
@@ -1165,8 +1170,10 @@ public class BlockManager implements BlockStatsMXBean {
     return locations;
   }
 
-  private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
-      final long offset, final long length, final int nrBlocksToReturn,
+  private void createLocatedBlockList(
+      LocatedBlockBuilder locatedBlocks,
+      final BlockInfo[] blocks,
+      final long offset, final long length,
       final AccessMode mode) throws IOException {
     int curBlk;
     long curPos = 0, blkSize = 0;
@@ -1181,21 +1188,22 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return Collections.emptyList();
+      return;
 
     long endOff = offset + length;
-    List<LocatedBlock> results = new ArrayList<>(blocks.length);
     do {
-      results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
+      locatedBlocks.addBlock(
+          createLocatedBlock(locatedBlocks, blocks[curBlk], curPos, mode));
       curPos += blocks[curBlk].getNumBytes();
       curBlk++;
     } while (curPos < endOff 
           && curBlk < blocks.length
-          && results.size() < nrBlocksToReturn);
-    return results;
+          && !locatedBlocks.isBlockMax());
+    return;
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
+  private LocatedBlock createLocatedBlock(LocatedBlockBuilder locatedBlocks,
+      final BlockInfo[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
     int curBlk;
     long curPos = 0;
@@ -1208,12 +1216,13 @@ public class BlockManager implements BlockStatsMXBean {
       curPos += blkSize;
     }
     
-    return createLocatedBlock(blocks[curBlk], curPos, mode);
+    return createLocatedBlock(locatedBlocks, blocks[curBlk], curPos, mode);
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
-    final AccessMode mode) throws IOException {
-    final LocatedBlock lb = createLocatedBlock(blk, pos);
+  private LocatedBlock createLocatedBlock(LocatedBlockBuilder locatedBlocks,
+      final BlockInfo blk, final long pos, final AccessMode mode)
+          throws IOException {
+    final LocatedBlock lb = createLocatedBlock(locatedBlocks, blk, pos);
     if (mode != null) {
       setBlockToken(lb, mode);
     }
@@ -1221,21 +1230,24 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
-      throws IOException {
+  private LocatedBlock createLocatedBlock(LocatedBlockBuilder locatedBlocks,
+      final BlockInfo blk, final long pos) throws IOException {
     if (!blk.isComplete()) {
       final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
             blk);
+        //TODO use locatedBlocks builder??
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
             blk);
-        return newLocatedBlock(eb, storages, pos, false);
+        return null == locatedBlocks
+            ? newLocatedBlock(eb, storages, pos, false)
+                : locatedBlocks.newLocatedBlock(eb, storages, pos, false);
       }
     }
 
@@ -1299,9 +1311,10 @@ public class BlockManager implements BlockStatsMXBean {
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
     final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(), blk);
-    return blockIndices == null ?
-        newLocatedBlock(eb, machines, pos, isCorrupt) :
-        newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
+    return blockIndices == null
+        ? null == locatedBlocks ? newLocatedBlock(eb, machines, pos, isCorrupt)
+            : locatedBlocks.newLocatedBlock(eb, machines, pos, isCorrupt)
+        : newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -1323,27 +1336,31 @@ public class BlockManager implements BlockStatsMXBean {
         LOG.debug("blocks = {}", java.util.Arrays.asList(blocks));
       }
       final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
-      final List<LocatedBlock> locatedblocks = createLocatedBlockList(
-          blocks, offset, length, Integer.MAX_VALUE, mode);
 
-      final LocatedBlock lastlb;
-      final boolean isComplete;
+      LocatedBlockBuilder locatedBlocks = providedStorageMap
+          .newLocatedBlocks(Integer.MAX_VALUE)
+          .fileLength(fileSizeExcludeBlocksUnderConstruction)
+          .lastUC(isFileUnderConstruction)
+          .encryption(feInfo)
+          .erasureCoding(ecPolicy);
+
+      createLocatedBlockList(locatedBlocks, blocks, offset, length, mode);
       if (!inSnapshot) {
         final BlockInfo last = blocks[blocks.length - 1];
         final long lastPos = last.isComplete()?
             fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
             : fileSizeExcludeBlocksUnderConstruction;
-        lastlb = createLocatedBlock(last, lastPos, mode);
-        isComplete = last.isComplete();
+
+        locatedBlocks
+          .lastBlock(createLocatedBlock(locatedBlocks, last, lastPos, mode))
+          .lastComplete(last.isComplete());
       } else {
-        lastlb = createLocatedBlock(blocks,
-            fileSizeExcludeBlocksUnderConstruction, mode);
-        isComplete = true;
+        locatedBlocks
+          .lastBlock(createLocatedBlock(locatedBlocks, blocks,
+              fileSizeExcludeBlocksUnderConstruction, mode))
+          .lastComplete(true);
       }
-      LocatedBlocks locations = new LocatedBlocks(
-          fileSizeExcludeBlocksUnderConstruction,
-          isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
-          ecPolicy);
+      LocatedBlocks locations = locatedBlocks.build();
       // Set caching information for the located blocks.
       CacheManager cm = namesystem.getCacheManager();
       if (cm != null) {
@@ -2442,7 +2459,10 @@ public class BlockManager implements BlockStatsMXBean {
 
       // To minimize startup time, we discard any second (or later) block reports
       // that we receive while still in startup phase.
-      DatanodeStorageInfo storageInfo = node.getStorageInfo(storage.getStorageID());
+      // !#! Register DN with provided storage, not with storage owned by DN
+      // !#! DN should still have a ref to the DNStorageInfo
+      DatanodeStorageInfo storageInfo =
+          providedStorageMap.getStorage(node, storage);
 
       if (storageInfo == null) {
         // We handle this for backwards compatibility.
@@ -2474,9 +2494,12 @@ public class BlockManager implements BlockStatsMXBean {
             nodeID.getDatanodeUuid());
         processFirstBlockReport(storageInfo, newReport);
       } else {
-        invalidatedBlocks = processReport(storageInfo, newReport, context);
+        // Block reports for provided storage are not
+        // maintained by DN heartbeats
+        if (!StorageType.PROVIDED.equals(storageInfo.getStorageType())) {
+          invalidatedBlocks = processReport(storageInfo, newReport, context);
+        }
       }
-      
       storageInfo.receivedBlockReport();
     } finally {
       endTime = Time.monotonicNow();
@@ -2690,7 +2713,7 @@ public class BlockManager implements BlockStatsMXBean {
    * @param report - the initial block report, to be processed
    * @throws IOException 
    */
-  private void processFirstBlockReport(
+  void processFirstBlockReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report) throws IOException {
     if (report == null) return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
new file mode 100644
index 0000000..d8bed16
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
@@ -0,0 +1,65 @@
+/**
+ * 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.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() throws IOException {
+    assert lock.hasWriteLock() : "Not holding write lock";
+    if (hasDNs) {
+      return;
+    }
+    LOG.info("Calling process first blk report from storage: " + storage);
+    // first pass; periodic refresh should call bm.processReport
+    bm.processFirstBlockReport(storage, new ProvidedBlockList(iterator()));
+    hasDNs = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
index c8923da..6ea5198 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
@@ -82,6 +82,12 @@ public class BlockStoragePolicySuite {
         HdfsConstants.COLD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.ARCHIVE}, StorageType.EMPTY_ARRAY,
         StorageType.EMPTY_ARRAY);
+    final byte providedId = HdfsConstants.PROVIDED_STORAGE_POLICY_ID;
+    policies[providedId] = new BlockStoragePolicy(providedId,
+      HdfsConstants.PROVIDED_STORAGE_POLICY_NAME,
+      new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
+      new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
+      new StorageType[]{StorageType.PROVIDED, StorageType.DISK});
     return new BlockStoragePolicySuite(hotId, policies);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index d35894c..28a3d1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -151,7 +151,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final LeavingServiceStatus leavingServiceStatus =
       new LeavingServiceStatus();
 
-  private final Map<String, DatanodeStorageInfo> storageMap =
+  protected final Map<String, DatanodeStorageInfo> storageMap =
       new HashMap<>();
 
   /**
@@ -322,6 +322,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
   boolean hasStaleStorages() {
     synchronized (storageMap) {
       for (DatanodeStorageInfo storage : storageMap.values()) {
+        if (StorageType.PROVIDED.equals(storage.getStorageType())) {
+          // to verify provided storage participated in this hb, requires
+          // check to pass DNDesc.
+          // e.g., storageInfo.verifyBlockReportId(this, curBlockReportId)
+          continue;
+        }
         if (storage.areBlockContentsStale()) {
           return true;
         }
@@ -443,17 +449,22 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.volumeFailures = volFailures;
     this.volumeFailureSummary = volumeFailureSummary;
     for (StorageReport report : reports) {
+      totalCapacity += report.getCapacity();
+      totalRemaining += report.getRemaining();
+      totalBlockPoolUsed += report.getBlockPoolUsed();
+      totalDfsUsed += report.getDfsUsed();
+      totalNonDfsUsed += report.getNonDfsUsed();
+
+      if (StorageType.PROVIDED.equals(
+          report.getStorage().getStorageType())) {
+        continue;
+      }
       DatanodeStorageInfo storage = updateStorage(report.getStorage());
       if (checkFailedStorages) {
         failedStorageInfos.remove(storage);
       }
 
       storage.receivedHeartbeat(report);
-      totalCapacity += report.getCapacity();
-      totalRemaining += report.getRemaining();
-      totalBlockPoolUsed += report.getBlockPoolUsed();
-      totalDfsUsed += report.getDfsUsed();
-      totalNonDfsUsed += report.getNonDfsUsed();
     }
 
     // Update total metrics for the node.
@@ -474,6 +485,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  void injectStorage(DatanodeStorageInfo s) {
+    synchronized (storageMap) {
+      DatanodeStorageInfo storage = storageMap.get(s.getStorageID());
+      if (null == storage) {
+        storageMap.put(s.getStorageID(), s);
+      } else {
+        assert storage == s : "found " + storage + " expected " + s;
+      }
+    }
+  }
+
   /**
    * Remove stale storages from storageMap. We must not remove any storages
    * as long as they have associated block replicas.

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index b1ccea2..76bf915 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -172,6 +172,10 @@ public class DatanodeStorageInfo {
     this.state = state;
   }
 
+  void setHeartbeatedSinceFailover(boolean value) {
+    heartbeatedSinceFailover = value;
+  }
+
   boolean areBlocksOnFailedStorage() {
     return getState() == State.FAILED && !blocks.isEmpty();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java
new file mode 100644
index 0000000..0056887
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java
@@ -0,0 +1,109 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class LocatedBlockBuilder {
+
+  protected long flen;
+  protected List<LocatedBlock> blocks = Collections.<LocatedBlock>emptyList();
+  protected boolean isUC;
+  protected LocatedBlock last;
+  protected boolean lastComplete;
+  protected FileEncryptionInfo feInfo;
+  private final int maxBlocks;
+  protected ErasureCodingPolicy ecPolicy;
+
+  LocatedBlockBuilder(int maxBlocks) {
+    this.maxBlocks = maxBlocks;
+  }
+
+  boolean isBlockMax() {
+    return blocks.size() >= maxBlocks;
+  }
+
+  LocatedBlockBuilder fileLength(long fileLength) {
+    flen = fileLength;
+    return this;
+  }
+
+  LocatedBlockBuilder addBlock(LocatedBlock block) {
+    if (blocks.isEmpty()) {
+      blocks = new ArrayList<>();
+    }
+    blocks.add(block);
+    return this;
+  }
+
+  // return new block so tokens can be set
+  LocatedBlock newLocatedBlock(ExtendedBlock eb,
+      DatanodeStorageInfo[] storage,
+      long pos, boolean isCorrupt) {
+    LocatedBlock blk =
+        BlockManager.newLocatedBlock(eb, storage, pos, isCorrupt);
+    return blk;
+  }
+
+  LocatedBlockBuilder lastUC(boolean underConstruction) {
+    isUC = underConstruction;
+    return this;
+  }
+
+  LocatedBlockBuilder lastBlock(LocatedBlock block) {
+    last = block;
+    return this;
+  }
+
+  LocatedBlockBuilder lastComplete(boolean complete) {
+    lastComplete = complete;
+    return this;
+  }
+
+  LocatedBlockBuilder encryption(FileEncryptionInfo fileEncryptionInfo) {
+    feInfo = fileEncryptionInfo;
+    return this;
+  }
+
+  LocatedBlockBuilder erasureCoding(ErasureCodingPolicy codingPolicy) {
+    ecPolicy = codingPolicy;
+    return this;
+  }
+
+  LocatedBlocks build(DatanodeDescriptor client) {
+    return build();
+  }
+
+  LocatedBlocks build() {
+    return new LocatedBlocks(flen, isUC, blocks, last,
+        lastComplete, feInfo, ecPolicy);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
new file mode 100644
index 0000000..d222344
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -0,0 +1,427 @@
+/**
+ * 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.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.util.RwLock;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.ByteString;
+
+/**
+ * This class allows us to manage and multiplex between storages local to
+ * datanodes, and provided storage.
+ */
+public class ProvidedStorageMap {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ProvidedStorageMap.class);
+
+  // limit to a single provider for now
+  private final BlockProvider blockProvider;
+  private final String storageId;
+  private final ProvidedDescriptor providedDescriptor;
+  private final DatanodeStorageInfo providedStorageInfo;
+  private boolean providedEnabled;
+
+  ProvidedStorageMap(RwLock lock, BlockManager bm, Configuration conf)
+      throws IOException {
+
+    storageId = conf.get(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+        DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
+
+    providedEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED,
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT);
+
+    if (!providedEnabled) {
+      // disable mapping
+      blockProvider = null;
+      providedDescriptor = null;
+      providedStorageInfo = null;
+      return;
+    }
+
+    DatanodeStorage ds = new DatanodeStorage(
+        storageId, State.NORMAL, StorageType.PROVIDED);
+    providedDescriptor = new ProvidedDescriptor();
+    providedStorageInfo = providedDescriptor.createProvidedStorage(ds);
+
+    // 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);
+  }
+
+  /**
+   * @param dn datanode descriptor
+   * @param s data node storage
+   * @return the {@link DatanodeStorageInfo} for the specified datanode.
+   * If {@code s} corresponds to a provided storage, the storage info
+   * representing provided storage is returned.
+   * @throws IOException
+   */
+  DatanodeStorageInfo getStorage(DatanodeDescriptor dn, DatanodeStorage s)
+      throws IOException {
+    if (providedEnabled && storageId.equals(s.getStorageID())) {
+      if (StorageType.PROVIDED.equals(s.getStorageType())) {
+        // poll service, initiate
+        blockProvider.start();
+        dn.injectStorage(providedStorageInfo);
+        return providedDescriptor.getProvidedStorage(dn, s);
+      }
+      LOG.warn("Reserved storage {} reported as non-provided from {}", s, dn);
+    }
+    return dn.getStorageInfo(s.getStorageID());
+  }
+
+  public LocatedBlockBuilder newLocatedBlocks(int maxValue) {
+    if (!providedEnabled) {
+      return new LocatedBlockBuilder(maxValue);
+    }
+    return new ProvidedBlocksBuilder(maxValue);
+  }
+
+  /**
+   * Builder used for creating {@link LocatedBlocks} when a block is provided.
+   */
+  class ProvidedBlocksBuilder extends LocatedBlockBuilder {
+
+    private ShadowDatanodeInfoWithStorage pending;
+
+    ProvidedBlocksBuilder(int maxBlocks) {
+      super(maxBlocks);
+      pending = new ShadowDatanodeInfoWithStorage(
+          providedDescriptor, storageId);
+    }
+
+    @Override
+    LocatedBlock newLocatedBlock(ExtendedBlock eb,
+        DatanodeStorageInfo[] storages, long pos, boolean isCorrupt) {
+
+      DatanodeInfoWithStorage[] locs =
+        new DatanodeInfoWithStorage[storages.length];
+      String[] sids = new String[storages.length];
+      StorageType[] types = new StorageType[storages.length];
+      for (int i = 0; i < storages.length; ++i) {
+        sids[i] = storages[i].getStorageID();
+        types[i] = storages[i].getStorageType();
+        if (StorageType.PROVIDED.equals(storages[i].getStorageType())) {
+          locs[i] = pending;
+        } else {
+          locs[i] = new DatanodeInfoWithStorage(
+              storages[i].getDatanodeDescriptor(), sids[i], types[i]);
+        }
+      }
+      return new LocatedBlock(eb, locs, sids, types, pos, isCorrupt, null);
+    }
+
+    @Override
+    LocatedBlocks build(DatanodeDescriptor client) {
+      // TODO: to support multiple provided storages, need to pass/maintain map
+      // set all fields of pending DatanodeInfo
+      List<String> excludedUUids = new ArrayList<String>();
+      for (LocatedBlock b: blocks) {
+        DatanodeInfo[] infos = b.getLocations();
+        StorageType[] types = b.getStorageTypes();
+
+        for (int i = 0; i < types.length; i++) {
+          if (!StorageType.PROVIDED.equals(types[i])) {
+            excludedUUids.add(infos[i].getDatanodeUuid());
+          }
+        }
+      }
+
+      DatanodeDescriptor dn = providedDescriptor.choose(client, excludedUUids);
+      if (dn == null) {
+        dn = providedDescriptor.choose(client);
+      }
+
+      pending.replaceInternal(dn);
+      return new LocatedBlocks(
+          flen, isUC, blocks, last, lastComplete, feInfo, ecPolicy);
+    }
+
+    @Override
+    LocatedBlocks build() {
+      return build(providedDescriptor.chooseRandom());
+    }
+  }
+
+  /**
+   * An abstract {@link DatanodeInfoWithStorage} to represent provided storage.
+   */
+  static class ShadowDatanodeInfoWithStorage extends DatanodeInfoWithStorage {
+    private String shadowUuid;
+
+    ShadowDatanodeInfoWithStorage(DatanodeDescriptor d, String storageId) {
+      super(d, storageId, StorageType.PROVIDED);
+    }
+
+    @Override
+    public String getDatanodeUuid() {
+      return shadowUuid;
+    }
+
+    public void setDatanodeUuid(String uuid) {
+      shadowUuid = uuid;
+    }
+
+    void replaceInternal(DatanodeDescriptor dn) {
+      updateRegInfo(dn); // overwrite DatanodeID (except UUID)
+      setDatanodeUuid(dn.getDatanodeUuid());
+      setCapacity(dn.getCapacity());
+      setDfsUsed(dn.getDfsUsed());
+      setRemaining(dn.getRemaining());
+      setBlockPoolUsed(dn.getBlockPoolUsed());
+      setCacheCapacity(dn.getCacheCapacity());
+      setCacheUsed(dn.getCacheUsed());
+      setLastUpdate(dn.getLastUpdate());
+      setLastUpdateMonotonic(dn.getLastUpdateMonotonic());
+      setXceiverCount(dn.getXceiverCount());
+      setNetworkLocation(dn.getNetworkLocation());
+      adminState = dn.getAdminState();
+      setUpgradeDomain(dn.getUpgradeDomain());
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return super.equals(obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return super.hashCode();
+    }
+  }
+
+  /**
+   * An abstract DatanodeDescriptor to track datanodes with provided storages.
+   * NOTE: never resolved through registerDatanode, so not in the topology.
+   */
+  static class ProvidedDescriptor extends DatanodeDescriptor {
+
+    private final NavigableMap<String, DatanodeDescriptor> dns =
+        new ConcurrentSkipListMap<>();
+
+    ProvidedDescriptor() {
+      super(new DatanodeID(
+            null,                         // String ipAddr,
+            null,                         // String hostName,
+            UUID.randomUUID().toString(), // String datanodeUuid,
+            0,                            // int xferPort,
+            0,                            // int infoPort,
+            0,                            // int infoSecurePort,
+            0));                          // int ipcPort
+    }
+
+    DatanodeStorageInfo getProvidedStorage(
+        DatanodeDescriptor dn, DatanodeStorage s) {
+      dns.put(dn.getDatanodeUuid(), dn);
+      // TODO: maintain separate RPC ident per dn
+      return storageMap.get(s.getStorageID());
+    }
+
+    DatanodeStorageInfo createProvidedStorage(DatanodeStorage ds) {
+      assert null == storageMap.get(ds.getStorageID());
+      DatanodeStorageInfo storage = new DatanodeStorageInfo(this, ds);
+      storage.setHeartbeatedSinceFailover(true);
+      storageMap.put(storage.getStorageID(), storage);
+      return storage;
+    }
+
+    DatanodeDescriptor choose(DatanodeDescriptor client) {
+      // exact match for now
+      DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
+      if (null == dn) {
+        dn = chooseRandom();
+      }
+      return dn;
+    }
+
+    DatanodeDescriptor choose(DatanodeDescriptor client,
+        List<String> excludedUUids) {
+      // exact match for now
+      DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
+
+      if (null == dn || excludedUUids.contains(client.getDatanodeUuid())) {
+        dn = null;
+        Set<String> exploredUUids = new HashSet<String>();
+
+        while(exploredUUids.size() < dns.size()) {
+          Map.Entry<String, DatanodeDescriptor> d =
+                  dns.ceilingEntry(UUID.randomUUID().toString());
+          if (null == d) {
+            d = dns.firstEntry();
+          }
+          String uuid = d.getValue().getDatanodeUuid();
+          //this node has already been explored, and was not selected earlier
+          if (exploredUUids.contains(uuid)) {
+            continue;
+          }
+          exploredUUids.add(uuid);
+          //this node has been excluded
+          if (excludedUUids.contains(uuid)) {
+            continue;
+          }
+          return dns.get(uuid);
+        }
+      }
+
+      return dn;
+    }
+
+    DatanodeDescriptor chooseRandom(DatanodeStorageInfo[] excludedStorages) {
+      // TODO: Currently this is not uniformly random;
+      // skewed toward sparse sections of the ids
+      Set<DatanodeDescriptor> excludedNodes =
+          new HashSet<DatanodeDescriptor>();
+      if (excludedStorages != null) {
+        for (int i= 0; i < excludedStorages.length; i++) {
+          LOG.info("Excluded: " + excludedStorages[i].getDatanodeDescriptor());
+          excludedNodes.add(excludedStorages[i].getDatanodeDescriptor());
+        }
+      }
+      Set<DatanodeDescriptor> exploredNodes = new HashSet<DatanodeDescriptor>();
+
+      while(exploredNodes.size() < dns.size()) {
+        Map.Entry<String, DatanodeDescriptor> d =
+            dns.ceilingEntry(UUID.randomUUID().toString());
+        if (null == d) {
+          d = dns.firstEntry();
+        }
+        DatanodeDescriptor node = d.getValue();
+        //this node has already been explored, and was not selected earlier
+        if (exploredNodes.contains(node)) {
+          continue;
+        }
+        exploredNodes.add(node);
+        //this node has been excluded
+        if (excludedNodes.contains(node)) {
+          continue;
+        }
+        return node;
+      }
+      return null;
+    }
+
+    DatanodeDescriptor chooseRandom() {
+      return chooseRandom(null);
+    }
+
+    @Override
+    void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) {
+      // pick a random datanode, delegate to it
+      DatanodeDescriptor node = chooseRandom(targets);
+      if (node != null) {
+        node.addBlockToBeReplicated(block, targets);
+      } else {
+        LOG.error("Cannot find a source node to replicate block: "
+            + block + " from");
+      }
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return (this == obj) || super.equals(obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return super.hashCode();
+    }
+  }
+
+  /**
+   * Used to emulate block reports for provided blocks.
+   */
+  static class ProvidedBlockList extends BlockListAsLongs {
+
+    private final Iterator<Block> inner;
+
+    ProvidedBlockList(Iterator<Block> inner) {
+      this.inner = inner;
+    }
+
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return new Iterator<BlockReportReplica>() {
+        @Override
+        public BlockReportReplica next() {
+          return new BlockReportReplica(inner.next());
+        }
+        @Override
+        public boolean hasNext() {
+          return inner.hasNext();
+        }
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+
+    @Override
+    public int getNumberOfBlocks() {
+      // VERIFY: only printed for debugging
+      return -1;
+    }
+
+    @Override
+    public ByteString getBlocksBuffer() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long[] getBlockListAsLongs() {
+      // should only be used for backwards compat, DN.ver > NN.ver
+      throw new UnsupportedOperationException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 169dfc2..0f1407a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4622,14 +4622,30 @@
   </property>
 
   <property>
+    <name>dfs.namenode.provided.enabled</name>
+    <value>false</value>
+    <description>
+      Enables the Namenode to handle provided storages.
+    </description>
+  </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.
+      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>
 
@@ -4637,7 +4653,7 @@
     <name>dfs.provided.df.class</name>
     <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
     <description>
-        The class that is used to measure usage statistics of provided stores.
+      The class that is used to measure usage statistics of provided stores.
     </description>
   </property>
 
@@ -4645,7 +4661,7 @@
     <name>dfs.provided.storage.id</name>
     <value>DS-PROVIDED</value>
     <description>
-        The storage ID used for provided stores.
+      The storage ID used for provided stores.
     </description>
   </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index ae256a5..55a7b3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -84,6 +84,7 @@ public class TestBlockStoragePolicy {
   static final byte ONESSD  = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
   static final byte ALLSSD  = HdfsConstants.ALLSSD_STORAGE_POLICY_ID;
   static final byte LAZY_PERSIST  = HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+  static final byte PROVIDED  = HdfsConstants.PROVIDED_STORAGE_POLICY_ID;
 
   @Test (timeout=300000)
   public void testConfigKeyEnabled() throws IOException {
@@ -143,6 +144,9 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(ALLSSD, "BlockStoragePolicy{ALL_SSD:" + ALLSSD +
         ", storageTypes=[SSD], creationFallbacks=[DISK], " +
         "replicationFallbacks=[DISK]}");
+    expectedPolicyStrings.put(PROVIDED, "BlockStoragePolicy{PROVIDED:" + PROVIDED +
+        ", storageTypes=[PROVIDED, DISK], creationFallbacks=[PROVIDED, DISK], " +
+        "replicationFallbacks=[PROVIDED, DISK]}");
 
     for(byte i = 1; i < 16; i++) {
       final BlockStoragePolicy policy = POLICY_SUITE.getPolicy(i); 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index 286f4a4..81405eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -300,7 +300,7 @@ public class TestDatanodeManager {
    */
   @Test
   public void testSortLocatedBlocks() throws IOException, URISyntaxException {
-    HelperFunction(null);
+    HelperFunction(null, 0);
   }
 
   /**
@@ -312,7 +312,7 @@ public class TestDatanodeManager {
    */
   @Test
   public void testgoodScript() throws IOException, URISyntaxException {
-    HelperFunction("/" + Shell.appendScriptExtension("topology-script"));
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"), 0);
   }
 
 
@@ -325,7 +325,21 @@ public class TestDatanodeManager {
    */
   @Test
   public void testBadScript() throws IOException, URISyntaxException {
-    HelperFunction("/"+ Shell.appendScriptExtension("topology-broken-script"));
+    HelperFunction("/"+ Shell.appendScriptExtension("topology-broken-script"), 0);
+  }
+
+  /**
+   * Test with different sorting functions but include datanodes
+   * with provided storage
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  @Test
+  public void testWithProvidedTypes() throws IOException, URISyntaxException {
+    HelperFunction(null, 1);
+    HelperFunction(null, 3);
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"), 1);
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"), 2);
   }
 
   /**
@@ -333,11 +347,12 @@ public class TestDatanodeManager {
    * we invoke this function with and without topology scripts
    *
    * @param scriptFileName - Script Name or null
+   * @param providedStorages - number of provided storages to add
    *
    * @throws URISyntaxException
    * @throws IOException
    */
-  public void HelperFunction(String scriptFileName)
+  public void HelperFunction(String scriptFileName, int providedStorages)
     throws URISyntaxException, IOException {
     // create the DatanodeManager which will be tested
     Configuration conf = new Configuration();
@@ -352,17 +367,25 @@ public class TestDatanodeManager {
     }
     DatanodeManager dm = mockDatanodeManager(fsn, conf);
 
+    int totalDNs = 5 + providedStorages;
+
     // register 5 datanodes, each with different storage ID and type
-    DatanodeInfo[] locs = new DatanodeInfo[5];
-    String[] storageIDs = new String[5];
-    StorageType[] storageTypes = new StorageType[]{
-      StorageType.ARCHIVE,
-      StorageType.DEFAULT,
-      StorageType.DISK,
-      StorageType.RAM_DISK,
-      StorageType.SSD
-    };
-    for (int i = 0; i < 5; i++) {
+    DatanodeInfo[] locs = new DatanodeInfo[totalDNs];
+    String[] storageIDs = new String[totalDNs];
+    List<StorageType> storageTypesList = new ArrayList<>(
+        Arrays.asList(StorageType.ARCHIVE,
+            StorageType.DEFAULT,
+            StorageType.DISK,
+            StorageType.RAM_DISK,
+            StorageType.SSD));
+
+    for (int i = 0; i < providedStorages; i++) {
+      storageTypesList.add(StorageType.PROVIDED);
+    }
+
+    StorageType[] storageTypes= storageTypesList.toArray(new StorageType[0]);
+
+    for (int i = 0; i < totalDNs; i++) {
       // register new datanode
       String uuid = "UUID-" + i;
       String ip = "IP-" + i;
@@ -398,9 +421,9 @@ public class TestDatanodeManager {
     DatanodeInfo[] sortedLocs = block.getLocations();
     storageIDs = block.getStorageIDs();
     storageTypes = block.getStorageTypes();
-    assertThat(sortedLocs.length, is(5));
-    assertThat(storageIDs.length, is(5));
-    assertThat(storageTypes.length, is(5));
+    assertThat(sortedLocs.length, is(totalDNs));
+    assertThat(storageIDs.length, is(totalDNs));
+    assertThat(storageTypes.length, is(totalDNs));
     for (int i = 0; i < sortedLocs.length; i++) {
       assertThat(((DatanodeInfoWithStorage) sortedLocs[i]).getStorageID(),
         is(storageIDs[i]));
@@ -414,6 +437,14 @@ public class TestDatanodeManager {
       is(DatanodeInfo.AdminStates.DECOMMISSIONED));
     assertThat(sortedLocs[sortedLocs.length - 2].getAdminState(),
       is(DatanodeInfo.AdminStates.DECOMMISSIONED));
+    // check that the StorageType of datanoodes immediately
+    // preceding the decommissioned datanodes is PROVIDED
+    for (int i = 0; i < providedStorages; i++) {
+      assertThat(
+          ((DatanodeInfoWithStorage)
+              sortedLocs[sortedLocs.length - 3 - i]).getStorageType(),
+          is(StorageType.PROVIDED));
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fee8ad0/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
new file mode 100644
index 0000000..3b75806
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -0,0 +1,345 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Random;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
+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 static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+public class TestNameNodeProvidedImplementation {
+
+  @Rule public TestName name = new TestName();
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestNameNodeProvidedImplementation.class);
+
+  final Random r = new Random();
+  final File fBASE = new File(MiniDFSCluster.getBaseDirectory());
+  final Path BASE = new Path(fBASE.toURI().toString());
+  final Path NAMEPATH = new Path(BASE, "providedDir");;
+  final Path NNDIRPATH = new Path(BASE, "nnDir");
+  final Path BLOCKFILE = new Path(NNDIRPATH, "blocks.csv");
+  final String SINGLEUSER = "usr1";
+  final String SINGLEGROUP = "grp1";
+
+  Configuration conf;
+  MiniDFSCluster cluster;
+
+  @Before
+  public void setSeed() throws Exception {
+    if (fBASE.exists() && !FileUtil.fullyDelete(fBASE)) {
+      throw new IOException("Could not fully delete " + fBASE);
+    }
+    long seed = r.nextLong();
+    r.setSeed(seed);
+    System.out.println(name.getMethodName() + " seed: " + seed);
+    conf = new HdfsConfiguration();
+    conf.set(SingleUGIResolver.USER, SINGLEUSER);
+    conf.set(SingleUGIResolver.GROUP, SINGLEGROUP);
+
+    conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+        DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
+
+    conf.setClass(DFSConfigKeys.DFS_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,
+        BLOCKFILE.toString());
+    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
+        BLOCKFILE.toString());
+    conf.set(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER, ",");
+
+    File imageDir = new File(NAMEPATH.toUri());
+    if (!imageDir.exists()) {
+      LOG.info("Creating directory: " + imageDir);
+      imageDir.mkdirs();
+    }
+
+    File nnDir = new File(NNDIRPATH.toUri());
+    if (!nnDir.exists()) {
+      nnDir.mkdirs();
+    }
+
+    // create 10 random files under BASE
+    for (int i=0; i < 10; i++) {
+      File newFile = new File(new Path(NAMEPATH, "file" + i).toUri());
+      if(!newFile.exists()) {
+        try {
+          LOG.info("Creating " + newFile.toString());
+          newFile.createNewFile();
+          Writer writer = new OutputStreamWriter(
+              new FileOutputStream(newFile.getAbsolutePath()), "utf-8");
+          for(int j=0; j < 10*i; j++) {
+            writer.write("0");
+          }
+          writer.flush();
+          writer.close();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    try {
+      if (cluster != null) {
+        cluster.shutdown(true, true);
+      }
+    } finally {
+      cluster = null;
+    }
+  }
+
+  void createImage(TreeWalk t, Path out,
+      Class<? extends BlockResolver> blockIdsClass) throws Exception {
+    ImageWriter.Options opts = ImageWriter.defaults();
+    opts.setConf(conf);
+    opts.output(out.toString())
+        .blocks(TextFileRegionFormat.class)
+        .blockIds(blockIdsClass);
+    try (ImageWriter w = new ImageWriter(opts)) {
+      for (TreePath e : t) {
+        w.accept(e);
+      }
+    }
+  }
+
+  void startCluster(Path nspath, int numDatanodes,
+      StorageType[] storageTypes,
+      StorageType[][] storageTypesPerDatanode)
+      throws IOException {
+    conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString());
+
+    if (storageTypesPerDatanode != null) {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .format(false)
+          .manageNameDfsDirs(false)
+          .numDataNodes(numDatanodes)
+          .storageTypes(storageTypesPerDatanode)
+          .build();
+    } else if (storageTypes != null) {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .format(false)
+          .manageNameDfsDirs(false)
+          .numDataNodes(numDatanodes)
+          .storagesPerDatanode(storageTypes.length)
+          .storageTypes(storageTypes)
+          .build();
+    } else {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .format(false)
+          .manageNameDfsDirs(false)
+          .numDataNodes(numDatanodes)
+          .build();
+    }
+    cluster.waitActive();
+  }
+
+  @Test(timeout = 20000)
+  public void testLoadImage() throws Exception {
+    final long seed = r.nextLong();
+    LOG.info("NAMEPATH: " + NAMEPATH);
+    createImage(new RandomTreeWalk(seed), NNDIRPATH, FixedBlockResolver.class);
+    startCluster(NNDIRPATH, 0, new StorageType[] {StorageType.PROVIDED}, null);
+
+    FileSystem fs = cluster.getFileSystem();
+    for (TreePath e : new RandomTreeWalk(seed)) {
+      FileStatus rs = e.getFileStatus();
+      Path hp = new Path(rs.getPath().toUri().getPath());
+      assertTrue(fs.exists(hp));
+      FileStatus hs = fs.getFileStatus(hp);
+      assertEquals(rs.getPath().toUri().getPath(),
+                   hs.getPath().toUri().getPath());
+      assertEquals(rs.getPermission(), hs.getPermission());
+      assertEquals(rs.getLen(), hs.getLen());
+      assertEquals(SINGLEUSER, hs.getOwner());
+      assertEquals(SINGLEGROUP, hs.getGroup());
+      assertEquals(rs.getAccessTime(), hs.getAccessTime());
+      assertEquals(rs.getModificationTime(), hs.getModificationTime());
+    }
+  }
+
+  @Test(timeout=20000)
+  public void testBlockLoad() throws Exception {
+    conf.setClass(ImageWriter.Options.UGI_CLASS,
+        SingleUGIResolver.class, UGIResolver.class);
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    startCluster(NNDIRPATH, 1, new StorageType[] {StorageType.PROVIDED}, null);
+  }
+
+  @Test(timeout=500000)
+  public void testDefaultReplication() throws Exception {
+    int targetReplication = 2;
+    conf.setInt(FixedBlockMultiReplicaResolver.REPLICATION, targetReplication);
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockMultiReplicaResolver.class);
+    // make the last Datanode with only DISK
+    startCluster(NNDIRPATH, 3, null,
+        new StorageType[][] {
+          {StorageType.PROVIDED},
+          {StorageType.PROVIDED},
+          {StorageType.DISK}}
+        );
+    // wait for the replication to finish
+    Thread.sleep(50000);
+
+    FileSystem fs = cluster.getFileSystem();
+    int count = 0;
+    for (TreePath e : new FSTreeWalk(NAMEPATH, conf)) {
+      FileStatus rs = e.getFileStatus();
+      Path hp = removePrefix(NAMEPATH, rs.getPath());
+      LOG.info("hp " + hp.toUri().getPath());
+      //skip HDFS specific files, which may have been created later on.
+      if (hp.toString().contains("in_use.lock")
+          || hp.toString().contains("current")) {
+        continue;
+      }
+      e.accept(count++);
+      assertTrue(fs.exists(hp));
+      FileStatus hs = fs.getFileStatus(hp);
+
+      if (rs.isFile()) {
+        BlockLocation[] bl = fs.getFileBlockLocations(
+            hs.getPath(), 0, hs.getLen());
+        int i = 0;
+        for(; i < bl.length; i++) {
+          int currentRep = bl[i].getHosts().length;
+          assertEquals(targetReplication , currentRep);
+        }
+      }
+    }
+  }
+
+
+  static Path removePrefix(Path base, Path walk) {
+    Path wpath = new Path(walk.toUri().getPath());
+    Path bpath = new Path(base.toUri().getPath());
+    Path ret = new Path("/");
+    while (!(bpath.equals(wpath) || "".equals(wpath.getName()))) {
+      ret = "".equals(ret.getName())
+        ? new Path("/", wpath.getName())
+        : new Path(new Path("/", wpath.getName()),
+                   new Path(ret.toString().substring(1)));
+      wpath = wpath.getParent();
+    }
+    if (!bpath.equals(wpath)) {
+      throw new IllegalArgumentException(base + " not a prefix of " + walk);
+    }
+    return ret;
+  }
+
+  @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}, null);
+    FileSystem fs = cluster.getFileSystem();
+    Thread.sleep(2000);
+    int count = 0;
+    // read NN metadata, verify contents match
+    for (TreePath e : new FSTreeWalk(NAMEPATH, conf)) {
+      FileStatus rs = e.getFileStatus();
+      Path hp = removePrefix(NAMEPATH, rs.getPath());
+      LOG.info("hp " + hp.toUri().getPath());
+      //skip HDFS specific files, which may have been created later on.
+      if(hp.toString().contains("in_use.lock")
+          || hp.toString().contains("current")) {
+        continue;
+      }
+      e.accept(count++);
+      assertTrue(fs.exists(hp));
+      FileStatus hs = fs.getFileStatus(hp);
+      assertEquals(hp.toUri().getPath(), hs.getPath().toUri().getPath());
+      assertEquals(rs.getPermission(), hs.getPermission());
+      assertEquals(rs.getOwner(), hs.getOwner());
+      assertEquals(rs.getGroup(), hs.getGroup());
+
+      if (rs.isFile()) {
+        assertEquals(rs.getLen(), hs.getLen());
+        try (ReadableByteChannel i = Channels.newChannel(
+              new FileInputStream(new File(rs.getPath().toUri())))) {
+          try (ReadableByteChannel j = Channels.newChannel(
+                fs.open(hs.getPath()))) {
+            ByteBuffer ib = ByteBuffer.allocate(4096);
+            ByteBuffer jb = ByteBuffer.allocate(4096);
+            while (true) {
+              int il = i.read(ib);
+              int jl = j.read(jb);
+              if (il < 0 || jl < 0) {
+                assertEquals(il, jl);
+                break;
+              }
+              ib.flip();
+              jb.flip();
+              int cmp = Math.min(ib.remaining(), jb.remaining());
+              for (int k = 0; k < cmp; ++k) {
+                assertEquals(ib.get(), jb.get());
+              }
+              ib.compact();
+              jb.compact();
+            }
+
+          }
+        }
+      }
+    }
+  }
+}


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 1682da11c36e89586f3d8c55d03661ef970e54f4
Parents: 69e3086
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 30 10:37:28 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
new file mode 100644
index 0000000..eee58ba
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.fs.Path;
+
+import javax.annotation.Nonnull;
+import java.util.Arrays;
+
+/**
+ * ProvidedStorageLocation is a location in an external storage system
+ * containing the data for a block (~Replica).
+ */
+public class ProvidedStorageLocation {
+  private final Path path;
+  private final long offset;
+  private final long length;
+  private final byte[] nonce;
+
+  public ProvidedStorageLocation(Path path, long offset, long length,
+      byte[] nonce) {
+    this.path = path;
+    this.offset = offset;
+    this.length = length;
+    this.nonce = Arrays.copyOf(nonce, nonce.length);
+  }
+
+  public @Nonnull Path getPath() {
+    return path;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public @Nonnull byte[] getNonce() {
+    // create a copy of the nonce and return it.
+    return Arrays.copyOf(nonce, nonce.length);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ProvidedStorageLocation that = (ProvidedStorageLocation) o;
+
+    if ((offset != that.offset) || (length != that.length)
+        || !path.equals(that.path)) {
+      return false;
+    }
+    return Arrays.equals(nonce, that.nonce);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = path.hashCode();
+    result = 31 * result + (int) (offset ^ (offset >>> 32));
+    result = 31 * result + (int) (length ^ (length >>> 32));
+    result = 31 * result + Arrays.hashCode(nonce);
+    return result;
+  }
+}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
new file mode 100644
index 0000000..98b3ee1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos;
+import org.apache.hadoop.ipc.ProtocolInfo;
+
+/**
+ * Protocol between the Namenode and the Datanode to read the AliasMap
+ * used for Provided storage.
+ * TODO add Kerberos support
+ */
+@ProtocolInfo(
+    protocolName =
+        "org.apache.hadoop.hdfs.server.aliasmap.AliasMapProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface AliasMapProtocolPB extends
+    AliasMapProtocolProtos.AliasMapProtocolService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..808c43b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.ReadResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.WriteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.WriteResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.*;
+
+/**
+ * AliasMapProtocolServerSideTranslatorPB is responsible for translating RPC
+ * calls and forwarding them to the internal InMemoryAliasMap.
+ */
+public class AliasMapProtocolServerSideTranslatorPB
+    implements AliasMapProtocolPB {
+
+  private final InMemoryAliasMapProtocol aliasMap;
+
+  public AliasMapProtocolServerSideTranslatorPB(
+      InMemoryAliasMapProtocol aliasMap) {
+    this.aliasMap = aliasMap;
+  }
+
+  private static final WriteResponseProto VOID_WRITE_RESPONSE =
+      WriteResponseProto.newBuilder().build();
+
+  @Override
+  public WriteResponseProto write(RpcController controller,
+      WriteRequestProto request) throws ServiceException {
+    try {
+      FileRegion toWrite =
+          PBHelper.convert(request.getKeyValuePair());
+
+      aliasMap.write(toWrite.getBlock(), toWrite.getProvidedStorageLocation());
+      return VOID_WRITE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReadResponseProto read(RpcController controller,
+      ReadRequestProto request) throws ServiceException {
+    try {
+      Block toRead =  PBHelperClient.convert(request.getKey());
+
+      Optional<ProvidedStorageLocation> optionalResult =
+          aliasMap.read(toRead);
+
+      ReadResponseProto.Builder builder = ReadResponseProto.newBuilder();
+      if (optionalResult.isPresent()) {
+        ProvidedStorageLocation providedStorageLocation = optionalResult.get();
+        builder.setValue(PBHelperClient.convert(providedStorageLocation));
+      }
+
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListResponseProto list(RpcController controller,
+      ListRequestProto request) throws ServiceException {
+    try {
+      BlockProto marker = request.getMarker();
+      IterationResult iterationResult;
+      if (marker.isInitialized()) {
+        iterationResult =
+            aliasMap.list(Optional.of(PBHelperClient.convert(marker)));
+      } else {
+        iterationResult = aliasMap.list(Optional.empty());
+      }
+      ListResponseProto.Builder responseBuilder =
+          ListResponseProto.newBuilder();
+      List<FileRegion> fileRegions = iterationResult.getFileRegions();
+
+      List<KeyValueProto> keyValueProtos = fileRegions.stream()
+          .map(PBHelper::convert).collect(Collectors.toList());
+      responseBuilder.addAllFileRegions(keyValueProtos);
+      Optional<Block> nextMarker = iterationResult.getNextBlock();
+      nextMarker
+          .map(m -> responseBuilder.setNextMarker(PBHelperClient.convert(m)));
+
+      return responseBuilder.build();
+
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..a79360f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.*;
+
+/**
+ * This class is the client side translator to translate requests made to the
+ * {@link InMemoryAliasMapProtocol} interface to the RPC server implementing
+ * {@link AliasMapProtocolPB}.
+ */
+public class InMemoryAliasMapProtocolClientSideTranslatorPB
+    implements InMemoryAliasMapProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory
+          .getLogger(InMemoryAliasMapProtocolClientSideTranslatorPB.class);
+
+  private AliasMapProtocolPB rpcProxy;
+
+  public InMemoryAliasMapProtocolClientSideTranslatorPB(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT);
+    InetSocketAddress aliasMapAddr = NetUtils.createSocketAddr(addr);
+
+    RPC.setProtocolEngine(conf, AliasMapProtocolPB.class,
+        ProtobufRpcEngine.class);
+    LOG.info("Connecting to address: " + addr);
+    try {
+      rpcProxy = RPC.getProxy(AliasMapProtocolPB.class,
+          RPC.getProtocolVersion(AliasMapProtocolPB.class), aliasMapAddr, null,
+          conf, NetUtils.getDefaultSocketFactory(conf), 0);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Override
+  public InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException {
+    ListRequestProto.Builder builder = ListRequestProto.newBuilder();
+    if (marker.isPresent()) {
+      builder.setMarker(PBHelperClient.convert(marker.get()));
+    }
+    ListRequestProto request = builder.build();
+    try {
+      ListResponseProto response = rpcProxy.list(null, request);
+      List<KeyValueProto> fileRegionsList = response.getFileRegionsList();
+
+      List<FileRegion> fileRegions = fileRegionsList
+          .stream()
+          .map(kv -> new FileRegion(
+              PBHelperClient.convert(kv.getKey()),
+              PBHelperClient.convert(kv.getValue()),
+              null
+          ))
+          .collect(Collectors.toList());
+      BlockProto nextMarker = response.getNextMarker();
+
+      if (nextMarker.isInitialized()) {
+        return new InMemoryAliasMap.IterationResult(fileRegions,
+            Optional.of(PBHelperClient.convert(nextMarker)));
+      } else {
+        return new InMemoryAliasMap.IterationResult(fileRegions,
+            Optional.empty());
+      }
+
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Nonnull
+  @Override
+  public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+
+    ReadRequestProto request =
+        ReadRequestProto
+            .newBuilder()
+            .setKey(PBHelperClient.convert(block))
+            .build();
+    try {
+      ReadResponseProto response = rpcProxy.read(null, request);
+
+      ProvidedStorageLocationProto providedStorageLocation =
+          response.getValue();
+      if (providedStorageLocation.isInitialized()) {
+        return Optional.of(PBHelperClient.convert(providedStorageLocation));
+      }
+      return Optional.empty();
+
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    WriteRequestProto request =
+        WriteRequestProto
+            .newBuilder()
+            .setKeyValuePair(KeyValueProto.newBuilder()
+                .setKey(PBHelperClient.convert(block))
+                .setValue(PBHelperClient.convert(providedStorageLocation))
+                .build())
+            .build();
+
+    try {
+      rpcProxy.write(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  public void stop() {
+    RPC.stopProxy(rpcProxy);
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
new file mode 100644
index 0000000..be891e5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.aliasmap;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.fusesource.leveldbjni.JniDBFactory;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import org.iq80.leveldb.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * InMemoryAliasMap is an implementation of the InMemoryAliasMapProtocol for
+ * use with LevelDB.
+ */
+public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
+    Configurable {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(InMemoryAliasMap.class);
+
+  private final DB levelDb;
+  private Configuration conf;
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @VisibleForTesting
+  static String createPathErrorMessage(String directory) {
+    return new StringBuilder()
+        .append("Configured directory '")
+        .append(directory)
+        .append("' doesn't exist")
+        .toString();
+  }
+
+  public static @Nonnull InMemoryAliasMap init(Configuration conf)
+      throws IOException {
+    Options options = new Options();
+    options.createIfMissing(true);
+    String directory =
+        conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
+    LOG.info("Attempting to load InMemoryAliasMap from \"{}\"", directory);
+    File path = new File(directory);
+    if (!path.exists()) {
+      String error = createPathErrorMessage(directory);
+      throw new IOException(error);
+    }
+    DB levelDb = JniDBFactory.factory.open(path, options);
+    InMemoryAliasMap aliasMap = new InMemoryAliasMap(levelDb);
+    aliasMap.setConf(conf);
+    return aliasMap;
+  }
+
+  @VisibleForTesting
+  InMemoryAliasMap(DB levelDb) {
+    this.levelDb = levelDb;
+  }
+
+  @Override
+  public IterationResult list(Optional<Block> marker) throws IOException {
+    return withIterator((DBIterator iterator) -> {
+      Integer batchSize =
+          conf.getInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT);
+      if (marker.isPresent()) {
+        iterator.seek(toProtoBufBytes(marker.get()));
+      } else {
+        iterator.seekToFirst();
+      }
+      int i = 0;
+      ArrayList<FileRegion> batch =
+          Lists.newArrayListWithExpectedSize(batchSize);
+      while (iterator.hasNext() && i < batchSize) {
+        Map.Entry<byte[], byte[]> entry = iterator.next();
+        Block block = fromBlockBytes(entry.getKey());
+        ProvidedStorageLocation providedStorageLocation =
+            fromProvidedStorageLocationBytes(entry.getValue());
+        batch.add(new FileRegion(block, providedStorageLocation, null));
+        ++i;
+      }
+      if (iterator.hasNext()) {
+        Block nextMarker = fromBlockBytes(iterator.next().getKey());
+        return new IterationResult(batch, Optional.of(nextMarker));
+      } else {
+        return new IterationResult(batch, Optional.empty());
+      }
+
+    });
+  }
+
+  public @Nonnull Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+
+    byte[] extendedBlockDbFormat = toProtoBufBytes(block);
+    byte[] providedStorageLocationDbFormat = levelDb.get(extendedBlockDbFormat);
+    if (providedStorageLocationDbFormat == null) {
+      return Optional.empty();
+    } else {
+      ProvidedStorageLocation providedStorageLocation =
+          fromProvidedStorageLocationBytes(providedStorageLocationDbFormat);
+      return Optional.of(providedStorageLocation);
+    }
+  }
+
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    byte[] extendedBlockDbFormat = toProtoBufBytes(block);
+    byte[] providedStorageLocationDbFormat =
+        toProtoBufBytes(providedStorageLocation);
+    levelDb.put(extendedBlockDbFormat, providedStorageLocationDbFormat);
+  }
+
+  public void close() throws IOException {
+    levelDb.close();
+  }
+
+  @Nonnull
+  public static ProvidedStorageLocation fromProvidedStorageLocationBytes(
+      @Nonnull byte[] providedStorageLocationDbFormat)
+      throws InvalidProtocolBufferException {
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        ProvidedStorageLocationProto
+            .parseFrom(providedStorageLocationDbFormat);
+    return PBHelperClient.convert(providedStorageLocationProto);
+  }
+
+  @Nonnull
+  public static Block fromBlockBytes(@Nonnull byte[] blockDbFormat)
+      throws InvalidProtocolBufferException {
+    BlockProto blockProto = BlockProto.parseFrom(blockDbFormat);
+    return PBHelperClient.convert(blockProto);
+  }
+
+  public static byte[] toProtoBufBytes(@Nonnull ProvidedStorageLocation
+      providedStorageLocation) throws IOException {
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        PBHelperClient.convert(providedStorageLocation);
+    ByteArrayOutputStream providedStorageLocationOutputStream =
+        new ByteArrayOutputStream();
+    providedStorageLocationProto.writeTo(providedStorageLocationOutputStream);
+    return providedStorageLocationOutputStream.toByteArray();
+  }
+
+  public static byte[] toProtoBufBytes(@Nonnull Block block)
+      throws IOException {
+    BlockProto blockProto =
+        PBHelperClient.convert(block);
+    ByteArrayOutputStream blockOutputStream = new ByteArrayOutputStream();
+    blockProto.writeTo(blockOutputStream);
+    return blockOutputStream.toByteArray();
+  }
+
+  private IterationResult withIterator(
+      CheckedFunction<DBIterator, IterationResult> func) throws IOException {
+    try (DBIterator iterator = levelDb.iterator()) {
+      return func.apply(iterator);
+    }
+  }
+
+  /**
+   * CheckedFunction is akin to {@link java.util.function.Function} but
+   * specifies an IOException.
+   * @param <T> Argument type.
+   * @param <R> Return type.
+   */
+  @FunctionalInterface
+  public interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
new file mode 100644
index 0000000..fb6e8b3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.aliasmap;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Protocol used by clients to read/write data about aliases of
+ * provided blocks for an in-memory implementation of the
+ * {@link org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap}.
+ */
+public interface InMemoryAliasMapProtocol {
+
+  /**
+   * The result of a read from the in-memory aliasmap. It contains the
+   * a list of FileRegions that are returned, along with the next block
+   * from which the read operation must continue.
+   */
+  class IterationResult {
+
+    private final List<FileRegion> batch;
+    private final Optional<Block> nextMarker;
+
+    public IterationResult(List<FileRegion> batch, Optional<Block> nextMarker) {
+      this.batch = batch;
+      this.nextMarker = nextMarker;
+    }
+
+    public List<FileRegion> getFileRegions() {
+      return batch;
+    }
+
+    public Optional<Block> getNextBlock() {
+      return nextMarker;
+    }
+  }
+
+  /**
+   * List the next batch of {@link FileRegion}s in the alias map starting from
+   * the given {@code marker}. To retrieve all {@link FileRegion}s stored in the
+   * alias map, multiple calls to this function might be required.
+   * @param marker the next block to get fileregions from.
+   * @return the {@link IterationResult} with a set of
+   * FileRegions and the next marker.
+   * @throws IOException
+   */
+  InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException;
+
+  /**
+   * Gets the {@link ProvidedStorageLocation} associated with the
+   * specified block.
+   * @param block the block to lookup
+   * @return the associated {@link ProvidedStorageLocation}.
+   * @throws IOException
+   */
+  @Nonnull
+  Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException;
+
+  /**
+   * Stores the block and it's associated {@link ProvidedStorageLocation}
+   * in the alias map.
+   * @param block
+   * @param providedStorageLocation
+   * @throws IOException
+   */
+  void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
new file mode 100644
index 0000000..91b1e83
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.aliasmap;
+
+import com.google.protobuf.BlockingService;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolServerSideTranslatorPB;
+import org.apache.hadoop.ipc.RPC;
+import javax.annotation.Nonnull;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Optional;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFunction;
+
+/**
+ * InMemoryLevelDBAliasMapServer is the entry point from the Namenode into
+ * the {@link InMemoryAliasMap}.
+ */
+public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
+    Configurable, Closeable {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(InMemoryLevelDBAliasMapServer.class);
+  private final CheckedFunction<Configuration, InMemoryAliasMap> initFun;
+  private RPC.Server aliasMapServer;
+  private Configuration conf;
+  private InMemoryAliasMap aliasMap;
+
+  public InMemoryLevelDBAliasMapServer(
+      CheckedFunction<Configuration, InMemoryAliasMap> initFun) {
+    this.initFun = initFun;
+
+  }
+
+  public void start() throws IOException {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      throw new UnsupportedOperationException("Unable to start "
+          + "InMemoryLevelDBAliasMapServer as security is enabled");
+    }
+    RPC.setProtocolEngine(getConf(), AliasMapProtocolPB.class,
+        ProtobufRpcEngine.class);
+    AliasMapProtocolServerSideTranslatorPB aliasMapProtocolXlator =
+        new AliasMapProtocolServerSideTranslatorPB(this);
+
+    BlockingService aliasMapProtocolService =
+        AliasMapProtocolService
+            .newReflectiveBlockingService(aliasMapProtocolXlator);
+
+    String rpcAddress =
+        conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+            DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT);
+    String[] split = rpcAddress.split(":");
+    String bindHost = split[0];
+    Integer port = Integer.valueOf(split[1]);
+
+    aliasMapServer = new RPC.Builder(conf)
+        .setProtocol(AliasMapProtocolPB.class)
+        .setInstance(aliasMapProtocolService)
+        .setBindAddress(bindHost)
+        .setPort(port)
+        .setNumHandlers(1)
+        .setVerbose(true)
+        .build();
+
+    LOG.info("Starting InMemoryLevelDBAliasMapServer on ", rpcAddress);
+    aliasMapServer.start();
+  }
+
+  @Override
+  public InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException {
+    return aliasMap.list(marker);
+  }
+
+  @Nonnull
+  @Override
+  public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+    return aliasMap.read(block);
+  }
+
+  @Override
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    aliasMap.write(block, providedStorageLocation);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    try {
+      this.aliasMap = initFun.apply(conf);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void close() {
+    LOG.info("Stopping InMemoryLevelDBAliasMapServer");
+    try {
+      aliasMap.close();
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    }
+    aliasMapServer.stop();
+  }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
new file mode 100644
index 0000000..7b0b789
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocolPB.InMemoryAliasMapProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+
+/**
+ * InMemoryLevelDBAliasMapClient is the client for the InMemoryAliasMapServer.
+ * This is used by the Datanode and fs2img to store and retrieve FileRegions
+ * based on the given Block.
+ */
+public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
+    implements Configurable {
+
+  private Configuration conf;
+  private InMemoryAliasMapProtocolClientSideTranslatorPB aliasMap;
+
+  @Override
+  public void close() {
+    aliasMap.stop();
+  }
+
+  class LevelDbReader extends BlockAliasMap.Reader<FileRegion> {
+
+    @Override
+    public Optional<FileRegion> resolve(Block block) throws IOException {
+      Optional<ProvidedStorageLocation> read = aliasMap.read(block);
+      return read.map(psl -> new FileRegion(block, psl, null));
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    private class LevelDbIterator
+        extends BlockAliasMap<FileRegion>.ImmutableIterator {
+
+      private Iterator<FileRegion> iterator;
+      private Optional<Block> nextMarker;
+
+      LevelDbIterator()  {
+        batch(Optional.empty());
+      }
+
+      private void batch(Optional<Block> newNextMarker) {
+        try {
+          InMemoryAliasMap.IterationResult iterationResult =
+              aliasMap.list(newNextMarker);
+          List<FileRegion> fileRegions = iterationResult.getFileRegions();
+          this.iterator = fileRegions.iterator();
+          this.nextMarker = iterationResult.getNextBlock();
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public boolean hasNext() {
+        return iterator.hasNext() || nextMarker.isPresent();
+      }
+
+      @Override
+      public FileRegion next() {
+        if (iterator.hasNext()) {
+          return iterator.next();
+        } else {
+          if (nextMarker.isPresent()) {
+            batch(nextMarker);
+            return next();
+          } else {
+            throw new NoSuchElementException();
+          }
+        }
+      }
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      return new LevelDbIterator();
+    }
+  }
+
+  class LevelDbWriter extends BlockAliasMap.Writer<FileRegion> {
+    @Override
+    public void store(FileRegion fileRegion) throws IOException {
+      aliasMap.write(fileRegion.getBlock(),
+          fileRegion.getProvidedStorageLocation());
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+
+  InMemoryLevelDBAliasMapClient() {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      throw new UnsupportedOperationException("Unable to start "
+          + "InMemoryLevelDBAliasMapClient as security is enabled");
+    }
+  }
+
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+    return new LevelDbReader();
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    return new LevelDbWriter();
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    this.aliasMap = new InMemoryAliasMapProtocolClientSideTranslatorPB(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void refresh() throws IOException {
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
index bcc9a38..0fbfc15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
@@ -22,6 +22,7 @@ import java.net.URI;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -38,6 +39,16 @@ public class FinalizedProvidedReplica extends ProvidedReplica {
         remoteFS);
   }
 
+  public FinalizedProvidedReplica(FileRegion fileRegion, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
+    super(fileRegion.getBlock().getBlockId(),
+        fileRegion.getProvidedStorageLocation().getPath().toUri(),
+        fileRegion.getProvidedStorageLocation().getOffset(),
+        fileRegion.getBlock().getNumBytes(),
+        fileRegion.getBlock().getGenerationStamp(),
+        volume, conf, remoteFS);
+  }
+
   public FinalizedProvidedReplica(long blockId, Path pathPrefix,
       String pathSuffix, long fileOffset, long blockLen, long genStamp,
       FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
index de68e2d..8748918 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
@@ -315,12 +315,7 @@ public class ReplicaBuilder {
             offset, length, genStamp, volume, conf, remoteFS);
       }
     } else {
-      info = new FinalizedProvidedReplica(fileRegion.getBlock().getBlockId(),
-          fileRegion.getPath().toUri(),
-          fileRegion.getOffset(),
-          fileRegion.getBlock().getNumBytes(),
-          fileRegion.getBlock().getGenerationStamp(),
-          volume, conf, remoteFS);
+      info = new FinalizedProvidedReplica(fileRegion, volume, conf, remoteFS);
     }
     return info;
   }

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
new file mode 100644
index 0000000..08f10bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "AliasMapProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message KeyValueProto {
+  optional BlockProto key = 1;
+  optional ProvidedStorageLocationProto value = 2;
+}
+
+message WriteRequestProto {
+  required KeyValueProto keyValuePair = 1;
+}
+
+message WriteResponseProto {
+}
+
+message ReadRequestProto {
+  required BlockProto key = 1;
+}
+
+message ReadResponseProto {
+  optional ProvidedStorageLocationProto value = 1;
+}
+
+message ListRequestProto {
+  optional BlockProto marker = 1;
+}
+
+message ListResponseProto {
+  repeated KeyValueProto fileRegions = 1;
+  optional BlockProto nextMarker = 2;
+}
+
+service AliasMapProtocolService {
+  rpc write(WriteRequestProto) returns(WriteResponseProto);
+  rpc read(ReadRequestProto) returns(ReadResponseProto);
+  rpc list(ListRequestProto) returns(ListResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 655f9cb..ddc07ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4653,6 +4653,40 @@
   </property>
 
   <property>
+    <name>dfs.provided.aliasmap.inmemory.batch-size</name>
+    <value>500</value>
+    <description>
+      The batch size when iterating over the database backing the aliasmap
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.dnrpc-address</name>
+    <value>0.0.0.0:50200</value>
+    <description>
+      The address where the aliasmap server will be running
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.leveldb.dir</name>
+    <value>/tmp</value>
+    <description>
+      The directory where the leveldb files will be kept
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.enabled</name>
+    <value>false</value>
+    <description>
+      Don't use the aliasmap by default. Some tests will fail
+      because they try to start the namenode twice with the
+      same parameters if you turn it on.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.provided.aliasmap.text.delimiter</name>
     <value>,</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
new file mode 100644
index 0000000..6f1ff3e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.aliasmap;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Optional;
+
+/**
+ * ITestInMemoryAliasMap is an integration test that writes and reads to
+ * an AliasMap. This is an integration test because it can't be run in parallel
+ * like normal unit tests since there is conflict over the port being in use.
+ */
+public class ITestInMemoryAliasMap {
+  private InMemoryAliasMap aliasMap;
+  private File tempDirectory;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    tempDirectory = Files.createTempDirectory("seagull").toFile();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDirectory.getAbsolutePath());
+    aliasMap = InMemoryAliasMap.init(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    aliasMap.close();
+    FileUtils.deleteDirectory(tempDirectory);
+  }
+
+  @Test
+  public void readNotFoundReturnsNothing() throws IOException {
+    Block block = new Block(42, 43, 44);
+
+    Optional<ProvidedStorageLocation> actualProvidedStorageLocationOpt
+        = aliasMap.read(block);
+
+    assertFalse(actualProvidedStorageLocationOpt.isPresent());
+  }
+
+  @Test
+  public void readWrite() throws Exception {
+    Block block = new Block(42, 43, 44);
+
+    Path path = new Path("eagle", "mouse");
+    long offset = 47;
+    long length = 48;
+    int nonceSize = 4;
+    byte[] nonce = new byte[nonceSize];
+    Arrays.fill(nonce, 0, (nonceSize - 1), Byte.parseByte("0011", 2));
+
+    ProvidedStorageLocation expectedProvidedStorageLocation =
+        new ProvidedStorageLocation(path, offset, length, nonce);
+
+    aliasMap.write(block, expectedProvidedStorageLocation);
+
+    Optional<ProvidedStorageLocation> actualProvidedStorageLocationOpt
+        = aliasMap.read(block);
+
+    assertTrue(actualProvidedStorageLocationOpt.isPresent());
+    assertEquals(expectedProvidedStorageLocation,
+        actualProvidedStorageLocationOpt.get());
+
+  }
+
+  @Test
+  public void list() throws IOException {
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    Block block3 = new Block(44, 45, 46);
+
+    Path path = new Path("eagle", "mouse");
+    int nonceSize = 4;
+    byte[] nonce = new byte[nonceSize];
+    Arrays.fill(nonce, 0, (nonceSize - 1), Byte.parseByte("0011", 2));
+    ProvidedStorageLocation expectedProvidedStorageLocation1 =
+        new ProvidedStorageLocation(path, 47, 48, nonce);
+    ProvidedStorageLocation expectedProvidedStorageLocation2 =
+        new ProvidedStorageLocation(path, 48, 49, nonce);
+    ProvidedStorageLocation expectedProvidedStorageLocation3 =
+        new ProvidedStorageLocation(path, 49, 50, nonce);
+
+    aliasMap.write(block1, expectedProvidedStorageLocation1);
+    aliasMap.write(block2, expectedProvidedStorageLocation2);
+    aliasMap.write(block3, expectedProvidedStorageLocation3);
+
+    InMemoryAliasMap.IterationResult list = aliasMap.list(Optional.empty());
+    // we should have 3 results
+    assertEquals(3, list.getFileRegions().size());
+    // no more results expected
+    assertFalse(list.getNextBlock().isPresent());
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1682da11/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
new file mode 100644
index 0000000..f699055
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.aliasmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+
+/**
+ * TestInMemoryAliasMap tests the initialization of an AliasMap. Most of the
+ * rest of the tests are in ITestInMemoryAliasMap since the tests are not
+ * thread safe (there is competition for the port).
+ */
+public class TestInMemoryAliasMap {
+
+  @Test
+  public void testInit() {
+    String nonExistingDirectory = "non-existing-directory";
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        nonExistingDirectory);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() -> InMemoryAliasMap.init(conf)).withMessage(
+            InMemoryAliasMap.createPathErrorMessage(nonExistingDirectory));
+  }
+}
\ No newline at end of file


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


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

Posted by vi...@apache.org.
HDFS-12685. [READ] FsVolumeImpl exception when scanning Provided storage volume


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

Branch: refs/heads/HDFS-9806
Commit: 69e3086e6ba459c64bf4541931fb0626dbfbbec4
Parents: b094886
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 30 10:11:12 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


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

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

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

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

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


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


[25/50] [abbrv] hadoop git commit: HDFS-12605. [READ] TestNameNodeProvidedImplementation#testProvidedDatanodeFailures fails after rebase

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


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

Branch: refs/heads/HDFS-9806
Commit: 9286e6699c96705c68d59abf61a85c4882e797e0
Parents: 1cf3d01
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Oct 18 13:53:11 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9286e669/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 3f937c4..d622b9e 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -481,13 +481,13 @@ public class TestNameNodeProvidedImplementation {
       assertEquals(providedDatanode2.getDatanodeUuid(),
           dnInfos[0].getDatanodeUuid());
 
-      //stop the 2nd provided datanode
-      cluster.stopDataNode(1);
+      // stop the 2nd provided datanode
+      MiniDFSCluster.DataNodeProperties providedDNProperties2 =
+          cluster.stopDataNode(0);
       // make NameNode detect that datanode is down
       BlockManagerTestUtil.noticeDeadDatanode(
           cluster.getNameNode(),
           providedDatanode2.getDatanodeId().getXferAddr());
-
       getAndCheckBlockLocations(client, filename, 0);
 
       //restart the provided datanode


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


[07/50] [abbrv] hadoop git commit: HDFS-12819. Setting/Unsetting EC policy shows warning if the directory is not empty. (Contributed by Lei (Eddy) Xu)

Posted by vi...@apache.org.
HDFS-12819. Setting/Unsetting EC policy shows warning if the directory is not empty. (Contributed by Lei (Eddy) Xu)


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

Branch: refs/heads/HDFS-9806
Commit: 1c15b1751c0698bd3063d5c25f556d4821b161d2
Parents: 6681dd1
Author: Lei Xu <le...@apache.org>
Authored: Fri Dec 15 10:04:43 2017 -0800
Committer: Lei Xu <le...@apache.org>
Committed: Fri Dec 15 10:04:43 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   | 14 ++++++++
 .../test/resources/testErasureCodingConf.xml    | 37 ++++++++++++++++++++
 2 files changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c15b175/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index a28f227..e30b083 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.tools;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -362,6 +364,12 @@ public class ECAdmin extends Configured implements Tool {
           System.out.println("Set erasure coding policy " + ecPolicyName +
               " on " + path);
         }
+        RemoteIterator<FileStatus> dirIt = dfs.listStatusIterator(p);
+        if (dirIt.hasNext()) {
+          System.out.println("Warning: setting erasure coding policy on a " +
+              "non-empty directory will not automatically convert existing" +
+              " files to " + ecPolicyName);
+        }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
         return 3;
@@ -412,6 +420,12 @@ public class ECAdmin extends Configured implements Tool {
       try {
         dfs.unsetErasureCodingPolicy(p);
         System.out.println("Unset erasure coding policy from " + path);
+        RemoteIterator<FileStatus> dirIt = dfs.listStatusIterator(p);
+        if (dirIt.hasNext()) {
+          System.out.println("Warning: unsetting erasure coding policy on a " +
+              "non-empty directory will not automatically convert existing" +
+              " files to replicated data.");
+        }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
         return 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c15b175/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 9988ff3..e667213 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -299,6 +299,24 @@
     </test>
 
     <test>
+      <description>setPolicy : set policy on non-empty directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <command>-fs NAMENODE -touchz /ecdir/file1</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-6-3-1024k -path /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Warning: setting erasure coding policy on an non-empty directory will not automatically convert existing data to RS-6-3-1024</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>unsetPolicy : unset inherited EC policy, has no effect</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
@@ -322,6 +340,25 @@
     </test>
 
     <test>
+      <description>unsetPolicy : unset policy on non-empty directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-6-3-1024k -path /ecdir</ec-admin-command>
+        <command>-fs NAMENODE -touchz /ecdir/file1</command>
+        <ec-admin-command>-fs NAMENODE -unsetPolicy -path /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Warning: unsetting erasure coding policy on an non-empty directory will not automatically convert existing data to replicated data</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /noec</command>


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 6c094a7ec1a8b92c8fa599c52ff80203f390ca2b
Parents: 3abe22e
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Aug 7 14:31:15 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c094a7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
index 722d573..e23d6be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -31,8 +32,9 @@ public class FinalizedProvidedReplica extends ProvidedReplica {
 
   public FinalizedProvidedReplica(long blockId, URI fileURI,
       long fileOffset, long blockLen, long genStamp,
-      FsVolumeSpi volume, Configuration conf) {
-    super(blockId, fileURI, fileOffset, blockLen, genStamp, volume, conf);
+      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
+    super(blockId, fileURI, fileOffset, blockLen, genStamp, volume, conf,
+        remoteFS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c094a7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
index 946ab5a..2b3bd13 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
@@ -65,16 +65,23 @@ public abstract class ProvidedReplica extends ReplicaInfo {
    * @param volume the volume this block belongs to
    */
   public ProvidedReplica(long blockId, URI fileURI, long fileOffset,
-      long blockLen, long genStamp, FsVolumeSpi volume, Configuration conf) {
+      long blockLen, long genStamp, FsVolumeSpi volume, Configuration conf,
+      FileSystem remoteFS) {
     super(volume, blockId, blockLen, genStamp);
     this.fileURI = fileURI;
     this.fileOffset = fileOffset;
     this.conf = conf;
-    try {
-      this.remoteFS = FileSystem.get(fileURI, this.conf);
-    } catch (IOException e) {
-      LOG.warn("Failed to obtain filesystem for " + fileURI);
-      this.remoteFS = null;
+    if (remoteFS != null) {
+      this.remoteFS = remoteFS;
+    } else {
+      LOG.warn(
+          "Creating an reference to the remote FS for provided block " + this);
+      try {
+        this.remoteFS = FileSystem.get(fileURI, this.conf);
+      } catch (IOException e) {
+        LOG.warn("Failed to obtain filesystem for " + fileURI);
+        this.remoteFS = null;
+      }
     }
   }
 
@@ -83,11 +90,7 @@ public abstract class ProvidedReplica extends ReplicaInfo {
     this.fileURI = r.fileURI;
     this.fileOffset = r.fileOffset;
     this.conf = r.conf;
-    try {
-      this.remoteFS = FileSystem.newInstance(fileURI, this.conf);
-    } catch (IOException e) {
-      this.remoteFS = null;
-    }
+    this.remoteFS = r.remoteFS;
   }
 
   @Override

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

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

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


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


[40/50] [abbrv] hadoop git commit: HDFS-12885. Add visibility/stability annotations. Contributed by Chris Douglas

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


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

Branch: refs/heads/HDFS-9806
Commit: 63a2e7cf36100cc0b9fb31c572e40b7511bfa286
Parents: c7bbd01
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Dec 5 09:51:09 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:29 2017 -0800

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


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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
index be891e5..f7be111 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.aliasmap;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -47,6 +49,8 @@ import java.util.Optional;
  * InMemoryAliasMap is an implementation of the InMemoryAliasMapProtocol for
  * use with LevelDB.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     Configurable {
 

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
index 91b1e83..197e663 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
@@ -17,6 +17,8 @@
 package org.apache.hadoop.hdfs.server.aliasmap;
 
 import com.google.protobuf.BlockingService;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
@@ -42,6 +44,8 @@ import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFun
  * InMemoryLevelDBAliasMapServer is the entry point from the Namenode into
  * the {@link InMemoryAliasMap}.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
     Configurable, Closeable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index c85eb2c..f6b0c0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -30,6 +30,8 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentSkipListMap;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -57,6 +59,8 @@ import com.google.protobuf.ByteString;
  * This class allows us to manage and multiplex between storages local to
  * datanodes, and provided storage.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class ProvidedStorageMap {
 
   private static final Logger LOG =

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
index 66971a3..6a3d55c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
@@ -24,6 +24,8 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBIterator;
 import static org.fusesource.leveldbjni.JniDBFactory.factory;
@@ -45,6 +47,8 @@ import org.slf4j.LoggerFactory;
 /**
  * A LevelDB based implementation of {@link BlockAliasMap}.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class LevelDBFileRegionAliasMap
       extends BlockAliasMap<FileRegion> implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
index b86b280..69d9cc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
@@ -34,6 +34,8 @@ import java.util.IdentityHashMap;
 import java.util.NoSuchElementException;
 import java.util.Optional;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -56,6 +58,8 @@ import com.google.common.annotations.VisibleForTesting;
  * This class is used for block maps stored as text files,
  * with a specified delimiter.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class TextFileRegionAliasMap
     extends BlockAliasMap<FileRegion> implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
index 0fbfc15..43fd234 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.net.URI;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -30,6 +32,8 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 /**
  * This class is used for provided replicas that are finalized.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class FinalizedProvidedReplica extends ProvidedReplica {
 
   public FinalizedProvidedReplica(long blockId, URI fileURI,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
index 8681421..3d20b00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
@@ -25,6 +25,8 @@ import java.net.URI;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.input.BoundedInputStream;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,6 +44,8 @@ import org.slf4j.LoggerFactory;
 /**
  * This abstract class is used as a base class for provided replicas.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public abstract class ProvidedReplica extends ReplicaInfo {
 
   public static final Logger LOG =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index 6bbfa91..445b455 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -65,7 +66,8 @@ import org.apache.hadoop.util.Time;
 /**
  * This class is used to create provided volumes.
  */
-public class ProvidedVolumeImpl extends FsVolumeImpl {
+@InterfaceAudience.Private
+class ProvidedVolumeImpl extends FsVolumeImpl {
 
   /**
    * Get a suffix of the full path, excluding the given prefix.

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
index f736112..7d66c64 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,6 +33,8 @@ import org.apache.hadoop.fs.Path;
 /**
  * Traversal of an external FileSystem.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FSTreeWalk extends TreeWalk {
 
   private final Path root;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
index b66c830..4598e9c 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
@@ -26,6 +26,8 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,6 +39,8 @@ import org.apache.hadoop.util.ToolRunner;
 /**
  * Create FSImage from an external namespace.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class FileSystemImage implements Tool {
 
   private Configuration conf;

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
index d327363..8f1382a 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
@@ -21,6 +21,8 @@ import java.io.IOException;
 
 import com.google.protobuf.ByteString;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -37,6 +39,8 @@ import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.D
  * Traversal cursor in external filesystem.
  * TODO: generalize, move FS/FileRegion to FSTreePath
  */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
 public class TreePath {
   private long id = -1;
   private final long parentId;

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63a2e7cf/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
index d002e4a..27152fd 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -32,6 +34,8 @@ import org.apache.hadoop.fs.Path;
 /**
  * Random, repeatable hierarchy generator.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class RandomTreeWalk extends TreeWalk {
 
   private final Path root;


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


[05/50] [abbrv] hadoop git commit: YARN-7190. Ensure only NM classpath in 2.x gets TSv2 related hbase jars, not the user classpath. Contributed by Varun Saxena.

Posted by vi...@apache.org.
YARN-7190. Ensure only NM classpath in 2.x gets TSv2 related hbase jars, not the user classpath. Contributed by Varun Saxena.


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

Branch: refs/heads/HDFS-9806
Commit: 09d996fdd429a85822a06ab87f0e2322d0d7ca68
Parents: 890d3d0
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Dec 15 21:50:28 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Dec 15 21:50:28 2017 +0530

----------------------------------------------------------------------
 .../resources/assemblies/hadoop-yarn-dist.xml   | 27 ++++++++++++++
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |  6 ++++
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |  6 ++++
 .../hadoop-yarn-server-resourcemanager/pom.xml  |  1 +
 .../pom.xml                                     | 23 ++++++++++++
 .../hadoop-yarn-server-timelineservice/pom.xml  |  1 +
 hadoop-yarn-project/pom.xml                     | 38 +++++++++++++++++---
 7 files changed, 97 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
index 83633ac..a77dd20 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
@@ -211,10 +211,37 @@
         <include>**/*</include>
       </includes>
     </fileSet>
+    <!-- Copy dependecies from hadoop-yarn-server-timelineservice as well -->
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/target/lib</directory>
+      <outputDirectory>share/hadoop/${hadoop.component}/timelineservice/lib</outputDirectory>
+    </fileSet>
   </fileSets>
   <moduleSets>
     <moduleSet>
+      <includes>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice</include>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase</include>
+      </includes>
+      <binaries>
+        <outputDirectory>share/hadoop/${hadoop.component}/timelineservice</outputDirectory>
+        <includeDependencies>false</includeDependencies>
+        <unpack>false</unpack>
+      </binaries>
+    </moduleSet>
+    <moduleSet>
+      <includes>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-tests</include>
+      </includes>
+      <binaries>
+        <outputDirectory>share/hadoop/${hadoop.component}/timelineservice/test</outputDirectory>
+        <includeDependencies>false</includeDependencies>
+        <unpack>false</unpack>
+      </binaries>
+    </moduleSet>
+    <moduleSet>
       <excludes>
+        <exclude>org.apache.hadoop:hadoop-yarn-server-timelineservice*</exclude>
         <exclude>org.apache.hadoop:hadoop-yarn-ui</exclude>
       </excludes>
       <binaries>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index 9a1dc19..69afe6f 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -121,6 +121,8 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
     ;;
     nodemanager)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/*"
+      hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.nodemanager.NodeManager'
       # Backwards compatibility
       if [[ -n "${YARN_NODEMANAGER_HEAPSIZE}" ]]; then
@@ -145,6 +147,8 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
     ;;
     resourcemanager)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/*"
+      hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.resourcemanager.ResourceManager'
       # Backwards compatibility
       if [[ -n "${YARN_RESOURCEMANAGER_HEAPSIZE}" ]]; then
@@ -178,6 +182,8 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
     ;;
     timelinereader)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/*"
+      hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'
     ;;
     timelineserver)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
index fed3d90..e1ac112 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
@@ -219,6 +219,8 @@ goto :eof
 
 :resourcemanager
   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\rm-config\log4j.properties
+  set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\*
+  set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\lib\*
   set CLASS=org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
   set YARN_OPTS=%YARN_OPTS% %YARN_RESOURCEMANAGER_OPTS%
   if defined YARN_RESOURCEMANAGER_HEAPSIZE (
@@ -248,6 +250,8 @@ goto :eof
 
 :timelinereader
   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\timelineserver-config\log4j.properties
+  set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\*
+  set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\lib\*
   set CLASS=org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer
   set YARN_OPTS=%YARN_OPTS% %YARN_TIMELINEREADER_OPTS%
   goto :eof
@@ -260,6 +264,8 @@ goto :eof
 
 :nodemanager
   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\nm-config\log4j.properties
+  set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\*
+  set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\lib\*
   set CLASS=org.apache.hadoop.yarn.server.nodemanager.NodeManager
   set YARN_OPTS=%YARN_OPTS% -server %HADOOP_NODEMANAGER_OPTS%
   if defined YARN_NODEMANAGER_HEAPSIZE (

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 25a201c..a4d7afc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -191,6 +191,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
index f200e9a..3739301 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
@@ -57,11 +57,13 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>
+      <scope>provided</scope>
     </dependency>
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
     </dependency>
 
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
@@ -75,16 +77,19 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-api</artifactId>
+      <scope>provided</scope>
     </dependency>
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-common</artifactId>
+      <scope>provided</scope>
     </dependency>
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
+      <scope>provided</scope>
     </dependency>
 
     <dependency>
@@ -121,6 +126,7 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
+      <scope>provided</scope>
       <exclusions>
         <exclusion>
           <groupId>org.apache.hadoop</groupId>
@@ -186,6 +192,23 @@
           </additionnalDependencies>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeScope>runtime</includeScope>
+              <excludeGroupIds>org.slf4j,org.apache.hadoop,com.github.stephenc.findbugs</excludeGroupIds>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
index 5cbfbf5..4236c51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
@@ -38,6 +38,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09d996fd/hadoop-yarn-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml
index 1b3c5f0..3ef9c45 100644
--- a/hadoop-yarn-project/pom.xml
+++ b/hadoop-yarn-project/pom.xml
@@ -76,10 +76,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-timelineservice-hbase</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-router</artifactId>
     </dependency>
     <dependency>
@@ -167,12 +163,44 @@
               </execution>
             </executions>
           </plugin>
+          <!-- Remove duplicate dependencies which exist in both yarn/lib folder and yarn/timelineservice/lib folder -->
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                   <echo file="${project.build.directory}/remove-duplicate-timelineservice-jars.sh">
+                      yarnjardir="${project.build.directory}/${project.artifactId}-${project.version}/share/hadoop/yarn"
+                      yarnlibdir="${yarnjardir}/lib/"
+                      cd "${yarnjardir}/timelineservice/lib"
+                      for jar in `ls`
+                      do
+                        if [ -f "${yarnlibdir}""${jar}" ];then
+                          rm -rf ${jar}
+                          echo "Removing duplicate jar $jar from share/hadoop/yarn/timelineservice/lib folder"
+                        fi
+                      done
+                    </echo>
+                    <exec executable="${shell-executable}" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./remove-duplicate-timelineservice-jars.sh"/>
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
         </plugins>
       </build>
     </profile>
   </profiles>
 
-
   <reporting>
     <plugins>
       <plugin>


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: e075a61f10feea387071ffeeadf9d4666935b2b0
Parents: 298fda2
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Dec 8 14:52:48 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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

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

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

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


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


[30/50] [abbrv] hadoop git commit: HDFS-12776. [READ] Increasing replication for PROVIDED files should create local replicas

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


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

Branch: refs/heads/HDFS-9806
Commit: def915fd79611789ff32ab8c193a319053139f7e
Parents: 94a7b68
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 9 13:03:41 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


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

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

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


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 3b6106949ec5c370a0450ff910b43d04153a795c
Parents: c1bf265
Author: Chris Douglas <cd...@apache.org>
Authored: Thu Dec 7 17:41:00 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:25:35 2017 -0800

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


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

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


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 834f28a5d45ce8c9b03ab1363193893bbae96e6f
Parents: def915f
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 9 14:09:14 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/834f28a5/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
index 2e57c9f..b66c830 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
@@ -68,6 +68,7 @@ public class FileSystemImage implements Tool {
     options.addOption("b", "blockclass", true, "Block output class");
     options.addOption("i", "blockidclass", true, "Block resolver class");
     options.addOption("c", "cachedirs", true, "Max active dirents");
+    options.addOption("cid", "clusterID", true, "Cluster ID");
     options.addOption("h", "help", false, "Print usage");
     return options;
   }
@@ -112,6 +113,9 @@ public class FileSystemImage implements Tool {
       case "c":
         opts.cache(Integer.parseInt(o.getValue()));
         break;
+      case "cid":
+        opts.clusterID(o.getValue());
+        break;
       default:
         throw new UnsupportedOperationException("Internal error");
       }

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

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


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 1cf3d01e68bb65e352f4f5db671743ffcb58acf7
Parents: fa33176
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Oct 3 14:44:17 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1cf3d01e/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
index c82c489..d002e4a 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
@@ -113,22 +113,18 @@ public class RandomTreeWalk extends TreeWalk {
     final long len = isDir ? 0 : r.nextInt(Integer.MAX_VALUE);
     final int nblocks = 0 == len ? 0 : (((int)((len - 1) / blocksize)) + 1);
     BlockLocation[] blocks = genBlocks(r, nblocks, blocksize, len);
-    try {
-      return new LocatedFileStatus(new FileStatus(
-          len,              /* long length,             */
-          isDir,            /* boolean isdir,           */
-          1,                /* int block_replication,   */
-          blocksize,        /* long blocksize,          */
-          0L,               /* long modification_time,  */
-          0L,               /* long access_time,        */
-          null,             /* FsPermission permission, */
-          "hadoop",         /* String owner,            */
-          "hadoop",         /* String group,            */
-          name),            /* Path path                */
-          blocks);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    return new LocatedFileStatus(new FileStatus(
+        len,              /* long length,             */
+        isDir,            /* boolean isdir,           */
+        1,                /* int block_replication,   */
+        blocksize,        /* long blocksize,          */
+        0L,               /* long modification_time,  */
+        0L,               /* long access_time,        */
+        null,             /* FsPermission permission, */
+        "hadoop",         /* String owner,            */
+        "hadoop",         /* String group,            */
+        name),            /* Path path                */
+        blocks);
   }
 
   BlockLocation[] genBlocks(Random r, int nblocks, int blocksize, long len) {


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


[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.

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


[02/50] [abbrv] hadoop git commit: YARN-7642. Add test case to verify context update after container promotion or demotion with or without auto update. Contributed by Weiwei Yang.

Posted by vi...@apache.org.
YARN-7642. Add test case to verify context update after container promotion or demotion with or without auto update. Contributed by Weiwei Yang.


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

Branch: refs/heads/HDFS-9806
Commit: 89b6c482c1720a7f7ac86ce022c403825a086fa0
Parents: e1cb278
Author: Weiwei Yang <ww...@apache.org>
Authored: Fri Dec 15 11:58:52 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Fri Dec 15 11:58:52 2017 +0800

----------------------------------------------------------------------
 .../yarn/client/api/impl/TestAMRMClient.java    | 157 ++++++++++++++++++-
 1 file changed, 151 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b6c482/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 742a43a..3ecc5cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -33,6 +33,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.concurrent.TimeoutException;
 import java.util.function.Supplier;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -51,6 +52,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -68,6 +70,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -96,6 +99,7 @@ import org.eclipse.jetty.util.log.Log;
 @RunWith(value = Parameterized.class)
 public class TestAMRMClient {
   private String schedulerName = null;
+  private boolean autoUpdate = false;
   private Configuration conf = null;
   private MiniYARNCluster yarnCluster = null;
   private YarnClient yarnClient = null;
@@ -115,16 +119,19 @@ public class TestAMRMClient {
   private String[] racks;
   private final static int DEFAULT_ITERATION = 3;
 
-  public TestAMRMClient(String schedulerName) {
+  public TestAMRMClient(String schedulerName, boolean autoUpdate) {
     this.schedulerName = schedulerName;
+    this.autoUpdate = autoUpdate;
   }
 
   @Parameterized.Parameters
   public static Collection<Object[]> data() {
-    List<Object[]> list = new ArrayList<Object[]>(2);
-    list.add(new Object[] {CapacityScheduler.class.getName()});
-    list.add(new Object[] {FairScheduler.class.getName()});
-    return list;
+    // Currently only capacity scheduler supports auto update.
+    return Arrays.asList(new Object[][] {
+        {CapacityScheduler.class.getName(), true},
+        {CapacityScheduler.class.getName(), false},
+        {FairScheduler.class.getName(), false}
+    });
   }
 
   @Before
@@ -137,6 +144,9 @@ public class TestAMRMClient {
       throws Exception {
     // start minicluster
     this.conf = conf;
+    if (autoUpdate) {
+      conf.setBoolean(YarnConfiguration.RM_AUTO_UPDATE_CONTAINERS, true);
+    }
     conf.set(YarnConfiguration.RM_SCHEDULER, schedulerName);
     conf.setLong(
       YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
@@ -1157,6 +1167,139 @@ public class TestAMRMClient {
     assertEquals(1, updatedContainers.size());
   }
 
+  @Test
+  public void testAMRMContainerPromotionAndDemotionWithAutoUpdate()
+      throws Exception {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    amClient.init(conf);
+    amClient.start();
+
+    // start am nm client
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+
+    // setup container request
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    Resource reqResource = Resource.newInstance(512, 1);
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(reqResource, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.OPPORTUNISTIC, true)));
+
+    // RM should allocate container within 1 calls to allocate()
+    AllocateResponse allocResponse = waitForAllocation(amClient, 1, 0);
+
+    assertEquals(1, allocResponse.getAllocatedContainers().size());
+    startContainer(allocResponse, nmClient);
+
+    Container c = allocResponse.getAllocatedContainers().get(0);
+    amClient.requestContainerUpdate(c,
+        UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED));
+
+    allocResponse = waitForAllocation(amClient, 0, 1);
+
+    // Make sure container is updated.
+    UpdatedContainer updatedContainer = allocResponse
+        .getUpdatedContainers().get(0);
+
+    // If container auto update is not enabled, we need to notify
+    // NM about this update.
+    if (!autoUpdate) {
+      nmClient.updateContainerResource(updatedContainer.getContainer());
+    }
+
+    // Wait until NM context updated, or fail on timeout.
+    waitForNMContextUpdate(updatedContainer, ExecutionType.GUARANTEED);
+
+    // Once promoted, demote it back to OPPORTUNISTIC
+    amClient.requestContainerUpdate(updatedContainer.getContainer(),
+        UpdateContainerRequest.newInstance(
+            updatedContainer.getContainer().getVersion(),
+            updatedContainer.getContainer().getId(),
+            ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+            null, ExecutionType.OPPORTUNISTIC));
+
+    allocResponse = waitForAllocation(amClient, 0, 1);
+
+    // Make sure container is updated.
+    updatedContainer = allocResponse.getUpdatedContainers().get(0);
+
+    if (!autoUpdate) {
+      nmClient.updateContainerResource(updatedContainer.getContainer());
+    }
+
+    // Wait until NM context updated, or fail on timeout.
+    waitForNMContextUpdate(updatedContainer, ExecutionType.OPPORTUNISTIC);
+
+    amClient.close();
+  }
+
+  private AllocateResponse waitForAllocation(AMRMClient amrmClient,
+      int expectedAllocatedContainerNum, int expectedUpdatedContainerNum)
+      throws Exception {
+    AllocateResponse allocResponse = null;
+    int iteration = 100;
+    while(iteration>0) {
+      allocResponse = amrmClient.allocate(0.1f);
+      int actualAllocated = allocResponse.getAllocatedContainers().size();
+      int actualUpdated = allocResponse.getUpdatedContainers().size();
+      if (expectedAllocatedContainerNum == actualAllocated &&
+          expectedUpdatedContainerNum == actualUpdated) {
+        break;
+      }
+      Thread.sleep(100);
+      iteration--;
+    }
+    return allocResponse;
+  }
+
+  private void waitForNMContextUpdate(UpdatedContainer updatedContainer,
+      ExecutionType expectedType) {
+    for (int i=0; i<nodeCount; i++) {
+      NodeManager nm = yarnCluster.getNodeManager(i);
+      if (nm.getNMContext().getNodeId()
+          .equals(updatedContainer.getContainer().getNodeId())) {
+        try {
+          GenericTestUtils.waitFor(() -> {
+            org.apache.hadoop.yarn.server.nodemanager.containermanager
+                .container.Container nmContainer =
+                nm.getNMContext().getContainers()
+                    .get(updatedContainer.getContainer().getId());
+            if (nmContainer != null) {
+              ExecutionType actual = nmContainer.getContainerTokenIdentifier()
+                  .getExecutionType();
+              return actual.equals(expectedType);
+            }
+            return false;
+          }, 1000, 30000);
+        } catch (TimeoutException e) {
+          fail("Times out waiting for container state in"
+              + " NM context to be updated");
+        } catch (InterruptedException e) {
+          // Ignorable.
+        }
+        break;
+      }
+
+      // Iterated all nodes but still can't get a match
+      if (i == nodeCount -1) {
+        fail("Container doesn't exist in NM context.");
+      }
+    }
+  }
+
   @Test(timeout=60000)
   public void testAMRMClientWithContainerPromotion()
       throws YarnException, IOException {
@@ -1446,7 +1589,9 @@ public class TestAMRMClient {
     for (UpdatedContainer updatedContainer : allocResponse
         .getUpdatedContainers()) {
       Container container = updatedContainer.getContainer();
-      nmClient.increaseContainerResource(container);
+      if (!autoUpdate) {
+        nmClient.increaseContainerResource(container);
+      }
       // NodeManager may still need some time to get the stable
       // container status
       while (true) {


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: f8b81c07bdf8362e50f3fad34b5e21cf442bd13c
Parents: f2b1c1a
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 16 03:52:12 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   1 -
 .../server/blockmanagement/BlockManager.java    |  19 ++-
 .../blockmanagement/DatanodeDescriptor.java     |  24 ++--
 .../blockmanagement/DatanodeStatistics.java     |   3 +
 .../server/blockmanagement/DatanodeStats.java   |   4 +-
 .../blockmanagement/HeartbeatManager.java       |   9 +-
 .../blockmanagement/ProvidedStorageMap.java     |  60 +++++++--
 .../blockmanagement/StorageTypeStats.java       |  33 ++++-
 .../fsdataset/impl/DefaultProvidedVolumeDF.java |  58 ---------
 .../fsdataset/impl/ProvidedVolumeDF.java        |  34 -----
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 101 ++++++++++++---
 .../federation/metrics/FederationMBean.java     |   6 +
 .../federation/metrics/FederationMetrics.java   |   5 +
 .../federation/metrics/NamenodeBeanMetrics.java |  10 ++
 .../resolver/MembershipNamenodeResolver.java    |   1 +
 .../resolver/NamenodeStatusReport.java          |  12 +-
 .../router/NamenodeHeartbeatService.java        |   3 +-
 .../store/records/MembershipStats.java          |   4 +
 .../records/impl/pb/MembershipStatsPBImpl.java  |  10 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  12 ++
 .../hdfs/server/namenode/NameNodeMXBean.java    |  10 +-
 .../namenode/metrics/FSNamesystemMBean.java     |   7 +-
 .../src/main/proto/FederationProtocol.proto     |   1 +
 .../src/main/resources/hdfs-default.xml         |   8 --
 .../src/main/webapps/hdfs/dfshealth.html        |   1 +
 .../blockmanagement/TestProvidedStorageMap.java |  39 +++---
 .../fsdataset/impl/TestProvidedImpl.java        |  55 ++------
 .../metrics/TestFederationMetrics.java          |   2 +
 .../TestNameNodeProvidedImplementation.java     | 125 ++++++++++++++++---
 29 files changed, 425 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b81c07/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 07502c1..f92c4e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -103,6 +103,8 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.hdfs.util.FoldedTreeSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.namenode.CacheManager;
@@ -2402,6 +2404,21 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  public long getProvidedCapacity() {
+    return providedStorageMap.getCapacity();
+  }
+
+  public void updateHeartbeat(DatanodeDescriptor node, StorageReport[] reports,
+      long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes,
+      VolumeFailureSummary volumeFailureSummary) {
+
+    for (StorageReport report: reports) {
+      providedStorageMap.updateStorage(node, report.getStorage());
+    }
+    node.updateHeartbeat(reports, cacheCapacity, cacheUsed, xceiverCount,
+        failedVolumes, volumeFailureSummary);
+  }
+
   /**
    * StatefulBlockInfo is used to build the "toUC" list, which is a list of
    * updates to the information about under-construction blocks.
@@ -2463,7 +2480,7 @@ public class BlockManager implements BlockStatsMXBean {
       // !#! Register DN with provided storage, not with storage owned by DN
       // !#! DN should still have a ref to the DNStorageInfo
       DatanodeStorageInfo storageInfo =
-          providedStorageMap.getStorage(node, storage, context);
+          providedStorageMap.getStorage(node, storage);
 
       if (storageInfo == null) {
         // We handle this for backwards compatibility.

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b81c07/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
deleted file mode 100644
index 24921c4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * The default usage statistics for a provided volume.
- */
-public class DefaultProvidedVolumeDF
-    implements ProvidedVolumeDF, Configurable {
-
-  @Override
-  public void setConf(Configuration conf) {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return null;
-  }
-
-  @Override
-  public long getCapacity() {
-    return Long.MAX_VALUE;
-  }
-
-  @Override
-  public long getSpaceUsed() {
-    return 0;
-  }
-
-  @Override
-  public long getBlockPoolUsed(String bpid) {
-    return 0;
-  }
-
-  @Override
-  public long getAvailable() {
-    return Long.MAX_VALUE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b81c07/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
deleted file mode 100644
index 4d28883..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
-
-/**
- * This interface is used to define the usage statistics
- * of the provided storage.
- */
-public interface ProvidedVolumeDF {
-
-  long getCapacity();
-
-  long getSpaceUsed();
-
-  long getBlockPoolUsed(String bpid);
-
-  long getAvailable();
-}

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b81c07/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 835d8c4..655f9cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4630,14 +4630,6 @@
   </property>
 
   <property>
-    <name>dfs.provided.df.class</name>
-    <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
-    <description>
-      The class that is used to measure usage statistics of provided stores.
-    </description>
-  </property>
-
-  <property>
     <name>dfs.provided.storage.id</name>
     <value>DS-PROVIDED</value>
     <description>

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8b81c07/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 22f00aa..f6d38f6 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -27,6 +27,7 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
+import java.util.Iterator;
 import java.util.Random;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.conf.Configuration;
@@ -44,13 +45,23 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.net.NodeBase;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -59,6 +70,7 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
 import static org.junit.Assert.*;
 
 public class TestNameNodeProvidedImplementation {
@@ -79,6 +91,7 @@ public class TestNameNodeProvidedImplementation {
   private final String filePrefix = "file";
   private final String fileSuffix = ".dat";
   private final int baseFileLen = 1024;
+  private long providedDataSize = 0;
 
   Configuration conf;
   MiniDFSCluster cluster;
@@ -135,6 +148,7 @@ public class TestNameNodeProvidedImplementation {
           }
           writer.flush();
           writer.close();
+          providedDataSize += newFile.length();
         } catch (IOException e) {
           e.printStackTrace();
         }
@@ -206,13 +220,14 @@ public class TestNameNodeProvidedImplementation {
     cluster.waitActive();
   }
 
-  @Test(timeout = 20000)
+  @Test(timeout=20000)
   public void testLoadImage() throws Exception {
     final long seed = r.nextLong();
     LOG.info("NAMEPATH: " + NAMEPATH);
     createImage(new RandomTreeWalk(seed), NNDIRPATH, FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 0, new StorageType[] {StorageType.PROVIDED},
-        null, false);
+    startCluster(NNDIRPATH, 0,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
 
     FileSystem fs = cluster.getFileSystem();
     for (TreePath e : new RandomTreeWalk(seed)) {
@@ -231,14 +246,83 @@ public class TestNameNodeProvidedImplementation {
     }
   }
 
-  @Test(timeout=20000)
-  public void testBlockLoad() throws Exception {
+  @Test(timeout=30000)
+  public void testProvidedReporting() throws Exception {
     conf.setClass(ImageWriter.Options.UGI_CLASS,
         SingleUGIResolver.class, UGIResolver.class);
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 1, new StorageType[] {StorageType.PROVIDED},
-        null, false);
+    int numDatanodes = 10;
+    startCluster(NNDIRPATH, numDatanodes,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
+    long diskCapacity = 1000;
+    // set the DISK capacity for testing
+    for (DataNode dn: cluster.getDataNodes()) {
+      for (FsVolumeSpi ref : dn.getFSDataset().getFsVolumeReferences()) {
+        if (ref.getStorageType() == StorageType.DISK) {
+          ((FsVolumeImpl) ref).setCapacityForTesting(diskCapacity);
+        }
+      }
+    }
+    // trigger heartbeats to update the capacities
+    cluster.triggerHeartbeats();
+    Thread.sleep(10000);
+    // verify namenode stats
+    FSNamesystem namesystem = cluster.getNameNode().getNamesystem();
+    DatanodeStatistics dnStats = namesystem.getBlockManager()
+        .getDatanodeManager().getDatanodeStatistics();
+
+    // total capacity reported includes only the local volumes and
+    // not the provided capacity
+    assertEquals(diskCapacity * numDatanodes, namesystem.getTotal());
+
+    // total storage used should be equal to the totalProvidedStorage
+    // no capacity should be remaining!
+    assertEquals(providedDataSize, dnStats.getProvidedCapacity());
+    assertEquals(providedDataSize, namesystem.getProvidedCapacityTotal());
+    assertEquals(providedDataSize, dnStats.getStorageTypeStats()
+        .get(StorageType.PROVIDED).getCapacityTotal());
+    assertEquals(providedDataSize, dnStats.getStorageTypeStats()
+        .get(StorageType.PROVIDED).getCapacityUsed());
+
+    // verify datanode stats
+    for (DataNode dn: cluster.getDataNodes()) {
+      for (StorageReport report : dn.getFSDataset()
+          .getStorageReports(namesystem.getBlockPoolId())) {
+        if (report.getStorage().getStorageType() == StorageType.PROVIDED) {
+          assertEquals(providedDataSize, report.getCapacity());
+          assertEquals(providedDataSize, report.getDfsUsed());
+          assertEquals(providedDataSize, report.getBlockPoolUsed());
+          assertEquals(0, report.getNonDfsUsed());
+          assertEquals(0, report.getRemaining());
+        }
+      }
+    }
+
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+            cluster.getNameNodePort()), cluster.getConfiguration(0));
+    BlockManager bm = namesystem.getBlockManager();
+    for (int fileId = 0; fileId < numFiles; fileId++) {
+      String filename = "/" + filePrefix + fileId + fileSuffix;
+      LocatedBlocks locatedBlocks = client.getLocatedBlocks(
+          filename, 0, baseFileLen);
+      for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
+        BlockInfo blockInfo =
+            bm.getStoredBlock(locatedBlock.getBlock().getLocalBlock());
+        Iterator<DatanodeStorageInfo> storagesItr = blockInfo.getStorageInfos();
+
+        DatanodeStorageInfo info = storagesItr.next();
+        assertEquals(StorageType.PROVIDED, info.getStorageType());
+        DatanodeDescriptor dnDesc = info.getDatanodeDescriptor();
+        // check the locations that are returned by FSCK have the right name
+        assertEquals(ProvidedStorageMap.ProvidedDescriptor.NETWORK_LOCATION
+            + PATH_SEPARATOR_STR + ProvidedStorageMap.ProvidedDescriptor.NAME,
+            NodeBase.getPath(dnDesc));
+        // no DatanodeStorageInfos should remain
+        assertFalse(storagesItr.hasNext());
+      }
+    }
   }
 
   @Test(timeout=500000)
@@ -250,8 +334,8 @@ public class TestNameNodeProvidedImplementation {
     // make the last Datanode with only DISK
     startCluster(NNDIRPATH, 3, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
     // wait for the replication to finish
@@ -308,8 +392,9 @@ public class TestNameNodeProvidedImplementation {
         FsUGIResolver.class, UGIResolver.class);
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3, new StorageType[] {StorageType.PROVIDED},
-        null, false);
+    startCluster(NNDIRPATH, 3,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
     FileSystem fs = cluster.getFileSystem();
     Thread.sleep(2000);
     int count = 0;
@@ -371,7 +456,7 @@ public class TestNameNodeProvidedImplementation {
     return fs.getFileBlockLocations(path, 0, fileLen);
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testClusterWithEmptyImage() throws IOException {
     // start a cluster with 2 datanodes without any provided storage
     startCluster(NNDIRPATH, 2, null,
@@ -404,7 +489,7 @@ public class TestNameNodeProvidedImplementation {
    * Tests setting replication of provided files.
    * @throws Exception
    */
-  @Test
+  @Test(timeout=30000)
   public void testSetReplicationForProvidedFiles() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
@@ -441,14 +526,14 @@ public class TestNameNodeProvidedImplementation {
     getAndCheckBlockLocations(client, filename, newReplication);
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testProvidedDatanodeFailures() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
             FixedBlockResolver.class);
     startCluster(NNDIRPATH, 3, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
@@ -511,7 +596,7 @@ public class TestNameNodeProvidedImplementation {
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
@@ -540,7 +625,7 @@ public class TestNameNodeProvidedImplementation {
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
@@ -570,7 +655,7 @@ public class TestNameNodeProvidedImplementation {
     }
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testSetClusterID() throws Exception {
     String clusterID = "PROVIDED-CLUSTER";
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
@@ -578,7 +663,7 @@ public class TestNameNodeProvidedImplementation {
     // 2 Datanodes, 1 PROVIDED and other DISK
     startCluster(NNDIRPATH, 2, null,
         new StorageType[][] {
-            {StorageType.PROVIDED},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
     NameNode nn = cluster.getNameNode();


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


[12/50] [abbrv] hadoop git commit: HDFS-10675. Datanode support to read from external stores.

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index adec209..15e71f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
@@ -241,10 +242,11 @@ public interface FsVolumeSpi
 
     private final FsVolumeSpi volume;
 
+    private final FileRegion fileRegion;
     /**
      * Get the file's length in async block scan
      */
-    private final long blockFileLength;
+    private final long blockLength;
 
     private final static Pattern CONDENSED_PATH_REGEX =
         Pattern.compile("(?<!^)(\\\\|/){2,}");
@@ -294,13 +296,30 @@ public interface FsVolumeSpi
      */
     public ScanInfo(long blockId, File blockFile, File metaFile,
         FsVolumeSpi vol) {
+      this(blockId, blockFile, metaFile, vol, null,
+          (blockFile != null) ? blockFile.length() : 0);
+    }
+
+    /**
+     * Create a ScanInfo object for a block. This constructor will examine
+     * the block data and meta-data files.
+     *
+     * @param blockId the block ID
+     * @param blockFile the path to the block data file
+     * @param metaFile the path to the block meta-data file
+     * @param vol the volume that contains the block
+     * @param fileRegion the file region (for provided blocks)
+     * @param length the length of the block data
+     */
+    public ScanInfo(long blockId, File blockFile, File metaFile,
+        FsVolumeSpi vol, FileRegion fileRegion, long length) {
       this.blockId = blockId;
       String condensedVolPath =
           (vol == null || vol.getBaseURI() == null) ? null :
             getCondensedPath(new File(vol.getBaseURI()).getAbsolutePath());
       this.blockSuffix = blockFile == null ? null :
         getSuffix(blockFile, condensedVolPath);
-      this.blockFileLength = (blockFile != null) ? blockFile.length() : 0;
+      this.blockLength = length;
       if (metaFile == null) {
         this.metaSuffix = null;
       } else if (blockFile == null) {
@@ -310,6 +329,7 @@ public interface FsVolumeSpi
             condensedVolPath + blockSuffix);
       }
       this.volume = vol;
+      this.fileRegion = fileRegion;
     }
 
     /**
@@ -328,8 +348,8 @@ public interface FsVolumeSpi
      *
      * @return the length of the data block
      */
-    public long getBlockFileLength() {
-      return blockFileLength;
+    public long getBlockLength() {
+      return blockLength;
     }
 
     /**
@@ -399,6 +419,10 @@ public interface FsVolumeSpi
           getMetaFile().getName()) :
             HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     }
+
+    public FileRegion getFileRegion() {
+      return fileRegion;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
new file mode 100644
index 0000000..24921c4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/DefaultProvidedVolumeDF.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * The default usage statistics for a provided volume.
+ */
+public class DefaultProvidedVolumeDF
+    implements ProvidedVolumeDF, Configurable {
+
+  @Override
+  public void setConf(Configuration conf) {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return null;
+  }
+
+  @Override
+  public long getCapacity() {
+    return Long.MAX_VALUE;
+  }
+
+  @Override
+  public long getSpaceUsed() {
+    return 0;
+  }
+
+  @Override
+  public long getBlockPoolUsed(String bpid) {
+    return 0;
+  }
+
+  @Override
+  public long getAvailable() {
+    return Long.MAX_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 095bc8f..db8d60c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -86,6 +86,7 @@ import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
@@ -1744,6 +1745,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       Set<String> missingVolumesReported = new HashSet<>();
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
+        //skip blocks in PROVIDED storage
+        if (b.getVolume().getStorageType() == StorageType.PROVIDED) {
+          continue;
+        }
         String volStorageID = b.getVolume().getStorageID();
         if (!builders.containsKey(volStorageID)) {
           if (!missingVolumesReported.contains(volStorageID)) {
@@ -1879,7 +1884,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       r = volumeMap.get(bpid, blockId);
     }
-
     if (r != null) {
       if (r.blockDataExists()) {
         return r;
@@ -2232,13 +2236,20 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @param vol Volume of the block file
    */
   @Override
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) throws IOException {
+  public void checkAndUpdate(String bpid, ScanInfo scanInfo)
+      throws IOException {
+
+    long blockId = scanInfo.getBlockId();
+    File diskFile = scanInfo.getBlockFile();
+    File diskMetaFile = scanInfo.getMetaFile();
+    FsVolumeSpi vol = scanInfo.getVolume();
+
     Block corruptBlock = null;
     ReplicaInfo memBlockInfo;
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       memBlockInfo = volumeMap.get(bpid, blockId);
-      if (memBlockInfo != null && memBlockInfo.getState() != ReplicaState.FINALIZED) {
+      if (memBlockInfo != null &&
+          memBlockInfo.getState() != ReplicaState.FINALIZED) {
         // Block is not finalized - ignore the difference
         return;
       }
@@ -2253,6 +2264,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           Block.getGenerationStamp(diskMetaFile.getName()) :
           HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
+      if (vol.getStorageType() == StorageType.PROVIDED) {
+        if (memBlockInfo == null) {
+          //replica exists on provided store but not in memory
+          ReplicaInfo diskBlockInfo =
+              new ReplicaBuilder(ReplicaState.FINALIZED)
+              .setFileRegion(scanInfo.getFileRegion())
+              .setFsVolume(vol)
+              .setConf(conf)
+              .build();
+
+          volumeMap.add(bpid, diskBlockInfo);
+          LOG.warn("Added missing block to memory " + diskBlockInfo);
+        } else {
+          //replica exists in memory but not in the provided store
+          volumeMap.remove(bpid, blockId);
+          LOG.warn("Deleting missing provided block " + memBlockInfo);
+        }
+        return;
+      }
+
       if (!diskFileExists) {
         if (memBlockInfo == null) {
           // Block file does not exist and block does not exist in memory
@@ -3028,7 +3059,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           newReplicaInfo =
               replicaState.getLazyPersistVolume().activateSavedReplica(bpid,
                   replicaInfo, replicaState);
-
           // Update the volumeMap entry.
           volumeMap.add(bpid, newReplicaInfo);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
index 32759c4..9f115a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileDescriptor;
 import java.io.FileInputStream;
@@ -32,10 +34,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DataChecksum;
 
 /** Utility methods. */
 @InterfaceAudience.Private
@@ -44,6 +48,22 @@ public class FsDatasetUtil {
     return f.getName().endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX);
   }
 
+  public static byte[] createNullChecksumByteArray() {
+    DataChecksum csum =
+        DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 512);
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    DataOutputStream dataOut = new DataOutputStream(out);
+    try {
+      BlockMetadataHeader.writeHeader(dataOut, csum);
+      dataOut.close();
+    } catch (IOException e) {
+      FsVolumeImpl.LOG.error(
+          "Exception in creating null checksum stream: " + e);
+      return null;
+    }
+    return out.toByteArray();
+  }
+
   static File getOrigFile(File unlinkTmpFile) {
     final String name = unlinkTmpFile.getName();
     if (!name.endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX)) {
@@ -135,8 +155,9 @@ public class FsDatasetUtil {
    * Compute the checksum for a block file that does not already have
    * its checksum computed, and save it to dstMeta file.
    */
-  public static void computeChecksum(File srcMeta, File dstMeta, File blockFile,
-      int smallBufferSize, Configuration conf) throws IOException {
+  public static void computeChecksum(File srcMeta, File dstMeta,
+      File blockFile, int smallBufferSize, Configuration conf)
+          throws IOException {
     Preconditions.checkNotNull(srcMeta);
     Preconditions.checkNotNull(dstMeta);
     Preconditions.checkNotNull(blockFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 7224e69..319bc0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -154,18 +154,24 @@ public class FsVolumeImpl implements FsVolumeSpi {
     this.reservedForReplicas = new AtomicLong(0L);
     this.storageLocation = sd.getStorageLocation();
     this.currentDir = sd.getCurrentDir();
-    File parent = currentDir.getParentFile();
-    this.usage = new DF(parent, conf);
     this.storageType = storageLocation.getStorageType();
     this.reserved = conf.getLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY
         + "." + StringUtils.toLowerCase(storageType.toString()), conf.getLong(
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT));
     this.configuredCapacity = -1;
+    if (currentDir != null) {
+      File parent = currentDir.getParentFile();
+      this.usage = new DF(parent, conf);
+      cacheExecutor = initializeCacheExecutor(parent);
+      this.metrics = DataNodeVolumeMetrics.create(conf, parent.getPath());
+    } else {
+      this.usage = null;
+      cacheExecutor = null;
+      this.metrics = null;
+    }
     this.conf = conf;
     this.fileIoProvider = fileIoProvider;
-    cacheExecutor = initializeCacheExecutor(parent);
-    this.metrics = DataNodeVolumeMetrics.create(conf, getBaseURI().getPath());
   }
 
   protected ThreadPoolExecutor initializeCacheExecutor(File parent) {
@@ -440,7 +446,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
   /**
    * Unplanned Non-DFS usage, i.e. Extra usage beyond reserved.
    *
-   * @return
+   * @return Disk usage excluding space used by HDFS and excluding space
+   * reserved for blocks open for write.
    * @throws IOException
    */
   public long getNonDfsUsed() throws IOException {
@@ -518,7 +525,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   public String[] getBlockPoolList() {
     return bpSlices.keySet().toArray(new String[bpSlices.keySet().size()]);   
   }
-    
+
   /**
    * Temporary files. They get moved to the finalized block directory when
    * the block is finalized.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
index 427f81b..2da9170 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 
@@ -67,6 +68,11 @@ public class FsVolumeImplBuilder {
   }
 
   FsVolumeImpl build() throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      return new ProvidedVolumeImpl(dataset, storageID, sd,
+          fileIoProvider != null ? fileIoProvider :
+            new FileIoProvider(null, null), conf);
+    }
     return new FsVolumeImpl(
         dataset, storageID, sd,
         fileIoProvider != null ? fileIoProvider :

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
new file mode 100644
index 0000000..4d28883
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeDF.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+/**
+ * This interface is used to define the usage statistics
+ * of the provided storage.
+ */
+public interface ProvidedVolumeDF {
+
+  long getCapacity();
+
+  long getSpaceUsed();
+
+  long getBlockPoolUsed(String bpid);
+
+  long getAvailable();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
new file mode 100644
index 0000000..a48e117
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -0,0 +1,526 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.TextFileRegionProvider;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
+import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
+import org.apache.hadoop.util.Timer;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectReader;
+import org.codehaus.jackson.map.ObjectWriter;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
+
+/**
+ * This class is used to create provided volumes.
+ */
+public class ProvidedVolumeImpl extends FsVolumeImpl {
+
+  static class ProvidedBlockPoolSlice {
+    private FsVolumeImpl providedVolume;
+
+    private FileRegionProvider provider;
+    private Configuration conf;
+    private String bpid;
+    private ReplicaMap bpVolumeMap;
+
+    ProvidedBlockPoolSlice(String bpid, ProvidedVolumeImpl volume,
+        Configuration conf) {
+      this.providedVolume = volume;
+      bpVolumeMap = new ReplicaMap(new AutoCloseableLock());
+      Class<? extends FileRegionProvider> fmt =
+          conf.getClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
+              TextFileRegionProvider.class, FileRegionProvider.class);
+      provider = ReflectionUtils.newInstance(fmt, conf);
+      this.conf = conf;
+      this.bpid = bpid;
+      bpVolumeMap.initBlockPool(bpid);
+      LOG.info("Created provider: " + provider.getClass());
+    }
+
+    FileRegionProvider getFileRegionProvider() {
+      return provider;
+    }
+
+    public void getVolumeMap(ReplicaMap volumeMap,
+        RamDiskReplicaTracker ramDiskReplicaMap) throws IOException {
+      Iterator<FileRegion> iter = provider.iterator();
+      while(iter.hasNext()) {
+        FileRegion region = iter.next();
+        if (region.getBlockPoolId() != null &&
+            region.getBlockPoolId().equals(bpid)) {
+          ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
+              .setBlockId(region.getBlock().getBlockId())
+              .setURI(region.getPath().toUri())
+              .setOffset(region.getOffset())
+              .setLength(region.getBlock().getNumBytes())
+              .setGenerationStamp(region.getBlock().getGenerationStamp())
+              .setFsVolume(providedVolume)
+              .setConf(conf).build();
+
+          ReplicaInfo oldReplica =
+              volumeMap.get(bpid, newReplica.getBlockId());
+          if (oldReplica == null) {
+            volumeMap.add(bpid, newReplica);
+            bpVolumeMap.add(bpid, newReplica);
+          } else {
+            throw new IOException(
+                "A block with id " + newReplica.getBlockId() +
+                " already exists in the volumeMap");
+          }
+        }
+      }
+    }
+
+    public boolean isEmpty() {
+      return bpVolumeMap.replicas(bpid).size() == 0;
+    }
+
+    public void shutdown(BlockListAsLongs blocksListsAsLongs) {
+      //nothing to do!
+    }
+
+    public void compileReport(LinkedList<ScanInfo> report,
+        ReportCompiler reportCompiler)
+            throws IOException, InterruptedException {
+      /* refresh the provider and return the list of blocks found.
+       * the assumption here is that the block ids in the external
+       * block map, after the refresh, are consistent with those
+       * from before the refresh, i.e., for blocks which did not change,
+       * the ids remain the same.
+       */
+      provider.refresh();
+      Iterator<FileRegion> iter = provider.iterator();
+      while(iter.hasNext()) {
+        reportCompiler.throttle();
+        FileRegion region = iter.next();
+        if (region.getBlockPoolId().equals(bpid)) {
+          LOG.info("Adding ScanInfo for blkid " +
+              region.getBlock().getBlockId());
+          report.add(new ScanInfo(region.getBlock().getBlockId(), null, null,
+              providedVolume, region, region.getLength()));
+        }
+      }
+    }
+  }
+
+  private URI baseURI;
+  private final Map<String, ProvidedBlockPoolSlice> bpSlices =
+      new ConcurrentHashMap<String, ProvidedBlockPoolSlice>();
+
+  private ProvidedVolumeDF df;
+
+  ProvidedVolumeImpl(FsDatasetImpl dataset, String storageID,
+      StorageDirectory sd, FileIoProvider fileIoProvider,
+      Configuration conf) throws IOException {
+    super(dataset, storageID, sd, fileIoProvider, conf);
+    assert getStorageLocation().getStorageType() == StorageType.PROVIDED:
+      "Only provided storages must use ProvidedVolume";
+
+    baseURI = getStorageLocation().getUri();
+    Class<? extends ProvidedVolumeDF> dfClass =
+        conf.getClass(DFSConfigKeys.DFS_PROVIDER_DF_CLASS,
+            DefaultProvidedVolumeDF.class, ProvidedVolumeDF.class);
+    df = ReflectionUtils.newInstance(dfClass, conf);
+  }
+
+  @Override
+  public String[] getBlockPoolList() {
+    return bpSlices.keySet().toArray(new String[bpSlices.keySet().size()]);
+  }
+
+  @Override
+  public long getCapacity() {
+    if (configuredCapacity < 0) {
+      return df.getCapacity();
+    }
+    return configuredCapacity;
+  }
+
+  @Override
+  public long getDfsUsed() throws IOException {
+    return df.getSpaceUsed();
+  }
+
+  @Override
+  long getBlockPoolUsed(String bpid) throws IOException {
+    return df.getBlockPoolUsed(bpid);
+  }
+
+  @Override
+  public long getAvailable() throws IOException {
+    return df.getAvailable();
+  }
+
+  @Override
+  long getActualNonDfsUsed() throws IOException {
+    return df.getSpaceUsed();
+  }
+
+  @Override
+  public long getNonDfsUsed() throws IOException {
+    return 0L;
+  }
+
+  @Override
+  public URI getBaseURI() {
+    return baseURI;
+  }
+
+  @Override
+  public File getFinalizedDir(String bpid) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void reserveSpaceForReplica(long bytesToReserve) {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public void releaseReservedSpace(long bytesToRelease) {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writerWithDefaultPrettyPrinter();
+  private static final ObjectReader READER =
+      new ObjectMapper().reader(ProvidedBlockIteratorState.class);
+
+  private static class ProvidedBlockIteratorState {
+    ProvidedBlockIteratorState() {
+      iterStartMs = Time.now();
+      lastSavedMs = iterStartMs;
+      atEnd = false;
+      lastBlockId = -1;
+    }
+
+    // The wall-clock ms since the epoch at which this iterator was last saved.
+    @JsonProperty
+    private long lastSavedMs;
+
+    // The wall-clock ms since the epoch at which this iterator was created.
+    @JsonProperty
+    private long iterStartMs;
+
+    @JsonProperty
+    private boolean atEnd;
+
+    //The id of the last block read when the state of the iterator is saved.
+    //This implementation assumes that provided blocks are returned
+    //in sorted order of the block ids.
+    @JsonProperty
+    private long lastBlockId;
+  }
+
+  private class ProviderBlockIteratorImpl
+      implements FsVolumeSpi.BlockIterator {
+
+    private String bpid;
+    private String name;
+    private FileRegionProvider provider;
+    private Iterator<FileRegion> blockIterator;
+    private ProvidedBlockIteratorState state;
+
+    ProviderBlockIteratorImpl(String bpid, String name,
+        FileRegionProvider provider) {
+      this.bpid = bpid;
+      this.name = name;
+      this.provider = provider;
+      rewind();
+    }
+
+    @Override
+    public void close() throws IOException {
+      //No action needed
+    }
+
+    @Override
+    public ExtendedBlock nextBlock() throws IOException {
+      if (null == blockIterator || !blockIterator.hasNext()) {
+        return null;
+      }
+      FileRegion nextRegion = null;
+      while (null == nextRegion && blockIterator.hasNext()) {
+        FileRegion temp = blockIterator.next();
+        if (temp.getBlock().getBlockId() < state.lastBlockId) {
+          continue;
+        }
+        if (temp.getBlockPoolId().equals(bpid)) {
+          nextRegion = temp;
+        }
+      }
+      if (null == nextRegion) {
+        return null;
+      }
+      state.lastBlockId = nextRegion.getBlock().getBlockId();
+      return new ExtendedBlock(bpid, nextRegion.getBlock());
+    }
+
+    @Override
+    public boolean atEnd() {
+      return blockIterator != null ? !blockIterator.hasNext(): true;
+    }
+
+    @Override
+    public void rewind() {
+      blockIterator = provider.iterator();
+      state = new ProvidedBlockIteratorState();
+    }
+
+    @Override
+    public void save() throws IOException {
+      //We do not persist the state of this iterator anywhere, locally.
+      //We just re-scan provided volumes as necessary.
+      state.lastSavedMs = Time.now();
+    }
+
+    @Override
+    public void setMaxStalenessMs(long maxStalenessMs) {
+      //do not use max staleness
+    }
+
+    @Override
+    public long getIterStartMs() {
+      return state.iterStartMs;
+    }
+
+    @Override
+    public long getLastSavedMs() {
+      return state.lastSavedMs;
+    }
+
+    @Override
+    public String getBlockPoolId() {
+      return bpid;
+    }
+
+    public void load() throws IOException {
+      //on load, we just rewind the iterator for provided volumes.
+      rewind();
+      LOG.trace("load({}, {}): loaded iterator {}: {}", getStorageID(),
+          bpid, name, WRITER.writeValueAsString(state));
+    }
+  }
+
+  @Override
+  public BlockIterator newBlockIterator(String bpid, String name) {
+    return new ProviderBlockIteratorImpl(bpid, name,
+        bpSlices.get(bpid).getFileRegionProvider());
+  }
+
+  @Override
+  public BlockIterator loadBlockIterator(String bpid, String name)
+      throws IOException {
+    ProviderBlockIteratorImpl iter = new ProviderBlockIteratorImpl(bpid, name,
+        bpSlices.get(bpid).getFileRegionProvider());
+    iter.load();
+    return iter;
+  }
+
+  @Override
+  ReplicaInfo addFinalizedBlock(String bpid, Block b,
+      ReplicaInfo replicaInfo, long bytesReserved) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public VolumeCheckResult check(VolumeCheckContext ignored)
+      throws DiskErrorException {
+    return VolumeCheckResult.HEALTHY;
+  }
+
+  @Override
+  void getVolumeMap(ReplicaMap volumeMap,
+      final RamDiskReplicaTracker ramDiskReplicaMap)
+          throws IOException {
+    LOG.info("Creating volumemap for provided volume " + this);
+    for(ProvidedBlockPoolSlice s : bpSlices.values()) {
+      s.getVolumeMap(volumeMap, ramDiskReplicaMap);
+    }
+  }
+
+  private ProvidedBlockPoolSlice getProvidedBlockPoolSlice(String bpid)
+      throws IOException {
+    ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp == null) {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
+    return bp;
+  }
+
+  @Override
+  void getVolumeMap(String bpid, ReplicaMap volumeMap,
+      final RamDiskReplicaTracker ramDiskReplicaMap)
+          throws IOException {
+    getProvidedBlockPoolSlice(bpid).getVolumeMap(volumeMap, ramDiskReplicaMap);
+  }
+
+  @VisibleForTesting
+  FileRegionProvider getFileRegionProvider(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).getFileRegionProvider();
+  }
+
+  @Override
+  public String toString() {
+    return this.baseURI.toString();
+  }
+
+  @Override
+  void addBlockPool(String bpid, Configuration conf) throws IOException {
+    addBlockPool(bpid, conf, null);
+  }
+
+  @Override
+  void addBlockPool(String bpid, Configuration conf, Timer timer)
+      throws IOException {
+    LOG.info("Adding block pool " + bpid +
+        " to volume with id " + getStorageID());
+    ProvidedBlockPoolSlice bp;
+    bp = new ProvidedBlockPoolSlice(bpid, this, conf);
+    bpSlices.put(bpid, bp);
+  }
+
+  void shutdown() {
+    if (cacheExecutor != null) {
+      cacheExecutor.shutdown();
+    }
+    Set<Entry<String, ProvidedBlockPoolSlice>> set = bpSlices.entrySet();
+    for (Entry<String, ProvidedBlockPoolSlice> entry : set) {
+      entry.getValue().shutdown(null);
+    }
+  }
+
+  @Override
+  void shutdownBlockPool(String bpid, BlockListAsLongs blocksListsAsLongs) {
+    ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp != null) {
+      bp.shutdown(blocksListsAsLongs);
+    }
+    bpSlices.remove(bpid);
+  }
+
+  @Override
+  boolean isBPDirEmpty(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).isEmpty();
+  }
+
+  @Override
+  void deleteBPDirectories(String bpid, boolean force) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public LinkedList<ScanInfo> compileReport(String bpid,
+      LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
+          throws InterruptedException, IOException {
+    LOG.info("Compiling report for volume: " + this + " bpid " + bpid);
+    //get the report from the appropriate block pool.
+    if(bpSlices.containsKey(bpid)) {
+      bpSlices.get(bpid).compileReport(report, reportCompiler);
+    }
+    return report;
+  }
+
+  @Override
+  public ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo,
+      long newGS, long estimateBlockLen) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline createRbw(ExtendedBlock b) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline convertTemporaryToRbw(ExtendedBlock b,
+      ReplicaInfo temp) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline createTemporary(ExtendedBlock b)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline updateRURCopyOnTruncate(ReplicaInfo rur,
+      String bpid, long newBlockId, long recoveryId, long newlength)
+          throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInfo moveBlockToTmpLocation(ExtendedBlock block,
+      ReplicaInfo replicaInfo, int smallBufferSize,
+      Configuration conf) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public File[] copyBlockToLazyPersistLocation(String bpId, long blockId,
+      long genStamp, ReplicaInfo replicaInfo, int smallBufferSize,
+      Configuration conf) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 8b89378..c5d14d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -686,7 +686,7 @@ public class Mover {
     }
   }
 
-  static class Cli extends Configured implements Tool {
+  public static class Cli extends Configured implements Tool {
     private static final String USAGE = "Usage: hdfs mover "
         + "[-p <files/dirs> | -f <local file>]"
         + "\n\t-p <files/dirs>\ta space separated list of HDFS files/dirs to migrate."

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
index 872ee74..45e001d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-class FSImageCompression {
+public class FSImageCompression {
 
   /** Codec to use to save or load image, or null if the image is not compressed */
   private CompressionCodec imageCodec;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index 63d1a28..4aae7d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -658,6 +658,10 @@ public class NNStorage extends Storage implements Closeable,
   void readProperties(StorageDirectory sd, StartupOption startupOption)
       throws IOException {
     Properties props = readPropertiesFile(sd.getVersionFile());
+    if (props == null) {
+      throw new IOException(
+          "Properties not found  for storage directory " + sd);
+    }
     if (HdfsServerConstants.RollingUpgradeStartupOption.ROLLBACK
         .matches(startupOption)) {
       int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
@@ -975,7 +979,11 @@ public class NNStorage extends Storage implements Closeable,
       StorageDirectory sd = sdit.next();
       try {
         Properties props = readPropertiesFile(sd.getVersionFile());
-        cid = props.getProperty("clusterID");
+        if (props == null) {
+          cid = null;
+        } else {
+          cid = props.getProperty("clusterID");
+        }
         LOG.info("current cluster id for sd="+sd.getCurrentDir() + 
             ";lv=" + layoutVersion + ";cid=" + cid);
         

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index dedf987..169dfc2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4622,6 +4622,84 @@
   </property>
 
   <property>
+    <name>dfs.provider.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.TextFileRegionProvider</value>
+    <description>
+        The class that is used to load information about blocks stored in
+        provided storages.
+        org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TextFileRegionProvider
+        is used as the default, which expects the blocks to be specified
+        using a delimited text file.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.df.class</name>
+    <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
+    <description>
+        The class that is used to measure usage statistics of provided stores.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.storage.id</name>
+    <value>DS-PROVIDED</value>
+    <description>
+        The storage ID used for provided stores.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.blockformat.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.TextFileRegionFormat</value>
+    <description>
+      The class that is used to specify the input format of the blocks on
+      provided storages. The default is
+      org.apache.hadoop.hdfs.server.common.TextFileRegionFormat which uses
+      file regions to describe blocks. The file regions are specified as a
+      delimited text file. Each file region is a 6-tuple containing the
+      block id, remote file path, offset into file, length of block, the
+      block pool id containing the block, and the generation stamp of the
+      block.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.textprovider.delimiter</name>
+    <value>,</value>
+    <description>
+        The delimiter used when the provided block map is specified as
+        a text file.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.textprovider.read.path</name>
+    <value></value>
+    <description>
+        The path specifying the provided block map as a text file, specified as
+        a URI.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.textprovider.read.codec</name>
+    <value></value>
+    <description>
+        The codec used to de-compress the provided block map.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.textprovider.write.path</name>
+    <value></value>
+    <description>
+        The path to which the provided block map should be written as a text
+        file, specified as a URI.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.lock.suppress.warning.interval</name>
     <value>10s</value>
     <description>Instrumentation reporting long critical sections will suppress

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
index 25eb5b6..8bc8b0d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
@@ -208,7 +208,7 @@ public class TestDFSRollback {
       UpgradeUtilities.createDataNodeVersionFile(
           dataCurrentDirs,
           storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
 
       cluster.startDataNodes(conf, 1, false, StartupOption.ROLLBACK, null);
       assertTrue(cluster.isDataNodeUp());
@@ -256,7 +256,7 @@ public class TestDFSRollback {
           NodeType.DATA_NODE);
       
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       
       startBlockPoolShouldFail(StartupOption.ROLLBACK, 
           cluster.getNamesystem().getBlockPoolId());
@@ -283,7 +283,7 @@ public class TestDFSRollback {
           NodeType.DATA_NODE);
      
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       
       startBlockPoolShouldFail(StartupOption.ROLLBACK, 
           cluster.getNamesystem().getBlockPoolId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
index d202223..0c09eda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
@@ -265,7 +265,7 @@ public class TestDFSStartupVersions {
           conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY), "current");
       log("DataNode version info", DATA_NODE, i, versions[i]);
       UpgradeUtilities.createDataNodeVersionFile(storage,
-          versions[i].storageInfo, bpid, versions[i].blockPoolId);
+          versions[i].storageInfo, bpid, versions[i].blockPoolId, conf);
       try {
         cluster.startDataNodes(conf, 1, false, StartupOption.REGULAR, null);
       } catch (Exception ignore) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
index fe1ede0..0d9f502 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
@@ -290,7 +290,7 @@ public class TestDFSUpgrade {
           UpgradeUtilities.getCurrentFsscTime(cluster), NodeType.DATA_NODE);
       
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       
       startBlockPoolShouldFail(StartupOption.REGULAR, UpgradeUtilities
           .getCurrentBlockPoolID(null));
@@ -308,7 +308,7 @@ public class TestDFSUpgrade {
           NodeType.DATA_NODE);
           
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo, 
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       // Ensure corresponding block pool failed to initialized
       startBlockPoolShouldFail(StartupOption.REGULAR, UpgradeUtilities
           .getCurrentBlockPoolID(null));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
index 9f4df70..621bd51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
@@ -384,8 +384,10 @@ public class UpgradeUtilities {
           new File(datanodeStorage.toString()));
       sd.setStorageUuid(DatanodeStorage.generateUuid());
       Properties properties = Storage.readPropertiesFile(sd.getVersionFile());
-      properties.setProperty("storageID", sd.getStorageUuid());
-      Storage.writeProperties(sd.getVersionFile(), properties);
+      if (properties != null) {
+        properties.setProperty("storageID", sd.getStorageUuid());
+        Storage.writeProperties(sd.getVersionFile(), properties);
+      }
 
       retVal[i] = newDir;
     }
@@ -461,8 +463,9 @@ public class UpgradeUtilities {
    * @param bpid Block pool Id
    */
   public static void createDataNodeVersionFile(File[] parent,
-      StorageInfo version, String bpid) throws IOException {
-    createDataNodeVersionFile(parent, version, bpid, bpid);
+      StorageInfo version, String bpid, Configuration conf)
+          throws IOException {
+    createDataNodeVersionFile(parent, version, bpid, bpid, conf);
   }
   
   /**
@@ -477,7 +480,8 @@ public class UpgradeUtilities {
    * @param bpidToWrite Block pool Id to write into the version file
    */
   public static void createDataNodeVersionFile(File[] parent,
-      StorageInfo version, String bpid, String bpidToWrite) throws IOException {
+      StorageInfo version, String bpid, String bpidToWrite, Configuration conf)
+          throws IOException {
     DataStorage storage = new DataStorage(version);
     storage.setDatanodeUuid("FixedDatanodeUuid");
 
@@ -485,7 +489,7 @@ public class UpgradeUtilities {
     for (int i = 0; i < parent.length; i++) {
       File versionFile = new File(parent[i], "VERSION");
       StorageDirectory sd = new StorageDirectory(parent[i].getParentFile());
-      DataStorage.createStorageID(sd, false);
+      DataStorage.createStorageID(sd, false, conf);
       storage.writeProperties(versionFile, sd);
       versionFiles[i] = versionFile;
       File bpDir = BlockPoolSliceStorage.getBpRoot(bpid, parent[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
new file mode 100644
index 0000000..eaaac22
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat.*;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test for the text based block format for provided block maps.
+ */
+public class TestTextBlockFormat {
+
+  static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
+
+  void check(TextWriter.Options opts, final Path vp,
+      final Class<? extends CompressionCodec> vc) throws IOException {
+    TextFileRegionFormat mFmt = new TextFileRegionFormat() {
+      @Override
+      public TextWriter createWriter(Path file, CompressionCodec codec,
+          String delim, Configuration conf) throws IOException {
+        assertEquals(vp, file);
+        if (null == vc) {
+          assertNull(codec);
+        } else {
+          assertEquals(vc, codec.getClass());
+        }
+        return null; // ignored
+      }
+    };
+    mFmt.getWriter(opts);
+  }
+
+  @Test
+  public void testWriterOptions() throws Exception {
+    TextWriter.Options opts = TextWriter.defaults();
+    assertTrue(opts instanceof WriterOptions);
+    WriterOptions wopts = (WriterOptions) opts;
+    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
+    assertEquals(def, wopts.getFile());
+    assertNull(wopts.getCodec());
+
+    opts.filename(OUTFILE);
+    check(opts, OUTFILE, null);
+
+    opts.filename(OUTFILE);
+    opts.codec("gzip");
+    Path cp = new Path(OUTFILE.getParent(), OUTFILE.getName() + ".gz");
+    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
+
+  }
+
+  @Test
+  public void testCSVReadWrite() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), ",")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, ",") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+        }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+  @Test
+  public void testCSVReadWriteTsv() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), "\t")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, "\t") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+      }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 212f953..c31df4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
@@ -616,7 +617,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     this.datanode = datanode;
     if (storage != null) {
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
-        DataStorage.createStorageID(storage.getStorageDir(i), false);
+        DataStorage.createStorageID(storage.getStorageDir(i), false, conf);
       }
       this.datanodeUuid = storage.getDatanodeUuid();
     } else {
@@ -1352,8 +1353,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) throws IOException {
+  public void checkAndUpdate(String bpid, ScanInfo info) throws IOException {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 13502d9..bfdaad9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.*;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
@@ -94,8 +95,8 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) {
+  public void checkAndUpdate(String bpid, ScanInfo info) {
+    return;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index a30329c..cfae1e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -119,11 +119,12 @@ public class TestFsDatasetImpl {
   
   private final static String BLOCKPOOL = "BP-TEST";
 
-  private static Storage.StorageDirectory createStorageDirectory(File root)
+  private static Storage.StorageDirectory createStorageDirectory(File root,
+      Configuration conf)
       throws SecurityException, IOException {
     Storage.StorageDirectory sd = new Storage.StorageDirectory(
         StorageLocation.parse(root.toURI().toString()));
-    DataStorage.createStorageID(sd, false);
+    DataStorage.createStorageID(sd, false, conf);
     return sd;
   }
 
@@ -137,7 +138,7 @@ public class TestFsDatasetImpl {
       File loc = new File(BASE_DIR + "/data" + i);
       dirStrings.add(new Path(loc.toString()).toUri().toString());
       loc.mkdirs();
-      dirs.add(createStorageDirectory(loc));
+      dirs.add(createStorageDirectory(loc, conf));
       when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
     }
 
@@ -197,7 +198,8 @@ public class TestFsDatasetImpl {
       String pathUri = new Path(path).toUri().toString();
       expectedVolumes.add(new File(pathUri).getAbsolutePath());
       StorageLocation loc = StorageLocation.parse(pathUri);
-      Storage.StorageDirectory sd = createStorageDirectory(new File(path));
+      Storage.StorageDirectory sd = createStorageDirectory(
+          new File(path), conf);
       DataStorage.VolumeBuilder builder =
           new DataStorage.VolumeBuilder(storage, sd);
       when(storage.prepareVolume(eq(datanode), eq(loc),
@@ -315,7 +317,8 @@ public class TestFsDatasetImpl {
     String newVolumePath = BASE_DIR + "/newVolumeToRemoveLater";
     StorageLocation loc = StorageLocation.parse(newVolumePath);
 
-    Storage.StorageDirectory sd = createStorageDirectory(new File(newVolumePath));
+    Storage.StorageDirectory sd = createStorageDirectory(
+        new File(newVolumePath), conf);
     DataStorage.VolumeBuilder builder =
         new DataStorage.VolumeBuilder(storage, sd);
     when(storage.prepareVolume(eq(datanode), eq(loc),
@@ -348,7 +351,7 @@ public class TestFsDatasetImpl {
         any(ReplicaMap.class),
         any(RamDiskReplicaLruTracker.class));
 
-    Storage.StorageDirectory sd = createStorageDirectory(badDir);
+    Storage.StorageDirectory sd = createStorageDirectory(badDir, conf);
     sd.lock();
     DataStorage.VolumeBuilder builder = new DataStorage.VolumeBuilder(storage, sd);
     when(storage.prepareVolume(eq(datanode),
@@ -492,7 +495,7 @@ public class TestFsDatasetImpl {
     String path = BASE_DIR + "/newData0";
     String pathUri = new Path(path).toUri().toString();
     StorageLocation loc = StorageLocation.parse(pathUri);
-    Storage.StorageDirectory sd = createStorageDirectory(new File(path));
+    Storage.StorageDirectory sd = createStorageDirectory(new File(path), conf);
     DataStorage.VolumeBuilder builder =
         new DataStorage.VolumeBuilder(storage, sd);
     when(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/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
new file mode 100644
index 0000000..2c119fe
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -0,0 +1,426 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.FileRegionProvider;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.DNConf;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Basic test cases for provided implementation.
+ */
+public class TestProvidedImpl {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFsDatasetImpl.class);
+  private static final String BASE_DIR =
+      new FileSystemTestHelper().getTestRootDir();
+  private static final int NUM_LOCAL_INIT_VOLUMES = 1;
+  private static final int NUM_PROVIDED_INIT_VOLUMES = 1;
+  private static final String[] BLOCK_POOL_IDS = {"bpid-0", "bpid-1"};
+  private static final int NUM_PROVIDED_BLKS = 10;
+  private static final long BLK_LEN = 128 * 1024;
+  private static final int MIN_BLK_ID = 0;
+  private static final int CHOSEN_BP_ID = 0;
+
+  private static String providedBasePath = BASE_DIR;
+
+  private Configuration conf;
+  private DataNode datanode;
+  private DataStorage storage;
+  private FsDatasetImpl dataset;
+  private static Map<Long, String> blkToPathMap;
+  private static List<FsVolumeImpl> providedVolumes;
+
+  /**
+   * A simple FileRegion iterator for tests.
+   */
+  public static class TestFileRegionIterator implements Iterator<FileRegion> {
+
+    private int numBlocks;
+    private int currentCount;
+    private String basePath;
+
+    public TestFileRegionIterator(String basePath, int minID, int numBlocks) {
+      this.currentCount = minID;
+      this.numBlocks = numBlocks;
+      this.basePath = basePath;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return currentCount < numBlocks;
+    }
+
+    @Override
+    public FileRegion next() {
+      FileRegion region = null;
+      if (hasNext()) {
+        File newFile = new File(basePath, "file" + currentCount);
+        if(!newFile.exists()) {
+          try {
+            LOG.info("Creating file for blkid " + currentCount);
+            blkToPathMap.put((long) currentCount, newFile.getAbsolutePath());
+            LOG.info("Block id " + currentCount + " corresponds to file " +
+                newFile.getAbsolutePath());
+            newFile.createNewFile();
+            Writer writer = new OutputStreamWriter(
+                new FileOutputStream(newFile.getAbsolutePath()), "utf-8");
+            for(int i=0; i< BLK_LEN/(Integer.SIZE/8); i++) {
+              writer.write(currentCount);
+            }
+            writer.flush();
+            writer.close();
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+        }
+        region = new FileRegion(currentCount, new Path(newFile.toString()),
+            0, BLK_LEN, BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+        currentCount++;
+      }
+      return region;
+    }
+
+    @Override
+    public void remove() {
+      //do nothing.
+    }
+
+    public void resetMinBlockId(int minId) {
+      currentCount = minId;
+    }
+
+    public void resetBlockCount(int numBlocks) {
+      this.numBlocks = numBlocks;
+    }
+
+  }
+
+  /**
+   * A simple FileRegion provider for tests.
+   */
+  public static class TestFileRegionProvider
+      extends FileRegionProvider implements Configurable {
+
+    private Configuration conf;
+    private int minId;
+    private int numBlocks;
+
+    TestFileRegionProvider() {
+      minId = MIN_BLK_ID;
+      numBlocks = NUM_PROVIDED_BLKS;
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      return new TestFileRegionIterator(providedBasePath, minId, numBlocks);
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public void refresh() {
+      //do nothing!
+    }
+
+    public void setMinBlkId(int minId) {
+      this.minId = minId;
+    }
+
+    public void setBlockCount(int numBlocks) {
+      this.numBlocks = numBlocks;
+    }
+  }
+
+  private static Storage.StorageDirectory createLocalStorageDirectory(
+      File root, Configuration conf)
+      throws SecurityException, IOException {
+    Storage.StorageDirectory sd =
+        new Storage.StorageDirectory(
+            StorageLocation.parse(root.toURI().toString()));
+    DataStorage.createStorageID(sd, false, conf);
+    return sd;
+  }
+
+  private static Storage.StorageDirectory createProvidedStorageDirectory(
+      String confString, Configuration conf)
+      throws SecurityException, IOException {
+    Storage.StorageDirectory sd =
+        new Storage.StorageDirectory(StorageLocation.parse(confString));
+    DataStorage.createStorageID(sd, false, conf);
+    return sd;
+  }
+
+  private static void createStorageDirs(DataStorage storage,
+      Configuration conf, int numDirs, int numProvidedDirs)
+          throws IOException {
+    List<Storage.StorageDirectory> dirs =
+        new ArrayList<Storage.StorageDirectory>();
+    List<String> dirStrings = new ArrayList<String>();
+    FileUtils.deleteDirectory(new File(BASE_DIR));
+    for (int i = 0; i < numDirs; i++) {
+      File loc = new File(BASE_DIR, "data" + i);
+      dirStrings.add(new Path(loc.toString()).toUri().toString());
+      loc.mkdirs();
+      dirs.add(createLocalStorageDirectory(loc, conf));
+      when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
+    }
+
+    for (int i = numDirs; i < numDirs + numProvidedDirs; i++) {
+      File loc = new File(BASE_DIR, "data" + i);
+      providedBasePath = loc.getAbsolutePath();
+      loc.mkdirs();
+      String dirString = "[PROVIDED]" +
+          new Path(loc.toString()).toUri().toString();
+      dirStrings.add(dirString);
+      dirs.add(createProvidedStorageDirectory(dirString, conf));
+      when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
+    }
+
+    String dataDir = StringUtils.join(",", dirStrings);
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDir);
+    when(storage.dirIterator()).thenReturn(dirs.iterator());
+    when(storage.getNumStorageDirs()).thenReturn(numDirs + numProvidedDirs);
+  }
+
+  private int getNumVolumes() {
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
+      return volumes.size();
+    } catch (IOException e) {
+      return 0;
+    }
+  }
+
+  private void compareBlkFile(InputStream ins, String filepath)
+      throws FileNotFoundException, IOException {
+    try (ReadableByteChannel i = Channels.newChannel(
+        new FileInputStream(new File(filepath)))) {
+      try (ReadableByteChannel j = Channels.newChannel(ins)) {
+        ByteBuffer ib = ByteBuffer.allocate(4096);
+        ByteBuffer jb = ByteBuffer.allocate(4096);
+        while (true) {
+          int il = i.read(ib);
+          int jl = j.read(jb);
+          if (il < 0 || jl < 0) {
+            assertEquals(il, jl);
+            break;
+          }
+          ib.flip();
+          jb.flip();
+          int cmp = Math.min(ib.remaining(), jb.remaining());
+          for (int k = 0; k < cmp; ++k) {
+            assertEquals(ib.get(), jb.get());
+          }
+          ib.compact();
+          jb.compact();
+        }
+      }
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    datanode = mock(DataNode.class);
+    storage = mock(DataStorage.class);
+    this.conf = new Configuration();
+    this.conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
+
+    when(datanode.getConf()).thenReturn(conf);
+    final DNConf dnConf = new DNConf(datanode);
+    when(datanode.getDnConf()).thenReturn(dnConf);
+
+    final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
+    when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
+    final ShortCircuitRegistry shortCircuitRegistry =
+        new ShortCircuitRegistry(conf);
+    when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry);
+
+    this.conf.setClass(DFSConfigKeys.DFS_PROVIDER_CLASS,
+        TestFileRegionProvider.class, FileRegionProvider.class);
+
+    blkToPathMap = new HashMap<Long, String>();
+    providedVolumes = new LinkedList<FsVolumeImpl>();
+
+    createStorageDirs(
+        storage, conf, NUM_LOCAL_INIT_VOLUMES, NUM_PROVIDED_INIT_VOLUMES);
+
+    dataset = new FsDatasetImpl(datanode, storage, conf);
+    FsVolumeReferences volumes = dataset.getFsVolumeReferences();
+    for (int i = 0; i < volumes.size(); i++) {
+      FsVolumeSpi vol = volumes.get(i);
+      if (vol.getStorageType() == StorageType.PROVIDED) {
+        providedVolumes.add((FsVolumeImpl) vol);
+      }
+    }
+
+    for (String bpid : BLOCK_POOL_IDS) {
+      dataset.addBlockPool(bpid, conf);
+    }
+
+    assertEquals(NUM_LOCAL_INIT_VOLUMES + NUM_PROVIDED_INIT_VOLUMES,
+        getNumVolumes());
+    assertEquals(0, dataset.getNumFailedVolumes());
+  }
+
+  @Test
+  public void testProvidedStorageID() throws IOException {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      assertEquals(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT,
+          providedVolumes.get(i).getStorageID());
+    }
+  }
+
+  @Test
+  public void testBlockLoad() throws IOException {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      FsVolumeImpl vol = providedVolumes.get(i);
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(volumeMap, null);
+
+      assertEquals(vol.getBlockPoolList().length, BLOCK_POOL_IDS.length);
+      for (int j = 0; j < BLOCK_POOL_IDS.length; j++) {
+        if (j != CHOSEN_BP_ID) {
+          //this block pool should not have any blocks
+          assertEquals(null, volumeMap.replicas(BLOCK_POOL_IDS[j]));
+        }
+      }
+      assertEquals(NUM_PROVIDED_BLKS,
+          volumeMap.replicas(BLOCK_POOL_IDS[CHOSEN_BP_ID]).size());
+    }
+  }
+
+  @Test
+  public void testProvidedBlockRead() throws IOException {
+    for (int id = 0; id < NUM_PROVIDED_BLKS; id++) {
+      ExtendedBlock eb = new ExtendedBlock(
+          BLOCK_POOL_IDS[CHOSEN_BP_ID], id, BLK_LEN,
+          HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+      InputStream ins = dataset.getBlockInputStream(eb, 0);
+      String filepath = blkToPathMap.get((long) id);
+      compareBlkFile(ins, filepath);
+    }
+  }
+
+  @Test
+  public void testProvidedBlockIterator() throws IOException {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      FsVolumeImpl vol = providedVolumes.get(i);
+      BlockIterator iter =
+          vol.newBlockIterator(BLOCK_POOL_IDS[CHOSEN_BP_ID], "temp");
+      Set<Long> blockIdsUsed = new HashSet<Long>();
+      while(!iter.atEnd()) {
+        ExtendedBlock eb = iter.nextBlock();
+        long blkId = eb.getBlockId();
+        assertTrue(blkId >= MIN_BLK_ID && blkId < NUM_PROVIDED_BLKS);
+        //all block ids must be unique!
+        assertTrue(!blockIdsUsed.contains(blkId));
+        blockIdsUsed.add(blkId);
+      }
+      assertEquals(NUM_PROVIDED_BLKS, blockIdsUsed.size());
+    }
+  }
+
+
+  @Test
+  public void testRefresh() throws IOException {
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
+      TestFileRegionProvider provider = (TestFileRegionProvider)
+          vol.getFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+      //equivalent to two new blocks appearing
+      provider.setBlockCount(NUM_PROVIDED_BLKS + 2);
+      //equivalent to deleting the first block
+      provider.setMinBlkId(MIN_BLK_ID + 1);
+
+      DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf);
+      scanner.reconcile();
+      ReplicaInfo info = dataset.getBlockReplica(
+          BLOCK_POOL_IDS[CHOSEN_BP_ID], NUM_PROVIDED_BLKS + 1);
+      //new replica should be added to the dataset
+      assertTrue(info != null);
+      try {
+        info = dataset.getBlockReplica(BLOCK_POOL_IDS[CHOSEN_BP_ID], 0);
+      } catch(Exception ex) {
+        LOG.info("Exception expected: " + ex);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02a28b93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
index d5a3948..db8c029 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
@@ -68,7 +68,10 @@ public class TestClusterId {
       fsImage.getStorage().dirIterator(NNStorage.NameNodeDirType.IMAGE);
     StorageDirectory sd = sdit.next();
     Properties props = Storage.readPropertiesFile(sd.getVersionFile());
-    String cid = props.getProperty("clusterID");
+    String cid = null;
+    if (props != null) {
+      cid = props.getProperty("clusterID");
+    }
     LOG.info("successfully formated : sd="+sd.getCurrentDir() + ";cid="+cid);
     return cid;
   }


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: 84ea0817a83dc396478a1f08c254a1c34df42390
Parents: 5ae7ea5
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Mon Nov 6 11:05:59 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

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

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


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


[31/50] [abbrv] hadoop git commit: HDFS-12778. [READ] Report multiple locations for PROVIDED blocks

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


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

Branch: refs/heads/HDFS-9806
Commit: 68e046cc7f5113575c1bf2138c03cd7207bbc47c
Parents: f8b81c0
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Tue Nov 21 14:54:57 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68e046cc/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index f6d38f6..9c82967 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -474,12 +474,12 @@ public class TestNameNodeProvidedImplementation {
   }
 
   private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
-      String filename, int expectedLocations) throws IOException {
-    LocatedBlocks locatedBlocks = client.getLocatedBlocks(
-        filename, 0, baseFileLen);
-    //given the start and length in the above call,
-    //only one LocatedBlock in LocatedBlocks
-    assertEquals(1, locatedBlocks.getLocatedBlocks().size());
+      String filename, long fileLen, long expectedBlocks, int expectedLocations)
+      throws IOException {
+    LocatedBlocks locatedBlocks = client.getLocatedBlocks(filename, 0, fileLen);
+    // given the start and length in the above call,
+    // only one LocatedBlock in LocatedBlocks
+    assertEquals(expectedBlocks, locatedBlocks.getLocatedBlocks().size());
     LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
     assertEquals(expectedLocations, locatedBlock.getLocations().length);
     return locatedBlock.getLocations();
@@ -513,17 +513,20 @@ public class TestNameNodeProvidedImplementation {
         file, newReplication, 10000);
     DFSClient client = new DFSClient(new InetSocketAddress("localhost",
         cluster.getNameNodePort()), cluster.getConfiguration(0));
-    getAndCheckBlockLocations(client, filename, newReplication);
+    getAndCheckBlockLocations(client, filename, baseFileLen, 1, newReplication);
 
     // set the replication back to 1
     newReplication = 1;
     LOG.info("Setting replication of file {} back to {}",
         filename, newReplication);
     fs.setReplication(file, newReplication);
+    // defaultReplication number of replicas should be returned
+    int defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
     DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
-        file, newReplication, 10000);
-    // the only replica left should be the PROVIDED datanode
-    getAndCheckBlockLocations(client, filename, newReplication);
+        file, (short) defaultReplication, 10000);
+    getAndCheckBlockLocations(client, filename, baseFileLen, 1,
+        defaultReplication);
   }
 
   @Test(timeout=30000)
@@ -545,8 +548,9 @@ public class TestNameNodeProvidedImplementation {
 
     if (numFiles >= 1) {
       String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
-
-      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      // 2 locations returned as there are 2 PROVIDED datanodes
+      DatanodeInfo[] dnInfos =
+          getAndCheckBlockLocations(client, filename, baseFileLen, 1, 2);
       //the location should be one of the provided DNs available
       assertTrue(
           dnInfos[0].getDatanodeUuid().equals(
@@ -564,7 +568,7 @@ public class TestNameNodeProvidedImplementation {
           providedDatanode1.getDatanodeId().getXferAddr());
 
       //should find the block on the 2nd provided datanode
-      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
       assertEquals(providedDatanode2.getDatanodeUuid(),
           dnInfos[0].getDatanodeUuid());
 
@@ -575,14 +579,14 @@ public class TestNameNodeProvidedImplementation {
       BlockManagerTestUtil.noticeDeadDatanode(
           cluster.getNameNode(),
           providedDatanode2.getDatanodeId().getXferAddr());
-      getAndCheckBlockLocations(client, filename, 0);
+      getAndCheckBlockLocations(client, filename, baseFileLen, 1, 0);
 
       //restart the provided datanode
       cluster.restartDataNode(providedDNProperties1, true);
       cluster.waitActive();
 
       //should find the block on the 1st provided datanode now
-      dnInfos = getAndCheckBlockLocations(client, filename, 1);
+      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
       //not comparing UUIDs as the datanode can now have a different one.
       assertEquals(providedDatanode1.getDatanodeId().getXferAddr(),
           dnInfos[0].getXferAddr());
@@ -593,20 +597,18 @@ public class TestNameNodeProvidedImplementation {
   public void testTransientDeadDatanodes() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
             FixedBlockResolver.class);
-    // 2 Datanodes, 1 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 2, null,
+    // 3 Datanodes, 2 PROVIDED and other DISK
+    startCluster(NNDIRPATH, 3, null,
         new StorageType[][] {
             {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
     DataNode providedDatanode = cluster.getDataNodes().get(0);
-
-    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-        cluster.getNameNodePort()), cluster.getConfiguration(0));
-
     for (int i= 0; i < numFiles; i++) {
-      verifyFileLocation(i);
+      // expect to have 2 locations as we have 2 provided Datanodes.
+      verifyFileLocation(i, 2);
       // NameNode thinks the datanode is down
       BlockManagerTestUtil.noticeDeadDatanode(
           cluster.getNameNode(),
@@ -614,7 +616,7 @@ public class TestNameNodeProvidedImplementation {
       cluster.waitActive();
       cluster.triggerHeartbeats();
       Thread.sleep(1000);
-      verifyFileLocation(i);
+      verifyFileLocation(i, 2);
     }
   }
 
@@ -622,17 +624,18 @@ public class TestNameNodeProvidedImplementation {
   public void testNamenodeRestart() throws Exception {
     createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
         FixedBlockResolver.class);
-    // 2 Datanodes, 1 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 2, null,
+    // 3 Datanodes, 2 PROVIDED and other DISK
+    startCluster(NNDIRPATH, 3, null,
         new StorageType[][] {
             {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
             {StorageType.DISK}},
         false);
 
-    verifyFileLocation(numFiles - 1);
+    verifyFileLocation(numFiles - 1, 2);
     cluster.restartNameNodes();
     cluster.waitActive();
-    verifyFileLocation(numFiles - 1);
+    verifyFileLocation(numFiles - 1, 2);
   }
 
   /**
@@ -640,18 +643,21 @@ public class TestNameNodeProvidedImplementation {
    * @param fileIndex the index of the file to verify.
    * @throws Exception
    */
-  private void verifyFileLocation(int fileIndex)
+  private void verifyFileLocation(int fileIndex, int replication)
       throws Exception {
-    DataNode providedDatanode = cluster.getDataNodes().get(0);
     DFSClient client = new DFSClient(
         new InetSocketAddress("localhost", cluster.getNameNodePort()),
         cluster.getConfiguration(0));
-    if (fileIndex <= numFiles && fileIndex >= 0) {
-      String filename = "/" + filePrefix + fileIndex + fileSuffix;
-      DatanodeInfo[] dnInfos = getAndCheckBlockLocations(client, filename, 1);
-      // location should be the provided DN
-      assertEquals(providedDatanode.getDatanodeUuid(),
-          dnInfos[0].getDatanodeUuid());
+    if (fileIndex < numFiles && fileIndex >= 0) {
+      String filename = filePrefix + fileIndex + fileSuffix;
+      File file = new File(new Path(NAMEPATH, filename).toUri());
+      long fileLen = file.length();
+      long blockSize = conf.getLong(FixedBlockResolver.BLOCKSIZE,
+          FixedBlockResolver.BLOCKSIZE_DEFAULT);
+      long numLocatedBlocks =
+          fileLen == 0 ? 1 : (long) Math.ceil(fileLen * 1.0 / blockSize);
+      getAndCheckBlockLocations(client, "/" + filename, fileLen,
+          numLocatedBlocks, replication);
     }
   }
 
@@ -669,4 +675,55 @@ public class TestNameNodeProvidedImplementation {
     NameNode nn = cluster.getNameNode();
     assertEquals(clusterID, nn.getNamesystem().getClusterId());
   }
+
+  @Test(timeout=30000)
+  public void testNumberOfProvidedLocations() throws Exception {
+    // set default replication to 4
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    // start with 4 PROVIDED location
+    startCluster(NNDIRPATH, 4,
+        new StorageType[]{
+            StorageType.PROVIDED, StorageType.DISK},
+        null,
+        false);
+    int expectedLocations = 4;
+    for (int i = 0; i < numFiles; i++) {
+      verifyFileLocation(i, expectedLocations);
+    }
+    // stop 2 datanodes, one after the other and verify number of locations.
+    for (int i = 1; i <= 2; i++) {
+      DataNode dn = cluster.getDataNodes().get(0);
+      cluster.stopDataNode(0);
+      // make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
+          dn.getDatanodeId().getXferAddr());
+
+      expectedLocations = 4 - i;
+      for (int j = 0; j < numFiles; j++) {
+        verifyFileLocation(j, expectedLocations);
+      }
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testNumberOfProvidedLocationsManyBlocks() throws Exception {
+    // increase number of blocks per file to at least 10 blocks per file
+    conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10);
+    // set default replication to 4
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    // start with 4 PROVIDED location
+    startCluster(NNDIRPATH, 4,
+        new StorageType[]{
+            StorageType.PROVIDED, StorageType.DISK},
+        null,
+        false);
+    int expectedLocations = 4;
+    for (int i = 0; i < numFiles; i++) {
+      verifyFileLocation(i, expectedLocations);
+    }
+  }
 }


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


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

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


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

Branch: refs/heads/HDFS-9806
Commit: c7bbd010c295ed8c99cc77a04c32db88587e1d48
Parents: 1682da1
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Sat Dec 2 12:22:00 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:29 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7bbd010/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
new file mode 100644
index 0000000..66971a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import static org.fusesource.leveldbjni.JniDBFactory.factory;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.fromBlockBytes;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.fromProvidedStorageLocationBytes;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.toProtoBufBytes;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A LevelDB based implementation of {@link BlockAliasMap}.
+ */
+public class LevelDBFileRegionAliasMap
+      extends BlockAliasMap<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private LevelDBOptions opts = new LevelDBOptions();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(LevelDBFileRegionAliasMap.class);
+
+  @Override
+  public void setConf(Configuration conf) {
+    opts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts) throws IOException {
+    if (null == opts) {
+      opts = this.opts;
+    }
+    if (!(opts instanceof LevelDBOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    LevelDBOptions o = (LevelDBOptions) opts;
+    return new LevelDBFileRegionAliasMap.LevelDBReader(
+        createDB(o.levelDBPath, false));
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    if (null == opts) {
+      opts = this.opts;
+    }
+    if (!(opts instanceof LevelDBOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    LevelDBOptions o = (LevelDBOptions) opts;
+    return new LevelDBFileRegionAliasMap.LevelDBWriter(
+        createDB(o.levelDBPath, true));
+  }
+
+  private static DB createDB(String levelDBPath, boolean createIfMissing)
+      throws IOException {
+    if (levelDBPath == null || levelDBPath.length() == 0) {
+      throw new IllegalArgumentException(
+          "A valid path needs to be specified for "
+              + LevelDBFileRegionAliasMap.class + " using the parameter "
+              + DFS_PROVIDED_ALIASMAP_LEVELDB_PATH);
+    }
+    org.iq80.leveldb.Options options = new org.iq80.leveldb.Options();
+    options.createIfMissing(createIfMissing);
+    return factory.open(new File(levelDBPath), options);
+  }
+
+  @Override
+  public void refresh() throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Do nothing.
+  }
+
+  /**
+   * Class specifying reader options for the {@link LevelDBFileRegionAliasMap}.
+   */
+  public static class LevelDBOptions implements LevelDBReader.Options,
+      LevelDBWriter.Options, Configurable {
+    private Configuration conf;
+    private String levelDBPath;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      this.levelDBPath = conf.get(DFS_PROVIDED_ALIASMAP_LEVELDB_PATH);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public LevelDBOptions filename(String levelDBPath) {
+      this.levelDBPath = levelDBPath;
+      return this;
+    }
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as LevelDB files.
+   */
+  public static class LevelDBReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link LevelDBReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(String levelDBPath);
+    }
+
+    private DB db;
+
+    LevelDBReader(DB db) {
+      this.db = db;
+    }
+
+    @Override
+    public Optional<FileRegion> resolve(Block block) throws IOException {
+      if (db == null) {
+        return Optional.empty();
+      }
+      // consider layering index w/ composable format
+      byte[] key = toProtoBufBytes(block);
+      byte[] value = db.get(key);
+      ProvidedStorageLocation psl = fromProvidedStorageLocationBytes(value);
+      return Optional.of(new FileRegion(block, psl));
+    }
+
+    static class FRIterator implements Iterator<FileRegion> {
+      private final DBIterator internal;
+
+      FRIterator(DBIterator internal) {
+        this.internal = internal;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return internal.hasNext();
+      }
+
+      @Override
+      public FileRegion next() {
+        Map.Entry<byte[], byte[]> entry = internal.next();
+        if (entry == null) {
+          return null;
+        }
+        try {
+          Block block = fromBlockBytes(entry.getKey());
+          ProvidedStorageLocation psl =
+              fromProvidedStorageLocationBytes(entry.getValue());
+          return new FileRegion(block, psl);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    public Iterator<FileRegion> iterator() {
+      if (db == null) {
+        return null;
+      }
+      DBIterator iterator = db.iterator();
+      iterator.seekToFirst();
+      return new FRIterator(iterator);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (db != null) {
+        db.close();
+      }
+    }
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as LevelDB files.
+   */
+  public static class LevelDBWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options filename(String levelDBPath);
+    }
+
+    private final DB db;
+
+    LevelDBWriter(DB db) {
+      this.db = db;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      byte[] key = toProtoBufBytes(token.getBlock());
+      byte[] value = toProtoBufBytes(token.getProvidedStorageLocation());
+      db.put(key, value);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (db != null) {
+        db.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7bbd010/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
new file mode 100644
index 0000000..21199e1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Tests for the {@link LevelDBFileRegionAliasMap}.
+ */
+public class TestLevelDBFileRegionAliasMap {
+
+  /**
+   * A basic test to verify that we can write data and read it back again.
+   * @throws Exception
+   */
+  @Test
+  public void testReadBack() throws Exception {
+    File dbFile = Files.createTempDirectory("fileregionformat")
+        .toFile();
+    try {
+      LevelDBFileRegionAliasMap frf = new LevelDBFileRegionAliasMap();
+      LevelDBFileRegionAliasMap.LevelDBOptions opts =
+          new LevelDBFileRegionAliasMap.LevelDBOptions()
+              .filename(dbFile.getAbsolutePath());
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts);
+
+      FileRegion fr = new FileRegion(1, new Path("/file"), 1, 1, 1);
+      writer.store(fr);
+      writer.close();
+
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts);
+      FileRegion fr2 = reader.resolve(new Block(1, 1, 1)).get();
+      assertEquals(fr, fr2);
+      reader.close();
+    } finally {
+      dbFile.delete();
+    }
+  }
+
+  @Test
+  /**
+   * A basic test to verify that we can read a bunch of data that we've written.
+   */
+  public void testIterate() throws Exception {
+    FileRegion[] regions = new FileRegion[10];
+    regions[0] = new FileRegion(1, new Path("/file1"), 0, 1024, 1);
+    regions[1] = new FileRegion(2, new Path("/file1"), 1024, 1024, 1);
+    regions[2] = new FileRegion(3, new Path("/file1"), 2048, 1024, 1);
+    regions[3] = new FileRegion(4, new Path("/file2"), 0, 1024, 1);
+    regions[4] = new FileRegion(5, new Path("/file2"), 1024, 1024, 1);
+    regions[5] = new FileRegion(6, new Path("/file2"), 2048, 1024, 1);
+    regions[6] = new FileRegion(7, new Path("/file2"), 3072, 1024, 1);
+    regions[7] = new FileRegion(8, new Path("/file3"), 0, 1024, 1);
+    regions[8] = new FileRegion(9, new Path("/file4"), 0, 1024, 1);
+    regions[9] = new FileRegion(10, new Path("/file5"), 0, 1024,  1);
+    File dbFile = Files.createTempDirectory("fileregionformat")
+        .toFile();
+    try {
+      LevelDBFileRegionAliasMap frf = new LevelDBFileRegionAliasMap();
+      LevelDBFileRegionAliasMap.LevelDBOptions opts =
+          new LevelDBFileRegionAliasMap.LevelDBOptions()
+              .filename(dbFile.getAbsolutePath());
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts);
+
+      for (FileRegion fr : regions) {
+        writer.store(fr);
+      }
+      writer.close();
+
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts);
+      Iterator<FileRegion> it = reader.iterator();
+      int last = -1;
+      int count = 0;
+      while(it.hasNext()) {
+        FileRegion fr = it.next();
+        int blockId = (int)fr.getBlock().getBlockId();
+        assertEquals(regions[blockId-1], fr);
+        assertNotEquals(blockId, last);
+        last = blockId;
+        count++;
+      }
+      assertEquals(count, 10);
+
+      reader.close();
+    } finally {
+      dbFile.delete();
+    }
+  }
+}


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


[36/50] [abbrv] hadoop git commit: HDFS-12777. [READ] Reduce memory and CPU footprint for PROVIDED volumes.

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


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

Branch: refs/heads/HDFS-9806
Commit: f2b1c1af9cc62c145f072cdb766ba6096682616a
Parents: 834f28a
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Nov 10 10:19:33 2017 -0800
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:28 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2b1c1af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 3b6d06c..8fb8551 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -530,6 +530,10 @@ public class DirectoryScanner implements Runnable {
           new HashMap<Integer, Future<ScanInfoPerBlockPool>>();
 
       for (int i = 0; i < volumes.size(); i++) {
+        if (volumes.get(i).getStorageType() == StorageType.PROVIDED) {
+          // Disable scanning PROVIDED volumes to keep overhead low
+          continue;
+        }
         ReportCompiler reportCompiler =
             new ReportCompiler(datanode, volumes.get(i));
         Future<ScanInfoPerBlockPool> result =

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

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

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

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

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


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


[24/50] [abbrv] hadoop git commit: HDFS-11902. [READ] Merge BlockFormatProvider and FileRegionProvider.

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


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

Branch: refs/heads/HDFS-9806
Commit: 5ae7ea559f0a8a10f13b5d3ee7704f3cc78a2ab7
Parents: 9286e66
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Nov 3 13:45:56 2017 -0700
Committer: Virajith Jalaparti <vi...@apache.org>
Committed: Fri Dec 15 10:18:27 2017 -0800

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
deleted file mode 100644
index 930263d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockFormatProvider.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.BlockAlias;
-import org.apache.hadoop.hdfs.server.common.BlockFormat;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Loads provided blocks from a {@link BlockFormat}.
- */
-public class BlockFormatProvider extends BlockProvider
-    implements Configurable {
-
-  private Configuration conf;
-  private BlockFormat<? extends BlockAlias> blockFormat;
-  public static final Logger LOG =
-      LoggerFactory.getLogger(BlockFormatProvider.class);
-
-  @Override
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  public void setConf(Configuration conf) {
-    Class<? extends BlockFormat> c = conf.getClass(
-        DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-        TextFileRegionFormat.class, BlockFormat.class);
-    blockFormat = ReflectionUtils.newInstance(c, conf);
-    LOG.info("Loaded BlockFormat class : " + c.getClass().getName());
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public Iterator<Block> iterator() {
-    try {
-      final BlockFormat.Reader<? extends BlockAlias> reader =
-          blockFormat.getReader(null);
-
-      return new Iterator<Block>() {
-
-        private final Iterator<? extends BlockAlias> inner = reader.iterator();
-
-        @Override
-        public boolean hasNext() {
-          return inner.hasNext();
-        }
-
-        @Override
-        public Block next() {
-          return inner.next().getBlock();
-        }
-
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to read provided blocks", e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
deleted file mode 100644
index 2214868..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockProvider.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap.ProvidedBlockList;
-import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.util.RwLock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Used to load provided blocks in the {@link BlockManager}.
- */
-public abstract class BlockProvider implements Iterable<Block> {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ProvidedStorageMap.class);
-
-  private RwLock lock;
-  private BlockManager bm;
-  private DatanodeStorageInfo storage;
-  private boolean hasDNs = false;
-
-  /**
-   * @param lock the namesystem lock
-   * @param bm block manager
-   * @param storage storage for provided blocks
-   */
-  void init(RwLock lock, BlockManager bm, DatanodeStorageInfo storage) {
-    this.bm = bm;
-    this.lock = lock;
-    this.storage = storage;
-  }
-
-  /**
-   * start the processing of block report for provided blocks.
-   * @throws IOException
-   */
-  void start(BlockReportContext context) throws IOException {
-    assert lock.hasWriteLock() : "Not holding write lock";
-    if (hasDNs) {
-      return;
-    }
-    if (storage.getBlockReportCount() == 0) {
-      LOG.info("Calling process first blk report from storage: " + storage);
-      // first pass; periodic refresh should call bm.processReport
-      bm.processFirstBlockReport(storage, new ProvidedBlockList(iterator()));
-    } else {
-      bm.processReport(storage, new ProvidedBlockList(iterator()), context);
-    }
-    hasDNs = true;
-  }
-
-  void stop() {
-    assert lock.hasWriteLock() : "Not holding write lock";
-    hasDNs = false;
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
deleted file mode 100644
index 66e7fdf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockFormat.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.common;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-
-/**
- * An abstract class used to read and write block maps for provided blocks.
- */
-public abstract class BlockFormat<T extends BlockAlias>  {
-
-  /**
-   * An abstract class that is used to read {@link BlockAlias}es
-   * for provided blocks.
-   */
-  public static abstract class Reader<U extends BlockAlias>
-      implements Iterable<U>, Closeable {
-
-    /**
-     * reader options.
-     */
-    public interface Options { }
-
-    public abstract U resolve(Block ident) throws IOException;
-
-  }
-
-  /**
-   * Returns the reader for the provided block map.
-   * @param opts reader options
-   * @return {@link Reader} to the block map.
-   * @throws IOException
-   */
-  public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
-
-  /**
-   * An abstract class used as a writer for the provided block map.
-   */
-  public static abstract class Writer<U extends BlockAlias>
-      implements Closeable {
-    /**
-     * writer options.
-     */
-    public interface Options { }
-
-    public abstract void store(U token) throws IOException;
-
-  }
-
-  /**
-   * Returns the writer for the provided block map.
-   * @param opts writer options.
-   * @return {@link Writer} to the block map.
-   * @throws IOException
-   */
-  public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
-
-  /**
-   * Refresh based on the underlying block map.
-   * @throws IOException
-   */
-  public abstract void refresh() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
deleted file mode 100644
index 2e94239..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegionProvider.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.common;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-
-/**
- * This class is a stub for reading file regions from the block map.
- */
-public class FileRegionProvider implements Iterable<FileRegion> {
-  @Override
-  public Iterator<FileRegion> iterator() {
-    return Collections.emptyListIterator();
-  }
-
-  public void refresh() throws IOException {
-    return;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
deleted file mode 100644
index eacd08f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionFormat.java
+++ /dev/null
@@ -1,442 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.common;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Collections;
-import java.util.IdentityHashMap;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * This class is used for block maps stored as text files,
- * with a specified delimiter.
- */
-public class TextFileRegionFormat
-    extends BlockFormat<FileRegion> implements Configurable {
-
-  private Configuration conf;
-  private ReaderOptions readerOpts = TextReader.defaults();
-  private WriterOptions writerOpts = TextWriter.defaults();
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(TextFileRegionFormat.class);
-  @Override
-  public void setConf(Configuration conf) {
-    readerOpts.setConf(conf);
-    writerOpts.setConf(conf);
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public Reader<FileRegion> getReader(Reader.Options opts)
-      throws IOException {
-    if (null == opts) {
-      opts = readerOpts;
-    }
-    if (!(opts instanceof ReaderOptions)) {
-      throw new IllegalArgumentException("Invalid options " + opts.getClass());
-    }
-    ReaderOptions o = (ReaderOptions) opts;
-    Configuration readerConf = (null == o.getConf())
-        ? new Configuration()
-            : o.getConf();
-    return createReader(o.file, o.delim, readerConf);
-  }
-
-  @VisibleForTesting
-  TextReader createReader(Path file, String delim, Configuration cfg)
-      throws IOException {
-    FileSystem fs = file.getFileSystem(cfg);
-    if (fs instanceof LocalFileSystem) {
-      fs = ((LocalFileSystem)fs).getRaw();
-    }
-    CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
-    CompressionCodec codec = factory.getCodec(file);
-    return new TextReader(fs, file, codec, delim);
-  }
-
-  @Override
-  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
-    if (null == opts) {
-      opts = writerOpts;
-    }
-    if (!(opts instanceof WriterOptions)) {
-      throw new IllegalArgumentException("Invalid options " + opts.getClass());
-    }
-    WriterOptions o = (WriterOptions) opts;
-    Configuration cfg = (null == o.getConf())
-        ? new Configuration()
-            : o.getConf();
-    if (o.codec != null) {
-      CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
-      CompressionCodec codec = factory.getCodecByName(o.codec);
-      String name = o.file.getName() + codec.getDefaultExtension();
-      o.filename(new Path(o.file.getParent(), name));
-      return createWriter(o.file, codec, o.delim, cfg);
-    }
-    return createWriter(o.file, null, o.delim, conf);
-  }
-
-  @VisibleForTesting
-  TextWriter createWriter(Path file, CompressionCodec codec, String delim,
-      Configuration cfg) throws IOException {
-    FileSystem fs = file.getFileSystem(cfg);
-    if (fs instanceof LocalFileSystem) {
-      fs = ((LocalFileSystem)fs).getRaw();
-    }
-    OutputStream tmp = fs.create(file);
-    java.io.Writer out = new BufferedWriter(new OutputStreamWriter(
-          (null == codec) ? tmp : codec.createOutputStream(tmp), "UTF-8"));
-    return new TextWriter(out, delim);
-  }
-
-  /**
-   * Class specifying reader options for the {@link TextFileRegionFormat}.
-   */
-  public static class ReaderOptions
-      implements TextReader.Options, Configurable {
-
-    private Configuration conf;
-    private String delim =
-        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
-    private Path file = new Path(
-        new File(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT)
-        .toURI().toString());
-
-    @Override
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-      String tmpfile = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_READ_PATH,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
-      file = new Path(tmpfile);
-      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT);
-      LOG.info("TextFileRegionFormat: read path " + tmpfile.toString());
-    }
-
-    @Override
-    public Configuration getConf() {
-      return conf;
-    }
-
-    @Override
-    public ReaderOptions filename(Path file) {
-      this.file = file;
-      return this;
-    }
-
-    @Override
-    public ReaderOptions delimiter(String delim) {
-      this.delim = delim;
-      return this;
-    }
-  }
-
-  /**
-   * Class specifying writer options for the {@link TextFileRegionFormat}.
-   */
-  public static class WriterOptions
-      implements TextWriter.Options, Configurable {
-
-    private Configuration conf;
-    private String codec = null;
-    private Path file =
-        new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
-    private String delim =
-        DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT;
-
-    @Override
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-      String tmpfile = conf.get(
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_WRITE_PATH, file.toString());
-      file = new Path(tmpfile);
-      codec = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_CODEC);
-      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER,
-          DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_DELIMITER_DEFAULT);
-    }
-
-    @Override
-    public Configuration getConf() {
-      return conf;
-    }
-
-    @Override
-    public WriterOptions filename(Path file) {
-      this.file = file;
-      return this;
-    }
-
-    public String getCodec() {
-      return codec;
-    }
-
-    public Path getFile() {
-      return file;
-    }
-
-    @Override
-    public WriterOptions codec(String codec) {
-      this.codec = codec;
-      return this;
-    }
-
-    @Override
-    public WriterOptions delimiter(String delim) {
-      this.delim = delim;
-      return this;
-    }
-
-  }
-
-  /**
-   * This class is used as a reader for block maps which
-   * are stored as delimited text files.
-   */
-  public static class TextReader extends Reader<FileRegion> {
-
-    /**
-     * Options for {@link TextReader}.
-     */
-    public interface Options extends Reader.Options {
-      Options filename(Path file);
-      Options delimiter(String delim);
-    }
-
-    static ReaderOptions defaults() {
-      return new ReaderOptions();
-    }
-
-    private final Path file;
-    private final String delim;
-    private final FileSystem fs;
-    private final CompressionCodec codec;
-    private final Map<FRIterator, BufferedReader> iterators;
-
-    protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
-        String delim) {
-      this(fs, file, codec, delim,
-          new IdentityHashMap<FRIterator, BufferedReader>());
-    }
-
-    TextReader(FileSystem fs, Path file, CompressionCodec codec, String delim,
-        Map<FRIterator, BufferedReader> iterators) {
-      this.fs = fs;
-      this.file = file;
-      this.codec = codec;
-      this.delim = delim;
-      this.iterators = Collections.synchronizedMap(iterators);
-    }
-
-    @Override
-    public FileRegion resolve(Block ident) throws IOException {
-      // consider layering index w/ composable format
-      Iterator<FileRegion> i = iterator();
-      try {
-        while (i.hasNext()) {
-          FileRegion f = i.next();
-          if (f.getBlock().equals(ident)) {
-            return f;
-          }
-        }
-      } finally {
-        BufferedReader r = iterators.remove(i);
-        if (r != null) {
-          // null on last element
-          r.close();
-        }
-      }
-      return null;
-    }
-
-    class FRIterator implements Iterator<FileRegion> {
-
-      private FileRegion pending;
-
-      @Override
-      public boolean hasNext() {
-        return pending != null;
-      }
-
-      @Override
-      public FileRegion next() {
-        if (null == pending) {
-          throw new NoSuchElementException();
-        }
-        FileRegion ret = pending;
-        try {
-          pending = nextInternal(this);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-        return ret;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    }
-
-    private FileRegion nextInternal(Iterator<FileRegion> i) throws IOException {
-      BufferedReader r = iterators.get(i);
-      if (null == r) {
-        throw new IllegalStateException();
-      }
-      String line = r.readLine();
-      if (null == line) {
-        iterators.remove(i);
-        return null;
-      }
-      String[] f = line.split(delim);
-      if (f.length != 6) {
-        throw new IOException("Invalid line: " + line);
-      }
-      return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
-          Long.parseLong(f[2]), Long.parseLong(f[3]), f[5],
-          Long.parseLong(f[4]));
-    }
-
-    public InputStream createStream() throws IOException {
-      InputStream i = fs.open(file);
-      if (codec != null) {
-        i = codec.createInputStream(i);
-      }
-      return i;
-    }
-
-    @Override
-    public Iterator<FileRegion> iterator() {
-      FRIterator i = new FRIterator();
-      try {
-        BufferedReader r =
-            new BufferedReader(new InputStreamReader(createStream(), "UTF-8"));
-        iterators.put(i, r);
-        i.pending = nextInternal(i);
-      } catch (IOException e) {
-        iterators.remove(i);
-        throw new RuntimeException(e);
-      }
-      return i;
-    }
-
-    @Override
-    public void close() throws IOException {
-      ArrayList<IOException> ex = new ArrayList<>();
-      synchronized (iterators) {
-        for (Iterator<BufferedReader> i = iterators.values().iterator();
-             i.hasNext();) {
-          try {
-            BufferedReader r = i.next();
-            r.close();
-          } catch (IOException e) {
-            ex.add(e);
-          } finally {
-            i.remove();
-          }
-        }
-        iterators.clear();
-      }
-      if (!ex.isEmpty()) {
-        throw MultipleIOException.createIOException(ex);
-      }
-    }
-
-  }
-
-  /**
-   * This class is used as a writer for block maps which
-   * are stored as delimited text files.
-   */
-  public static class TextWriter extends Writer<FileRegion> {
-
-    /**
-     * Interface for Writer options.
-     */
-    public interface Options extends Writer.Options {
-      Options codec(String codec);
-      Options filename(Path file);
-      Options delimiter(String delim);
-    }
-
-    public static WriterOptions defaults() {
-      return new WriterOptions();
-    }
-
-    private final String delim;
-    private final java.io.Writer out;
-
-    public TextWriter(java.io.Writer out, String delim) {
-      this.out = out;
-      this.delim = delim;
-    }
-
-    @Override
-    public void store(FileRegion token) throws IOException {
-      out.append(String.valueOf(token.getBlock().getBlockId())).append(delim);
-      out.append(token.getPath().toString()).append(delim);
-      out.append(Long.toString(token.getOffset())).append(delim);
-      out.append(Long.toString(token.getLength())).append(delim);
-      out.append(Long.toString(token.getGenerationStamp())).append(delim);
-      out.append(token.getBlockPoolId()).append("\n");
-    }
-
-    @Override
-    public void close() throws IOException {
-      out.close();
-    }
-
-  }
-
-  @Override
-  public void refresh() throws IOException {
-    //nothing to do;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
deleted file mode 100644
index 0fa667e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/TextFileRegionProvider.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.common;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * This class is used to read file regions from block maps
- * specified using delimited text.
- */
-public class TextFileRegionProvider
-    extends FileRegionProvider implements Configurable {
-
-  private Configuration conf;
-  private BlockFormat<FileRegion> fmt;
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void setConf(Configuration conf) {
-    fmt = ReflectionUtils.newInstance(
-        conf.getClass(DFSConfigKeys.DFS_PROVIDER_BLK_FORMAT_CLASS,
-            TextFileRegionFormat.class,
-            BlockFormat.class),
-        conf);
-    ((Configurable)fmt).setConf(conf); //redundant?
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public Iterator<FileRegion> iterator() {
-    try {
-      final BlockFormat.Reader<FileRegion> r = fmt.getReader(null);
-      return new Iterator<FileRegion>() {
-
-        private final Iterator<FileRegion> inner = r.iterator();
-
-        @Override
-        public boolean hasNext() {
-          return inner.hasNext();
-        }
-
-        @Override
-        public FileRegion next() {
-          return inner.next();
-        }
-
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    } catch (IOException e) {
-      throw new RuntimeException("Failed to read provided blocks", e);
-    }
-  }
-
-  @Override
-  public void refresh() throws IOException {
-    fmt.refresh();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
new file mode 100644
index 0000000..d276fb5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common.blockaliasmap;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.BlockAlias;
+
+/**
+ * An abstract class used to read and write block maps for provided blocks.
+ */
+public abstract class BlockAliasMap<T extends BlockAlias> {
+
+  /**
+   * An abstract class that is used to read {@link BlockAlias}es
+   * for provided blocks.
+   */
+  public static abstract class Reader<U extends BlockAlias>
+      implements Iterable<U>, Closeable {
+
+    /**
+     * reader options.
+     */
+    public interface Options { }
+
+    /**
+     * @param ident block to resolve
+     * @return BlockAlias correspoding to the provided block.
+     * @throws IOException
+     */
+    public abstract U resolve(Block ident) throws IOException;
+
+  }
+
+  /**
+   * Returns a reader to the alias map.
+   * @param opts reader options
+   * @return {@link Reader} to the alias map.
+   * @throws IOException
+   */
+  public abstract Reader<T> getReader(Reader.Options opts) throws IOException;
+
+  /**
+   * An abstract class used as a writer for the provided block map.
+   */
+  public static abstract class Writer<U extends BlockAlias>
+      implements Closeable {
+    /**
+     * writer options.
+     */
+    public interface Options { }
+
+    public abstract void store(U token) throws IOException;
+
+  }
+
+  /**
+   * Returns the writer for the alias map.
+   * @param opts writer options.
+   * @return {@link Writer} to the alias map.
+   * @throws IOException
+   */
+  public abstract Writer<T> getWriter(Writer.Options opts) throws IOException;
+
+  /**
+   * Refresh the alias map.
+   * @throws IOException
+   */
+  public abstract void refresh() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
new file mode 100644
index 0000000..80f48c1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
@@ -0,0 +1,445 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is used for block maps stored as text files,
+ * with a specified delimiter.
+ */
+public class TextFileRegionAliasMap
+    extends BlockAliasMap<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private ReaderOptions readerOpts = TextReader.defaults();
+  private WriterOptions writerOpts = TextWriter.defaults();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TextFileRegionAliasMap.class);
+  @Override
+  public void setConf(Configuration conf) {
+    readerOpts.setConf(conf);
+    writerOpts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts)
+      throws IOException {
+    if (null == opts) {
+      opts = readerOpts;
+    }
+    if (!(opts instanceof ReaderOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    ReaderOptions o = (ReaderOptions) opts;
+    Configuration readerConf = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    return createReader(o.file, o.delim, readerConf);
+  }
+
+  @VisibleForTesting
+  TextReader createReader(Path file, String delim, Configuration cfg)
+      throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+    CompressionCodec codec = factory.getCodec(file);
+    return new TextReader(fs, file, codec, delim);
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts) throws IOException {
+    if (null == opts) {
+      opts = writerOpts;
+    }
+    if (!(opts instanceof WriterOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    WriterOptions o = (WriterOptions) opts;
+    Configuration cfg = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    if (o.codec != null) {
+      CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+      CompressionCodec codec = factory.getCodecByName(o.codec);
+      String name = o.file.getName() + codec.getDefaultExtension();
+      o.filename(new Path(o.file.getParent(), name));
+      return createWriter(o.file, codec, o.delim, cfg);
+    }
+    return createWriter(o.file, null, o.delim, conf);
+  }
+
+  @VisibleForTesting
+  TextWriter createWriter(Path file, CompressionCodec codec, String delim,
+      Configuration cfg) throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    OutputStream tmp = fs.create(file);
+    java.io.Writer out = new BufferedWriter(new OutputStreamWriter(
+          (null == codec) ? tmp : codec.createOutputStream(tmp), "UTF-8"));
+    return new TextWriter(out, delim);
+  }
+
+  /**
+   * Class specifying reader options for the {@link TextFileRegionAliasMap}.
+   */
+  public static class ReaderOptions
+      implements TextReader.Options, Configurable {
+
+    private Configuration conf;
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
+    private Path file = new Path(
+        new File(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT).toURI()
+            .toString());
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile =
+          conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_PATH,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
+      file = new Path(tmpfile);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+      LOG.info("TextFileRegionAliasMap: read path " + tmpfile.toString());
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public ReaderOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    @Override
+    public ReaderOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+  }
+
+  /**
+   * Class specifying writer options for the {@link TextFileRegionAliasMap}.
+   */
+  public static class WriterOptions
+      implements TextWriter.Options, Configurable {
+
+    private Configuration conf;
+    private String codec = null;
+    private Path file =
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);;
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile = conf.get(
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_PATH, file.toString());
+      file = new Path(tmpfile);
+      codec = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_CODEC);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public WriterOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    public String getCodec() {
+      return codec;
+    }
+
+    public Path getFile() {
+      return file;
+    }
+
+    @Override
+    public WriterOptions codec(String codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    @Override
+    public WriterOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link TextReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    static ReaderOptions defaults() {
+      return new ReaderOptions();
+    }
+
+    private final Path file;
+    private final String delim;
+    private final FileSystem fs;
+    private final CompressionCodec codec;
+    private final Map<FRIterator, BufferedReader> iterators;
+
+    protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
+        String delim) {
+      this(fs, file, codec, delim,
+          new IdentityHashMap<FRIterator, BufferedReader>());
+    }
+
+    TextReader(FileSystem fs, Path file, CompressionCodec codec, String delim,
+        Map<FRIterator, BufferedReader> iterators) {
+      this.fs = fs;
+      this.file = file;
+      this.codec = codec;
+      this.delim = delim;
+      this.iterators = Collections.synchronizedMap(iterators);
+    }
+
+    @Override
+    public FileRegion resolve(Block ident) throws IOException {
+      // consider layering index w/ composable format
+      Iterator<FileRegion> i = iterator();
+      try {
+        while (i.hasNext()) {
+          FileRegion f = i.next();
+          if (f.getBlock().equals(ident)) {
+            return f;
+          }
+        }
+      } finally {
+        BufferedReader r = iterators.remove(i);
+        if (r != null) {
+          // null on last element
+          r.close();
+        }
+      }
+      return null;
+    }
+
+    class FRIterator implements Iterator<FileRegion> {
+
+      private FileRegion pending;
+
+      @Override
+      public boolean hasNext() {
+        return pending != null;
+      }
+
+      @Override
+      public FileRegion next() {
+        if (null == pending) {
+          throw new NoSuchElementException();
+        }
+        FileRegion ret = pending;
+        try {
+          pending = nextInternal(this);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return ret;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    private FileRegion nextInternal(Iterator<FileRegion> i) throws IOException {
+      BufferedReader r = iterators.get(i);
+      if (null == r) {
+        throw new IllegalStateException();
+      }
+      String line = r.readLine();
+      if (null == line) {
+        iterators.remove(i);
+        return null;
+      }
+      String[] f = line.split(delim);
+      if (f.length != 6) {
+        throw new IOException("Invalid line: " + line);
+      }
+      return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
+          Long.parseLong(f[2]), Long.parseLong(f[3]), f[5],
+          Long.parseLong(f[4]));
+    }
+
+    public InputStream createStream() throws IOException {
+      InputStream i = fs.open(file);
+      if (codec != null) {
+        i = codec.createInputStream(i);
+      }
+      return i;
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      FRIterator i = new FRIterator();
+      try {
+        BufferedReader r =
+            new BufferedReader(new InputStreamReader(createStream(), "UTF-8"));
+        iterators.put(i, r);
+        i.pending = nextInternal(i);
+      } catch (IOException e) {
+        iterators.remove(i);
+        throw new RuntimeException(e);
+      }
+      return i;
+    }
+
+    @Override
+    public void close() throws IOException {
+      ArrayList<IOException> ex = new ArrayList<>();
+      synchronized (iterators) {
+        for (Iterator<BufferedReader> i = iterators.values().iterator();
+             i.hasNext();) {
+          try {
+            BufferedReader r = i.next();
+            r.close();
+          } catch (IOException e) {
+            ex.add(e);
+          } finally {
+            i.remove();
+          }
+        }
+        iterators.clear();
+      }
+      if (!ex.isEmpty()) {
+        throw MultipleIOException.createIOException(ex);
+      }
+    }
+
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options codec(String codec);
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    public static WriterOptions defaults() {
+      return new WriterOptions();
+    }
+
+    private final String delim;
+    private final java.io.Writer out;
+
+    public TextWriter(java.io.Writer out, String delim) {
+      this.out = out;
+      this.delim = delim;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      out.append(String.valueOf(token.getBlock().getBlockId())).append(delim);
+      out.append(token.getPath().toString()).append(delim);
+      out.append(Long.toString(token.getOffset())).append(delim);
+      out.append(Long.toString(token.getLength())).append(delim);
+      out.append(Long.toString(token.getGenerationStamp())).append(delim);
+      out.append(token.getBlockPoolId()).append("\n");
+    }
+
+    @Override
+    public void close() throws IOException {
+      out.close();
+    }
+
+  }
+
+  @Override
+  public void refresh() throws IOException {
+    //nothing to do;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
new file mode 100644
index 0000000..b906791
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.common.blockaliasmap;
+
+/**
+ * The AliasMap defines mapping of PROVIDED HDFS blocks to data in remote
+ * storage systems.
+ */
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 0f1407a..835d8c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4630,26 +4630,6 @@
   </property>
 
   <property>
-    <name>dfs.namenode.block.provider.class</name>
-    <value>org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider</value>
-    <description>
-      The class that is used to load provided blocks in the Namenode.
-    </description>
-  </property>
-
-  <property>
-    <name>dfs.provider.class</name>
-    <value>org.apache.hadoop.hdfs.server.common.TextFileRegionProvider</value>
-    <description>
-      The class that is used to load information about blocks stored in
-      provided storages.
-      org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TextFileRegionProvider
-      is used as the default, which expects the blocks to be specified
-      using a delimited text file.
-    </description>
-  </property>
-
-  <property>
     <name>dfs.provided.df.class</name>
     <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.DefaultProvidedVolumeDF</value>
     <description>
@@ -4666,12 +4646,12 @@
   </property>
 
   <property>
-    <name>dfs.provided.blockformat.class</name>
-    <value>org.apache.hadoop.hdfs.server.common.TextFileRegionFormat</value>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap</value>
     <description>
       The class that is used to specify the input format of the blocks on
       provided storages. The default is
-      org.apache.hadoop.hdfs.server.common.TextFileRegionFormat which uses
+      org.apache.hadoop.hdfs.server.common.TextFileRegionAliasMap which uses
       file regions to describe blocks. The file regions are specified as a
       delimited text file. Each file region is a 6-tuple containing the
       block id, remote file path, offset into file, length of block, the
@@ -4681,7 +4661,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.delimiter</name>
+    <name>dfs.provided.aliasmap.text.delimiter</name>
     <value>,</value>
     <description>
         The delimiter used when the provided block map is specified as
@@ -4690,7 +4670,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.read.path</name>
+    <name>dfs.provided.aliasmap.text.read.path</name>
     <value></value>
     <description>
         The path specifying the provided block map as a text file, specified as
@@ -4699,7 +4679,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.read.codec</name>
+    <name>dfs.provided.aliasmap.text.codec</name>
     <value></value>
     <description>
         The codec used to de-compress the provided block map.
@@ -4707,7 +4687,7 @@
   </property>
 
   <property>
-    <name>dfs.provided.textprovider.write.path</name>
+    <name>dfs.provided.aliasmap.text.write.path</name>
     <value></value>
     <description>
         The path to which the provided block map should be written as a text

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
deleted file mode 100644
index eaaac22..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestTextBlockFormat.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.common;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStreamWriter;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.server.common.TextFileRegionFormat.*;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.compress.CompressionCodec;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-/**
- * Test for the text based block format for provided block maps.
- */
-public class TestTextBlockFormat {
-
-  static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
-
-  void check(TextWriter.Options opts, final Path vp,
-      final Class<? extends CompressionCodec> vc) throws IOException {
-    TextFileRegionFormat mFmt = new TextFileRegionFormat() {
-      @Override
-      public TextWriter createWriter(Path file, CompressionCodec codec,
-          String delim, Configuration conf) throws IOException {
-        assertEquals(vp, file);
-        if (null == vc) {
-          assertNull(codec);
-        } else {
-          assertEquals(vc, codec.getClass());
-        }
-        return null; // ignored
-      }
-    };
-    mFmt.getWriter(opts);
-  }
-
-  @Test
-  public void testWriterOptions() throws Exception {
-    TextWriter.Options opts = TextWriter.defaults();
-    assertTrue(opts instanceof WriterOptions);
-    WriterOptions wopts = (WriterOptions) opts;
-    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_BLOCK_MAP_PATH_DEFAULT);
-    assertEquals(def, wopts.getFile());
-    assertNull(wopts.getCodec());
-
-    opts.filename(OUTFILE);
-    check(opts, OUTFILE, null);
-
-    opts.filename(OUTFILE);
-    opts.codec("gzip");
-    Path cp = new Path(OUTFILE.getParent(), OUTFILE.getName() + ".gz");
-    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
-
-  }
-
-  @Test
-  public void testCSVReadWrite() throws Exception {
-    final DataOutputBuffer out = new DataOutputBuffer();
-    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
-    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
-    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
-    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), ",")) {
-      csv.store(r1);
-      csv.store(r2);
-      csv.store(r3);
-    }
-    Iterator<FileRegion> i3;
-    try (TextReader csv = new TextReader(null, null, null, ",") {
-      @Override
-      public InputStream createStream() {
-        DataInputBuffer in = new DataInputBuffer();
-        in.reset(out.getData(), 0, out.getLength());
-        return in;
-        }}) {
-      Iterator<FileRegion> i1 = csv.iterator();
-      assertEquals(r1, i1.next());
-      Iterator<FileRegion> i2 = csv.iterator();
-      assertEquals(r1, i2.next());
-      assertEquals(r2, i2.next());
-      assertEquals(r3, i2.next());
-      assertEquals(r2, i1.next());
-      assertEquals(r3, i1.next());
-
-      assertFalse(i1.hasNext());
-      assertFalse(i2.hasNext());
-      i3 = csv.iterator();
-    }
-    try {
-      i3.next();
-    } catch (IllegalStateException e) {
-      return;
-    }
-    fail("Invalid iterator");
-  }
-
-  @Test
-  public void testCSVReadWriteTsv() throws Exception {
-    final DataOutputBuffer out = new DataOutputBuffer();
-    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
-    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
-    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
-    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), "\t")) {
-      csv.store(r1);
-      csv.store(r2);
-      csv.store(r3);
-    }
-    Iterator<FileRegion> i3;
-    try (TextReader csv = new TextReader(null, null, null, "\t") {
-      @Override
-      public InputStream createStream() {
-        DataInputBuffer in = new DataInputBuffer();
-        in.reset(out.getData(), 0, out.getLength());
-        return in;
-      }}) {
-      Iterator<FileRegion> i1 = csv.iterator();
-      assertEquals(r1, i1.next());
-      Iterator<FileRegion> i2 = csv.iterator();
-      assertEquals(r1, i2.next());
-      assertEquals(r2, i2.next());
-      assertEquals(r3, i2.next());
-      assertEquals(r2, i1.next());
-      assertEquals(r3, i1.next());
-
-      assertFalse(i1.hasNext());
-      assertFalse(i2.hasNext());
-      i3 = csv.iterator();
-    }
-    try {
-      i3.next();
-    } catch (IllegalStateException e) {
-      return;
-    }
-    fail("Invalid iterator");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ae7ea55/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
new file mode 100644
index 0000000..79308a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.common.blockaliasmap.impl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.*;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test for the text based block format for provided block maps.
+ */
+public class TestTextBlockAliasMap {
+
+  static final Path OUTFILE = new Path("hdfs://dummyServer:0000/dummyFile.txt");
+
+  void check(TextWriter.Options opts, final Path vp,
+      final Class<? extends CompressionCodec> vc) throws IOException {
+    TextFileRegionAliasMap mFmt = new TextFileRegionAliasMap() {
+      @Override
+      public TextWriter createWriter(Path file, CompressionCodec codec,
+          String delim, Configuration conf) throws IOException {
+        assertEquals(vp, file);
+        if (null == vc) {
+          assertNull(codec);
+        } else {
+          assertEquals(vc, codec.getClass());
+        }
+        return null; // ignored
+      }
+    };
+    mFmt.getWriter(opts);
+  }
+
+  @Test
+  public void testWriterOptions() throws Exception {
+    TextWriter.Options opts = TextWriter.defaults();
+    assertTrue(opts instanceof WriterOptions);
+    WriterOptions wopts = (WriterOptions) opts;
+    Path def = new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_PATH_DEFAULT);
+    assertEquals(def, wopts.getFile());
+    assertNull(wopts.getCodec());
+
+    opts.filename(OUTFILE);
+    check(opts, OUTFILE, null);
+
+    opts.filename(OUTFILE);
+    opts.codec("gzip");
+    Path cp = new Path(OUTFILE.getParent(), OUTFILE.getName() + ".gz");
+    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
+
+  }
+
+  @Test
+  public void testCSVReadWrite() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), ",")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, ",") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+        }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+  @Test
+  public void testCSVReadWriteTsv() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), "\t")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, "\t") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+      }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+}


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