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 in...@apache.org on 2017/09/06 16:50:01 UTC

[01/29] hadoop git commit: HDFS-11882. Precisely calculate acked length of striped block groups in updatePipeline. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10467 32a642eaf -> 3f7df14a2 (forced update)


HDFS-11882. Precisely calculate acked length of striped block groups in updatePipeline.


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

Branch: refs/heads/HDFS-10467
Commit: ccd2ac60ecc5fccce56debf21a068e663c1d5f11
Parents: 0ba8ff4
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 5 14:16:03 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 5 14:16:03 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 160 +++++++++++++++++--
 .../TestDFSStripedOutputStreamWithFailure.java  |  77 ++++++++-
 2 files changed, 222 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd2ac60/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index d5206d1..408b325 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -641,7 +641,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       // wait till all the healthy streamers to
       // 1) get the updated block info
       // 2) create new block outputstream
-      newFailed = waitCreatingNewStreams(healthySet);
+      newFailed = waitCreatingStreamers(healthySet);
       if (newFailed.size() + failedStreamers.size() >
           numAllBlocks - numDataBlocks) {
         throw new IOException(
@@ -668,6 +668,14 @@ public class DFSStripedOutputStream extends DFSOutputStream
     }
   }
 
+  /**
+   * Check if the streamers were successfully updated, adding failed streamers
+   * in the <i>failed</i> return parameter.
+   * @param failed Return parameter containing failed streamers from
+   *               <i>streamers</i>.
+   * @param streamers Set of streamers that are being updated
+   * @return total number of successful updates and failures
+   */
   private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
       Set<StripedDataStreamer> streamers) {
     for (StripedDataStreamer streamer : streamers) {
@@ -682,7 +690,15 @@ public class DFSStripedOutputStream extends DFSOutputStream
     return coordinator.updateStreamerMap.size() + failed.size();
   }
 
-  private Set<StripedDataStreamer> waitCreatingNewStreams(
+  /**
+   * Waits for streamers to be created.
+   *
+   * @param healthyStreamers Set of healthy streamers
+   * @return Set of streamers that failed.
+   *
+   * @throws IOException
+   */
+  private Set<StripedDataStreamer> waitCreatingStreamers(
       Set<StripedDataStreamer> healthyStreamers) throws IOException {
     Set<StripedDataStreamer> failed = new HashSet<>();
     final int expectedNum = healthyStreamers.size();
@@ -773,9 +789,10 @@ public class DFSStripedOutputStream extends DFSOutputStream
       }
     }
 
-    // should update the block group length based on the acked length
+    // Update the NameNode with the acked length of the block group
+    // Save and restore the unacked length
     final long sentBytes = currentBlockGroup.getNumBytes();
-    final long ackedBytes = getNumAckedStripes() * cellSize * numDataBlocks;
+    final long ackedBytes = getAckedLength();
     Preconditions.checkState(ackedBytes <= sentBytes,
         "Acked:" + ackedBytes + ", Sent:" + sentBytes);
     currentBlockGroup.setNumBytes(ackedBytes);
@@ -787,23 +804,140 @@ public class DFSStripedOutputStream extends DFSOutputStream
   }
 
   /**
-   * Get the number of acked stripes. An acked stripe means at least data block
-   * number size cells of the stripe were acked.
+   * Return the length of each block in the block group.
+   * Unhealthy blocks have a length of -1.
+   *
+   * @return List of block lengths.
    */
-  private long getNumAckedStripes() {
-    int minStripeNum = Integer.MAX_VALUE;
+  private List<Long> getBlockLengths() {
+    List<Long> blockLengths = new ArrayList<>(numAllBlocks);
     for (int i = 0; i < numAllBlocks; i++) {
       final StripedDataStreamer streamer = getStripedDataStreamer(i);
+      long numBytes = -1;
       if (streamer.isHealthy()) {
-        int curStripeNum = 0;
         if (streamer.getBlock() != null) {
-          curStripeNum = (int) (streamer.getBlock().getNumBytes() / cellSize);
+          numBytes = streamer.getBlock().getNumBytes();
         }
-        minStripeNum = Math.min(curStripeNum, minStripeNum);
+      }
+      blockLengths.add(numBytes);
+    }
+    return blockLengths;
+  }
+
+  /**
+   * Get the length of acked bytes in the block group.
+   *
+   * <p>
+   *   A full stripe is acked when at least numDataBlocks streamers have
+   *   the corresponding cells of the stripe, and all previous full stripes are
+   *   also acked. This enforces the constraint that there is at most one
+   *   partial stripe.
+   * </p>
+   * <p>
+   *   Partial stripes write all parity cells. Empty data cells are not written.
+   *   Parity cells are the length of the longest data cell(s). For example,
+   *   with RS(3,2), if we have data cells with lengths [1MB, 64KB, 0], the
+   *   parity blocks will be length [1MB, 1MB].
+   * </p>
+   * <p>
+   *   To be considered acked, a partial stripe needs at least numDataBlocks
+   *   empty or written cells.
+   * </p>
+   * <p>
+   *   Currently, partial stripes can only happen when closing the file at a
+   *   non-stripe boundary, but this could also happen during (currently
+   *   unimplemented) hflush/hsync support.
+   * </p>
+   */
+  private long getAckedLength() {
+    // Determine the number of full stripes that are sufficiently durable
+    final long sentBytes = currentBlockGroup.getNumBytes();
+    final long numFullStripes = sentBytes / numDataBlocks / cellSize;
+    final long fullStripeLength = numFullStripes * numDataBlocks * cellSize;
+    assert fullStripeLength <= sentBytes : "Full stripe length can't be " +
+        "greater than the block group length";
+
+    long ackedLength = 0;
+
+    // Determine the length contained by at least `numDataBlocks` blocks.
+    // Since it's sorted, all the blocks after `offset` are at least as long,
+    // and there are at least `numDataBlocks` at or after `offset`.
+    List<Long> blockLengths = Collections.unmodifiableList(getBlockLengths());
+    List<Long> sortedBlockLengths = new ArrayList<>(blockLengths);
+    Collections.sort(sortedBlockLengths);
+    if (numFullStripes > 0) {
+      final int offset = sortedBlockLengths.size() - numDataBlocks;
+      ackedLength = sortedBlockLengths.get(offset) * numDataBlocks;
+    }
+
+    // If the acked length is less than the expected full stripe length, then
+    // we're missing a full stripe. Return the acked length.
+    if (ackedLength < fullStripeLength) {
+      return ackedLength;
+    }
+    // If the expected length is exactly a stripe boundary, then we're also done
+    if (ackedLength == sentBytes) {
+      return ackedLength;
+    }
+
+    /*
+    Otherwise, we're potentially dealing with a partial stripe.
+    The partial stripe is laid out as follows:
+
+      0 or more full data cells, `cellSize` in length.
+      0 or 1 partial data cells.
+      0 or more empty data cells.
+      `numParityBlocks` parity cells, the length of the longest data cell.
+
+    If the partial stripe is sufficiently acked, we'll update the ackedLength.
+    */
+
+    // How many full and empty data cells do we expect?
+    final int numFullDataCells = (int)
+        ((sentBytes - fullStripeLength) / cellSize);
+    final int partialLength = (int) (sentBytes - fullStripeLength) % cellSize;
+    final int numPartialDataCells = partialLength == 0 ? 0 : 1;
+    final int numEmptyDataCells = numDataBlocks - numFullDataCells -
+        numPartialDataCells;
+    // Calculate the expected length of the parity blocks.
+    final int parityLength = numFullDataCells > 0 ? cellSize : partialLength;
+
+    final long fullStripeBlockOffset = fullStripeLength / numDataBlocks;
+
+    // Iterate through each type of streamers, checking the expected length.
+    long[] expectedBlockLengths = new long[numAllBlocks];
+    int idx = 0;
+    // Full cells
+    for (; idx < numFullDataCells; idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset + cellSize;
+    }
+    // Partial cell
+    for (; idx < numFullDataCells + numPartialDataCells; idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset + partialLength;
+    }
+    // Empty cells
+    for (; idx < numFullDataCells + numPartialDataCells + numEmptyDataCells;
+         idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset;
+    }
+    // Parity cells
+    for (; idx < numAllBlocks; idx++) {
+      expectedBlockLengths[idx] = fullStripeBlockOffset + parityLength;
+    }
+
+    // Check expected lengths against actual streamer lengths.
+    // Update if we have sufficient durability.
+    int numBlocksWithCorrectLength = 0;
+    for (int i = 0; i < numAllBlocks; i++) {
+      if (blockLengths.get(i) == expectedBlockLengths[i]) {
+        numBlocksWithCorrectLength++;
       }
     }
-    assert minStripeNum != Integer.MAX_VALUE;
-    return minStripeNum;
+    if (numBlocksWithCorrectLength >= numDataBlocks) {
+      ackedLength = sentBytes;
+    }
+
+    return ackedLength;
   }
 
   private int stripeDataSize() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccd2ac60/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 9915a2f..f63a353 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -61,6 +60,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 /**
  * Test striped file write operation with data node failures.
@@ -391,6 +391,79 @@ public class TestDFSStripedOutputStreamWithFailure {
   }
 
   /**
+   * When the two DataNodes with partial data blocks fail.
+   */
+  @Test
+  public void runTestWithDifferentLengths() throws Exception {
+    assumeTrue("Skip this test case in the subclasses. Once is enough.",
+        this.getClass().equals(TestDFSStripedOutputStreamWithFailure.class));
+
+    final HdfsConfiguration conf = newHdfsConfiguration();
+
+    final int[] fileLengths = {
+        // Full stripe then partial on cell boundary
+        cellSize * (dataBlocks * 2 - 2),
+        // Full stripe and a partial on non-cell boundary
+        (cellSize * dataBlocks) + 123,
+    };
+    try {
+      for (int length: fileLengths) {
+        // select the two DNs with partial block to kill
+        final int[] dnIndex = {dataBlocks - 2, dataBlocks - 1};
+        final int[] killPos = getKillPositions(length, dnIndex.length);
+        try {
+          LOG.info("runTestWithMultipleFailure2: length==" + length
+              + ", killPos=" + Arrays.toString(killPos)
+              + ", dnIndex=" + Arrays.toString(dnIndex));
+          setup(conf);
+          runTest(length, killPos, dnIndex, false);
+        } catch (Throwable e) {
+          final String err = "failed, killPos=" + Arrays.toString(killPos)
+              + ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
+          LOG.error(err);
+          throw e;
+        }
+      }
+    } finally {
+      tearDown();
+    }
+  }
+
+  /**
+   * Test writing very short EC files with many failures.
+   */
+  @Test
+  public void runTestWithShortStripe() throws Exception {
+    assumeTrue("Skip this test case in the subclasses. Once is enough.",
+        this.getClass().equals(TestDFSStripedOutputStreamWithFailure.class));
+
+    final HdfsConfiguration conf = newHdfsConfiguration();
+    // Write a file with a 1 cell partial stripe
+    final int length = cellSize - 123;
+    // Kill all but one DN
+    final int[] dnIndex = new int[dataBlocks + parityBlocks - 1];
+    for (int i = 0; i < dnIndex.length; i++) {
+      dnIndex[i] = i;
+    }
+    final int[] killPos = getKillPositions(length, dnIndex.length);
+
+    try {
+      LOG.info("runTestWithShortStripe: length==" + length + ", killPos="
+          + Arrays.toString(killPos) + ", dnIndex="
+          + Arrays.toString(dnIndex));
+      setup(conf);
+      runTest(length, killPos, dnIndex, false);
+    } catch (Throwable e) {
+      final String err = "failed, killPos=" + Arrays.toString(killPos)
+          + ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
+      LOG.error(err);
+      throw e;
+    } finally {
+      tearDown();
+    }
+  }
+
+  /**
    * runTest implementation.
    * @param length file length
    * @param killPos killing positions in ascending order
@@ -558,7 +631,7 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   private void run(int offset) {
     int base = getBase();
-    Assume.assumeTrue(base >= 0);
+    assumeTrue(base >= 0);
     final int i = offset + base;
     final Integer length = getLength(i);
     if (length == null) {


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


[05/29] hadoop git commit: HADOOP-14103. Sort out hadoop-aws contract-test-options.xml. Contributed by John Zhuge.

Posted by in...@apache.org.
HADOOP-14103. Sort out hadoop-aws contract-test-options.xml. Contributed by John Zhuge.


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

Branch: refs/heads/HDFS-10467
Commit: 50506e90a843aaf1f771a32b872fdd7704b619f5
Parents: d7f2704
Author: John Zhuge <jz...@apache.org>
Authored: Tue Sep 5 23:26:31 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Tue Sep 5 23:26:57 2017 -0700

----------------------------------------------------------------------
 .../site/markdown/tools/hadoop-aws/testing.md   | 129 +++----------------
 1 file changed, 18 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50506e90/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
index 3b9b5c4..19d322d 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
@@ -76,13 +76,10 @@ over long-haul connections. Please help us identify and fix these problems
 
 ## Setting up the tests
 
-To integration test the S3* filesystem clients, you need to provide two files
-which pass in authentication details to the test runner.
+To integration test the S3* filesystem clients, you need to provide
+`auth-keys.xml` which passes in authentication details to the test runner.
 
-1. `auth-keys.xml`
-1. `contract-test-options.xml`
-
-These are both Hadoop XML configuration files, which must be placed into
+It is a Hadoop XML configuration file, which must be placed into
 `hadoop-tools/hadoop-aws/src/test/resources`.
 
 ### File `core-site.xml`
@@ -106,6 +103,8 @@ each filesystem for its testing.
 
 1. `test.fs.s3n.name` : the URL of the bucket for S3n tests
 1. `test.fs.s3a.name` : the URL of the bucket for S3a tests
+1. `fs.contract.test.fs.s3n` : the URL of the bucket for S3n filesystem contract tests
+1. `fs.contract.test.fs.s3a` : the URL of the bucket for S3a filesystem contract tests
 
 *Note* that running s3a and s3n tests in parallel mode, against the same bucket
 is unreliable.  We recommend using separate buckets or testing one connector
@@ -128,8 +127,8 @@ Example:
   </property>
 
   <property>
-    <name>test.fs.s3a.name</name>
-    <value>s3a://test-aws-s3a/</value>
+    <name>fs.contract.test.fs.s3n</name>
+    <value>${test.fs.s3n.name}</value>
   </property>
 
   <property>
@@ -143,6 +142,16 @@ Example:
   </property>
 
   <property>
+    <name>test.fs.s3a.name</name>
+    <value>s3a://test-aws-s3a/</value>
+  </property>
+
+  <property>
+    <name>fs.contract.test.fs.s3a</name>
+    <value>${test.fs.s3a.name}</value>
+  </property>
+
+  <property>
     <name>fs.s3a.access.key</name>
     <description>AWS access key ID. Omit for IAM role-based authentication.</description>
     <value>DONOTCOMMITTHISKEYTOSCM</value>
@@ -163,107 +172,6 @@ Example:
 </configuration>
 ```
 
-### File `contract-test-options.xml`
-
-The file `hadoop-tools/hadoop-aws/src/test/resources/contract-test-options.xml`
-must be created and configured for the test filesystems.
-
-If a specific file `fs.contract.test.fs.*` test path is not defined for
-any of the filesystems, those tests will be skipped.
-
-The standard S3 authentication details must also be provided. This can be
-through copy-and-paste of the `auth-keys.xml` credentials, or it can be
-through direct XInclude inclusion.
-
-Here is an an example `contract-test-options.xml` which places all test options
-into the `auth-keys.xml` file, so offering a single place to keep credentials
-and define test endpoint bindings.
-
-```xml
-<configuration>
-  <include xmlns="http://www.w3.org/2001/XInclude"
-    href="auth-keys.xml"/>
-</configuration>
-```
-
-### s3n://
-
-
-In the file `src/test/resources/contract-test-options.xml`, the filesystem
-name must be defined in the property `fs.contract.test.fs.s3n`.
-The standard configuration options to define the S3N authentication details
-must also be provided.
-
-Example:
-
-```xml
-<property>
-  <name>fs.contract.test.fs.s3n</name>
-  <value>s3n://test-aws-s3n/</value>
-</property>
-```
-
-### s3a://
-
-
-In the file `src/test/resources/contract-test-options.xml`, the filesystem
-name must be defined in the property `fs.contract.test.fs.s3a`.
-The standard configuration options to define the S3N authentication details
-must also be provided.
-
-Example:
-
-```xml
-<property>
-  <name>fs.contract.test.fs.s3a</name>
-  <value>s3a://test-aws-s3a/</value>
-</property>
-```
-
-### Complete example of `contract-test-options.xml`
-
-
-```xml
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!---
-  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.
--->
-
-<configuration>
-
-  <include xmlns="http://www.w3.org/2001/XInclude"
-  href="auth-keys.xml"/>
-
-  <property>
-    <name>fs.contract.test.fs.s3a</name>
-    <value>s3a://test-aws-s3a/</value>
-  </property>
-
-  <property>
-    <name>fs.contract.test.fs.s3n</name>
-    <value>s3n://test-aws-s3n/</value>
-  </property>
-
-</configuration>
-```
-
-This example pulls in the `auth-keys.xml` file for the credentials.
-This provides one single place to keep the keys up to date —and means
-that the file `contract-test-options.xml` does not contain any
-secret credentials itself. As the auth keys XML file is kept out of the
-source code tree, it is not going to get accidentally committed.
-
 ### Configuring S3a Encryption
 
 For S3a encryption tests to run correctly, the
@@ -346,8 +254,7 @@ like `ITestS3A*` shown above, it may cause unpredictable test failures.
 ### Testing against different regions
 
 S3A can connect to different regions —the tests support this. Simply
-define the target region in `contract-test-options.xml` or any `auth-keys.xml`
-file referenced.
+define the target region in `auth-keys.xml`.
 
 ```xml
 <property>


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


[14/29] hadoop git commit: HDFS-11554. [Documentation] Router-based federation documentation. Contributed by Inigo Goiri.

Posted by in...@apache.org.
HDFS-11554. [Documentation] Router-based federation documentation. Contributed by Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 20203eb98098d53bb563ddbe967ebb90ea8abced
Parents: 400b8e9
Author: Inigo Goiri <in...@apache.org>
Authored: Wed Aug 16 17:23:29 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../src/site/markdown/HDFSRouterFederation.md   | 170 +++++++++++++++++++
 .../site/resources/images/routerfederation.png  | Bin 0 -> 24961 bytes
 2 files changed, 170 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/20203eb9/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
new file mode 100644
index 0000000..f094238
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
@@ -0,0 +1,170 @@
+<!---
+  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 Router-based Federation
+============================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Introduction
+------------
+
+NameNodes have scalability limits because of the metadata overhead comprised of inodes (files and directories) and file blocks, the number of Datanode heartbeats, and the number of HDFS RPC client requests.
+The common solution is to split the filesystem into smaller subclusters [HDFS Federation](.Federation.html) and provide a federated view [ViewFs](.ViewFs.html).
+The problem is how to maintain the split of the subclusters (e.g., namespace partition), which forces users to connect to multiple subclusters and manage the allocation of folders/files to them.
+
+
+Architecture
+------------
+
+A natural extension to this partitioned federation is to add a layer of software responsible for federating the namespaces.
+This extra layer allows users to access any subcluster transparently, lets subclusters manage their own block pools independently, and supports rebalancing of data across subclusters.
+To accomplish these goals, the federation layer directs block accesses to the proper subcluster, maintains the state of the namespaces, and provides mechanisms for data rebalancing.
+This layer must be scalable, highly available, and fault tolerant.
+
+This federation layer comprises multiple components.
+The _Router_ component that has the same interface as a NameNode, and forwards the client requests to the correct subcluster, based on ground-truth information from a State Store.
+The _State Store_ combines a remote _Mount Table_ (in the flavor of [ViewFs](.ViewFs.html), but shared between clients) and utilization (load/capacity) information about the subclusters.
+This approach has the same architecture as [YARN federation](../hadoop-yarn/Federation.html).
+
+![Router-based Federation Sequence Diagram | width=800](./images/routerfederation.png)
+
+
+### Example flow
+The simplest configuration deploys a Router on each NameNode machine.
+The Router monitors the local NameNode and heartbeats the state to the State Store.
+When a regular DFS client contacts any of the Routers to access a file in the federated filesystem, the Router checks the Mount Table in the State Store (i.e., the local cache) to find out which subcluster contains the file.
+Then it checks the Membership table in the State Store (i.e., the local cache) for the NameNode responsible for the subcluster.
+After it has identified the correct NameNode, the Router proxies the request.
+The client accesses Datanodes directly.
+
+
+### Router
+There can be multiple Routers in the system with soft state.
+Each Router has two roles:
+
+* Federated interface: expose a single, global NameNode interface to the clients and forward the requests to the active NameNode in the correct subcluster
+* NameNode heartbeat: maintain the information about a NameNode in the State Store
+
+#### Federated interface
+The Router receives a client request, checks the State Store for the correct subcluster, and forwards the request to the active NameNode of that subcluster.
+The reply from the NameNode then flows in the opposite direction.
+The Routers are stateless and can be behind a load balancer.
+For performance, the Router also caches remote mount table entries and the state of the subclusters.
+To make sure that changes have been propagated to all Routers, each Router heartbeats its state to the State Store.
+
+The communications between the Routers and the State Store are cached (with timed expiration for freshness).
+This improves the performance of the system.
+
+#### NameNode heartbeat
+For this role, the Router periodically checks the state of a NameNode (usually on the same server) and reports their high availability (HA) state and load/space status to the State Store.
+Note that this is an optional role, as a Router can be independent of any subcluster.
+For performance with NameNode HA, the Router uses the high availability state information in the State Store to forward the request to the NameNode that is most likely to be active.
+Note that this service can be embedded into the NameNode itself to simplify the operation.
+
+#### Availability and fault tolerance
+The Router operates with failures at multiple levels.
+
+* **Federated interface HA:**
+The Routers are stateless and metadata operations are atomic at the NameNodes.
+If a Router becomes unavailable, any Router can take over for it.
+The clients configure their DFS HA client (e.g., ConfiguredFailoverProvider or RequestHedgingProxyProvider) with all the Routers in the federation as endpoints.
+
+* **NameNode heartbeat HA:**
+For high availability and flexibility, multiple Routers can monitor the same NameNode and heartbeat the information to the State Store.
+This increases clients' resiliency to stale information, should a Router fail.
+Conflicting NameNode information in the State Store is resolved by each Router via a quorum.
+
+* **Unavailable NameNodes:**
+If a Router cannot contact the active NameNode, then it will try the other NameNodes in the subcluster.
+It will first try those reported as standby and then the unavailable ones.
+If the Router cannot reach any NameNode, then it throws an exception.
+
+* **Expired NameNodes:**
+If a NameNode heartbeat has not been recorded in the State Store for a multiple of the heartbeat interval, the monitoring Router will record that the NameNode has expired and no Routers will attempt to access it.
+If an updated heartbeat is subsequently recorded for the NameNode, the monitoring Router will restore the NameNode from the expired state.
+
+#### Interfaces
+To interact with the users and the administrators, the Router exposes multiple interfaces.
+
+* **RPC:**
+The Router RPC implements the most common interfaces clients use to interact with HDFS.
+The current implementation has been tested using analytics workloads written in plain MapReduce, Spark, and Hive (on Tez, Spark, and MapReduce).
+Advanced functions like snapshotting, encryption and tiered storage are left for future versions.
+All unimplemented functions will throw exceptions.
+
+* **Admin:**
+Adminstrators can query information from clusters and add/remove entries from the mount table over RPC.
+This interface is also exposed through the command line to get and modify information from the federation.
+
+* **Web UI:**
+The Router exposes a Web UI visualizing the state of the federation, mimicking the current NameNode UI.
+It displays information about the mount table, membership information about each subcluster, and the status of the Routers.
+
+* **WebHDFS:**
+The Router provides the HDFS REST interface (WebHDFS) in addition to the RPC one.
+
+* **JMX:**
+It exposes metrics through JMX mimicking the NameNode.
+This is used by the Web UI to get the cluster status.
+
+Some operations are not available in Router-based federation.
+The Router throws exceptions for those.
+Examples users may encounter include the following.
+
+* Rename file/folder in two different nameservices.
+* Copy file/folder in two different nameservices.
+* Write into a file/folder being rebalanced.
+
+
+### State Store
+The (logically centralized, but physically distributed) State Store maintains:
+
+* The state of the subclusters in terms of their block access load, available disk space, HA state, etc.
+* The mapping between folder/files and subclusters, i.e. the remote mount table.
+
+The backend of the State Store is pluggable.
+We leverage the fault tolerance of the backend implementations.
+The main information stored in the State Store and its implementation:
+
+* **Membership**:
+The membership information encodes the state of the NameNodes in the federation.
+This includes information about the subcluster, such as storage capacity and the number of nodes.
+The Router periodically heartbeats this information about one or more NameNodes.
+Given that multiple Routers can monitor a single NameNode, the heartbeat from every Router is stored.
+The Routers apply a quorum of the data when querying this information from the State Store.
+The Routers discard the entries older than a certain threshold (e.g., ten Router heartbeat periods).
+
+* **Mount Table**:
+This table hosts the mapping between folders and subclusters.
+It is similar to the mount table in [ViewFs](.ViewFs.html) where it specifies the federated folder, the destination subcluster and the path in that folder.
+
+
+Deployment
+----------
+
+By default, the Router is ready to take requests and monitor the NameNode in the local machine.
+It needs to know the State Store endpoint by setting `dfs.federation.router.store.driver.class`.
+The rest of the options are documented in [hdfs-default.xml](./hdfs-default.xml).
+
+Once the Router is configured, it can be started:
+
+    [hdfs]$ $HADOOP_HOME/bin/hdfs router
+
+To manage the mount table:
+
+    [hdfs]$ $HADOOP_HOME/bin/hdfs federation -add /tmp DC1 /tmp
+    [hdfs]$ $HADOOP_HOME/bin/hdfs federation -add /data/wl1 DC2 /data/wl1
+    [hdfs]$ $HADOOP_HOME/bin/hdfs federation -add /data/wl2 DC3 /data/wl2
+    [hdfs]$ $HADOOP_HOME/bin/hdfs federation -ls

http://git-wip-us.apache.org/repos/asf/hadoop/blob/20203eb9/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/routerfederation.png
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/routerfederation.png b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/routerfederation.png
new file mode 100644
index 0000000..2b158bb
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/routerfederation.png differ


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


[21/29] hadoop git commit: HDFS-10631. Federation State Store ZooKeeper implementation. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10631. Federation State Store ZooKeeper implementation. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 3f7df14a208d35c4075146e53a3f5956c39a5238
Parents: 4007ebb
Author: Inigo Goiri <in...@apache.org>
Authored: Mon Aug 21 11:40:41 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   9 +
 .../driver/impl/StateStoreSerializableImpl.java |  19 ++
 .../driver/impl/StateStoreZooKeeperImpl.java    | 298 +++++++++++++++++++
 .../store/driver/TestStateStoreDriverBase.java  |   2 +-
 .../store/driver/TestStateStoreZK.java          | 105 +++++++
 5 files changed, 432 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f7df14a/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 93216db..d22d6ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -203,6 +203,15 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-framework</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f7df14a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
index e9b3fdf..e2038fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
@@ -30,6 +30,11 @@ import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
  */
 public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
 
+  /** Mark for slashes in path names. */
+  protected static final String SLASH_MARK = "0SLASH0";
+  /** Mark for colon in path names. */
+  protected static final String COLON_MARK = "_";
+
   /** Default serializer for this driver. */
   private StateStoreSerializer serializer;
 
@@ -74,4 +79,18 @@ public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
       String data, Class<T> clazz, boolean includeDates) throws IOException {
     return serializer.deserialize(data, clazz);
   }
+
+  /**
+   * Get the primary key for a record. If we don't want to store in folders, we
+   * need to remove / from the name.
+   *
+   * @param record Record to get the primary key for.
+   * @return Primary key for the record.
+   */
+  protected static String getPrimaryKey(BaseRecord record) {
+    String primaryKey = record.getPrimaryKey();
+    primaryKey = primaryKey.replaceAll("/", SLASH_MARK);
+    primaryKey = primaryKey.replaceAll(":", COLON_MARK);
+    return primaryKey;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f7df14a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java
new file mode 100644
index 0000000..ddcd537
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java
@@ -0,0 +1,298 @@
+/**
+ * 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.store.driver.impl;
+
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.filterMultiple;
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.getRecordName;
+import static org.apache.hadoop.util.curator.ZKCuratorManager.getNodePath;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link StateStoreDriver} driver implementation that uses ZooKeeper as a
+ * backend.
+ * <p>
+ * The structure of the znodes in the ensemble is:
+ * PARENT_PATH
+ * |--- MOUNT
+ * |--- MEMBERSHIP
+ * |--- REBALANCER
+ * |--- ROUTERS
+ */
+public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreZooKeeperImpl.class);
+
+
+  /** Configuration keys. */
+  public static final String FEDERATION_STORE_ZK_DRIVER_PREFIX =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "driver.zk.";
+  public static final String FEDERATION_STORE_ZK_PARENT_PATH =
+      FEDERATION_STORE_ZK_DRIVER_PREFIX + "parent-path";
+  public static final String FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT =
+      "/hdfs-federation";
+
+
+  /** Directory to store the state store data. */
+  private String baseZNode;
+
+  /** Interface to ZooKeeper. */
+  private ZKCuratorManager zkManager;
+
+
+  @Override
+  public boolean initDriver() {
+    LOG.info("Initializing ZooKeeper connection");
+
+    Configuration conf = getConf();
+    baseZNode = conf.get(
+        FEDERATION_STORE_ZK_PARENT_PATH,
+        FEDERATION_STORE_ZK_PARENT_PATH_DEFAULT);
+    try {
+      this.zkManager = new ZKCuratorManager(conf);
+      this.zkManager.start();
+    } catch (IOException e) {
+      LOG.error("Cannot initialize the ZK connection", e);
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean initRecordStorage(
+      String className, Class<T> clazz) {
+    try {
+      String checkPath = getNodePath(baseZNode, className);
+      zkManager.createRootDirRecursively(checkPath);
+      return true;
+    } catch (Exception e) {
+      LOG.error("Cannot initialize ZK node for {}: {}",
+          className, e.getMessage());
+      return false;
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (zkManager  != null) {
+      zkManager.close();
+    }
+  }
+
+  @Override
+  public boolean isDriverReady() {
+    return zkManager != null;
+  }
+
+  @Override
+  public <T extends BaseRecord> QueryResult<T> get(Class<T> clazz)
+      throws IOException {
+    return get(clazz, (String)null);
+  }
+
+  @Override
+  public <T extends BaseRecord> QueryResult<T> get(Class<T> clazz, String sub)
+      throws IOException {
+    verifyDriverReady();
+    List<T> ret = new ArrayList<>();
+    String znode = getZNodeForClass(clazz);
+    try {
+      List<String> children = zkManager.getChildren(znode);
+      for (String child : children) {
+        try {
+          String path = getNodePath(znode, child);
+          Stat stat = new Stat();
+          String data = zkManager.getStringData(path, stat);
+          boolean corrupted = false;
+          if (data == null || data.equals("")) {
+            // All records should have data, otherwise this is corrupted
+            corrupted = true;
+          } else {
+            try {
+              T record = createRecord(data, stat, clazz);
+              ret.add(record);
+            } catch (IOException e) {
+              LOG.error("Cannot create record type \"{}\" from \"{}\": {}",
+                  clazz.getSimpleName(), data, e.getMessage());
+              corrupted = true;
+            }
+          }
+
+          if (corrupted) {
+            LOG.error("Cannot get data for {} at {}, cleaning corrupted data",
+                child, path);
+            zkManager.delete(path);
+          }
+        } catch (Exception e) {
+          LOG.error("Cannot get data for {}: {}", child, e.getMessage());
+        }
+      }
+    } catch (Exception e) {
+      String msg = "Cannot get children for \"" + znode + "\": " +
+          e.getMessage();
+      LOG.error(msg);
+      throw new IOException(msg);
+    }
+    return new QueryResult<T>(ret, getTime());
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean putAll(
+      List<T> records, boolean update, boolean error) throws IOException {
+    verifyDriverReady();
+    if (records.isEmpty()) {
+      return true;
+    }
+
+    // All records should be the same
+    T record0 = records.get(0);
+    Class<? extends BaseRecord> recordClass = record0.getClass();
+    String znode = getZNodeForClass(recordClass);
+
+    boolean status = true;
+    for (T record : records) {
+      String primaryKey = getPrimaryKey(record);
+      String recordZNode = getNodePath(znode, primaryKey);
+      byte[] data = serialize(record);
+      if (!writeNode(recordZNode, data, update, error)){
+        status = false;
+      }
+    }
+    return status;
+  }
+
+  @Override
+  public <T extends BaseRecord> int remove(
+      Class<T> clazz, Query<T> query) throws IOException {
+    verifyDriverReady();
+    if (query == null) {
+      return 0;
+    }
+
+    // Read the current data
+    List<T> records = null;
+    try {
+      QueryResult<T> result = get(clazz);
+      records = result.getRecords();
+    } catch (IOException ex) {
+      LOG.error("Cannot get existing records", ex);
+      return 0;
+    }
+
+    // Check the records to remove
+    String znode = getZNodeForClass(clazz);
+    List<T> recordsToRemove = filterMultiple(query, records);
+
+    // Remove the records
+    int removed = 0;
+    for (T existingRecord : recordsToRemove) {
+      LOG.info("Removing \"{}\"", existingRecord);
+      try {
+        String primaryKey = getPrimaryKey(existingRecord);
+        String path = getNodePath(znode, primaryKey);
+        if (zkManager.delete(path)) {
+          removed++;
+        } else {
+          LOG.error("Did not remove \"{}\"", existingRecord);
+        }
+      } catch (Exception e) {
+        LOG.error("Cannot remove \"{}\"", existingRecord, e);
+      }
+    }
+    return removed;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean removeAll(Class<T> clazz)
+      throws IOException {
+    boolean status = true;
+    String znode = getZNodeForClass(clazz);
+    LOG.info("Deleting all children under {}", znode);
+    try {
+      List<String> children = zkManager.getChildren(znode);
+      for (String child : children) {
+        String path = getNodePath(znode, child);
+        LOG.info("Deleting {}", path);
+        zkManager.delete(path);
+      }
+    } catch (Exception e) {
+      LOG.error("Cannot remove {}: {}", znode, e.getMessage());
+      status = false;
+    }
+    return status;
+  }
+
+  private boolean writeNode(
+      String znode, byte[] bytes, boolean update, boolean error) {
+    try {
+      boolean created = zkManager.create(znode);
+      if (!update && !created && error) {
+        LOG.info("Cannot write record \"{}\", it already exists", znode);
+        return false;
+      }
+
+      // Write data
+      zkManager.setData(znode, bytes, -1);
+      return true;
+    } catch (Exception e) {
+      LOG.error("Cannot write record \"{}\": {}", znode, e.getMessage());
+    }
+    return false;
+  }
+
+  /**
+   * Get the ZNode for a class.
+   *
+   * @param clazz Record class to evaluate.
+   * @return The ZNode for the class.
+   */
+  private <T extends BaseRecord> String getZNodeForClass(Class<T> clazz) {
+    String className = getRecordName(clazz);
+    return getNodePath(baseZNode, className);
+  }
+
+  /**
+   * Creates a record from a string returned by ZooKeeper.
+   *
+   * @param source Object from ZooKeeper.
+   * @param clazz The data record type to create.
+   * @return The created record.
+   * @throws IOException
+   */
+  private <T extends BaseRecord> T createRecord(
+      String data, Stat stat, Class<T> clazz) throws IOException {
+    T record = newRecord(data, clazz, false);
+    record.setDateCreated(stat.getCtime());
+    record.setDateModified(stat.getMtime());
+    return record;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f7df14a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
index 8239fb1..65e763b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
@@ -89,7 +89,7 @@ public class TestStateStoreDriverBase {
   }
 
   private String generateRandomString() {
-    String randomString = "/randomString-" + RANDOM.nextInt();
+    String randomString = "randomString-" + RANDOM.nextInt();
     return randomString;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f7df14a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.java
new file mode 100644
index 0000000..36353ff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreZK.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.federation.store.driver;
+
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreZooKeeperImpl;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the ZooKeeper implementation of the State Store driver.
+ */
+public class TestStateStoreZK extends TestStateStoreDriverBase {
+
+  private static TestingServer curatorTestingServer;
+  private static CuratorFramework curatorFramework;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    curatorTestingServer = new TestingServer();
+    curatorTestingServer.start();
+    String connectString = curatorTestingServer.getConnectString();
+    curatorFramework = CuratorFrameworkFactory.builder()
+        .connectString(connectString)
+        .retryPolicy(new RetryNTimes(100, 100))
+        .build();
+    curatorFramework.start();
+
+    // Create the ZK State Store
+    Configuration conf =
+        getStateStoreConfiguration(StateStoreZooKeeperImpl.class);
+    conf.set(CommonConfigurationKeys.ZK_ADDRESS, connectString);
+    // Disable auto-repair of connection
+    conf.setLong(DFSConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS,
+        TimeUnit.HOURS.toMillis(1));
+    getStateStore(conf);
+  }
+
+  @AfterClass
+  public static void tearDownCluster() {
+    curatorFramework.close();
+    try {
+      curatorTestingServer.stop();
+    } catch (IOException e) {
+    }
+  }
+
+  @Before
+  public void startup() throws IOException {
+    removeAll(getStateStoreDriver());
+  }
+
+  @Test
+  public void testInsert()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testUpdate()
+      throws IllegalArgumentException, ReflectiveOperationException,
+      IOException, SecurityException {
+    testPut(getStateStoreDriver());
+  }
+
+  @Test
+  public void testDelete()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testRemove(getStateStoreDriver());
+  }
+
+  @Test
+  public void testFetchErrors()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testFetchErrors(getStateStoreDriver());
+  }
+}
\ 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


[13/29] hadoop git commit: HDFS-10687. Federation Membership State Store internal API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10687. Federation Membership State Store internal API. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 79fce5e0cfb11fe5cf4bc9bae72bf7f601afbf60
Parents: 4d1180e
Author: Inigo Goiri <in...@apache.org>
Authored: Mon Jul 31 10:55:21 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   3 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   1 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  17 +-
 .../resolver/MembershipNamenodeResolver.java    | 290 ++++++++++++
 .../federation/router/FederationUtil.java       |  42 +-
 .../federation/store/CachedRecordStore.java     | 237 ++++++++++
 .../federation/store/MembershipStore.java       | 126 +++++
 .../federation/store/StateStoreCache.java       |  36 ++
 .../store/StateStoreCacheUpdateService.java     |  67 +++
 .../federation/store/StateStoreService.java     | 202 +++++++-
 .../store/impl/MembershipStoreImpl.java         | 311 +++++++++++++
 .../federation/store/impl/package-info.java     |  31 ++
 .../GetNamenodeRegistrationsRequest.java        |  52 +++
 .../GetNamenodeRegistrationsResponse.java       |  55 +++
 .../store/protocol/GetNamespaceInfoRequest.java |  30 ++
 .../protocol/GetNamespaceInfoResponse.java      |  52 +++
 .../protocol/NamenodeHeartbeatRequest.java      |  52 +++
 .../protocol/NamenodeHeartbeatResponse.java     |  49 ++
 .../UpdateNamenodeRegistrationRequest.java      |  72 +++
 .../UpdateNamenodeRegistrationResponse.java     |  51 ++
 .../impl/pb/FederationProtocolPBTranslator.java | 145 ++++++
 .../GetNamenodeRegistrationsRequestPBImpl.java  |  87 ++++
 .../GetNamenodeRegistrationsResponsePBImpl.java |  99 ++++
 .../impl/pb/GetNamespaceInfoRequestPBImpl.java  |  60 +++
 .../impl/pb/GetNamespaceInfoResponsePBImpl.java |  95 ++++
 .../impl/pb/NamenodeHeartbeatRequestPBImpl.java |  93 ++++
 .../pb/NamenodeHeartbeatResponsePBImpl.java     |  71 +++
 ...UpdateNamenodeRegistrationRequestPBImpl.java |  95 ++++
 ...pdateNamenodeRegistrationResponsePBImpl.java |  73 +++
 .../store/protocol/impl/pb/package-info.java    |  29 ++
 .../store/records/MembershipState.java          | 329 +++++++++++++
 .../store/records/MembershipStats.java          | 126 +++++
 .../records/impl/pb/MembershipStatePBImpl.java  | 334 +++++++++++++
 .../records/impl/pb/MembershipStatsPBImpl.java  | 191 ++++++++
 .../src/main/proto/FederationProtocol.proto     | 107 +++++
 .../src/main/resources/hdfs-default.xml         |  18 +-
 .../resolver/TestNamenodeResolver.java          | 284 ++++++++++++
 .../store/FederationStateStoreTestUtils.java    |  23 +-
 .../federation/store/TestStateStoreBase.java    |  81 ++++
 .../store/TestStateStoreMembershipState.java    | 463 +++++++++++++++++++
 .../store/driver/TestStateStoreDriverBase.java  |  69 ++-
 .../store/records/TestMembershipState.java      | 129 ++++++
 42 files changed, 4745 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 9582fcb..4b958b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -15,6 +15,9 @@
        <Package name="org.apache.hadoop.hdfs.qjournal.protocol" />
      </Match>
      <Match>
+       <Package name="org.apache.hadoop.hdfs.federation.protocol.proto" />
+     </Match>
+     <Match>
        <Bug pattern="EI_EXPOSE_REP" />
      </Match>
      <Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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 425572f..cc7a975 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -331,6 +331,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>QJournalProtocol.proto</include>
                   <include>editlog.proto</include>
                   <include>fsimage.proto</include>
+                  <include>FederationProtocol.proto</include>
                 </includes>
               </source>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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 8b39e88..afb5bbf 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
@@ -27,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
 import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl;
@@ -1163,8 +1165,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "org.apache.hadoop.hdfs.server.federation.MockResolver";
   public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS =
       FEDERATION_ROUTER_PREFIX + "namenode.resolver.client.class";
-  public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
-      "org.apache.hadoop.hdfs.server.federation.MockResolver";
+  public static final Class<? extends ActiveNamenodeResolver>
+      FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
+          MembershipNamenodeResolver.class;
 
   // HDFS Router-based federation State Store
   public static final String FEDERATION_STORE_PREFIX =
@@ -1186,6 +1189,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long FEDERATION_STORE_CONNECTION_TEST_MS_DEFAULT =
       TimeUnit.MINUTES.toMillis(1);
 
+  public static final String DFS_ROUTER_CACHE_TIME_TO_LIVE_MS =
+      FEDERATION_ROUTER_PREFIX + "cache.ttl";
+  public static final long DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+
+  public static final String FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS =
+      FEDERATION_STORE_PREFIX + "membership.expiration";
+  public static final long FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(5);
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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
new file mode 100644
index 0000000..b0ced24
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
@@ -0,0 +1,290 @@
+/**
+ * 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.resolver;
+
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.ACTIVE;
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.EXPIRED;
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.UNAVAILABLE;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreCache;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements a cached lookup of the most recently active namenode for a
+ * particular nameservice. Relies on the {@link StateStoreService} to
+ * discover available nameservices and namenodes.
+ */
+public class MembershipNamenodeResolver
+    implements ActiveNamenodeResolver, StateStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MembershipNamenodeResolver.class);
+
+  /** Reference to the State Store. */
+  private final StateStoreService stateStore;
+  /** Membership State Store interface. */
+  private final MembershipStore membershipInterface;
+
+  /** Parent router ID. */
+  private String routerId;
+
+  /** Cached lookup of NN for nameservice. Invalidated on cache refresh. */
+  private Map<String, List<? extends FederationNamenodeContext>> cacheNS;
+  /** Cached lookup of NN for block pool. Invalidated on cache refresh. */
+  private Map<String, List<? extends FederationNamenodeContext>> cacheBP;
+
+
+  public MembershipNamenodeResolver(
+      Configuration conf, StateStoreService store) throws IOException {
+    this.stateStore = store;
+
+    this.cacheNS = new ConcurrentHashMap<>();
+    this.cacheBP = new ConcurrentHashMap<>();
+
+    if (this.stateStore != null) {
+      // Request cache updates from the state store
+      this.stateStore.registerCacheExternal(this);
+
+      // Initialize the interface to get the membership
+      this.membershipInterface = this.stateStore.getRegisteredRecordStore(
+          MembershipStore.class);
+    } else {
+      this.membershipInterface = null;
+    }
+
+    if (this.membershipInterface == null) {
+      throw new IOException("State Store does not have an interface for " +
+          MembershipStore.class.getSimpleName());
+    }
+  }
+
+  @Override
+  public boolean loadCache(boolean force) {
+    // Our cache depends on the store, update it first
+    try {
+      this.membershipInterface.loadCache(force);
+    } catch (IOException e) {
+      LOG.error("Cannot update membership from the State Store", e);
+    }
+
+    // Force refresh of active NN cache
+    cacheBP.clear();
+    cacheNS.clear();
+    return true;
+  }
+
+  @Override
+  public void updateActiveNamenode(
+      final String nsId, final InetSocketAddress address) throws IOException {
+
+    // Called when we have an RPC miss and successful hit on an alternate NN.
+    // Temporarily update our cache, it will be overwritten on the next update.
+    try {
+      MembershipState partial = MembershipState.newInstance();
+      String rpcAddress = address.getHostName() + ":" + address.getPort();
+      partial.setRpcAddress(rpcAddress);
+      partial.setNameserviceId(nsId);
+
+      GetNamenodeRegistrationsRequest request =
+          GetNamenodeRegistrationsRequest.newInstance(partial);
+
+      GetNamenodeRegistrationsResponse response =
+          this.membershipInterface.getNamenodeRegistrations(request);
+      List<MembershipState> records = response.getNamenodeMemberships();
+
+      if (records != null && records.size() == 1) {
+        MembershipState record = records.get(0);
+        UpdateNamenodeRegistrationRequest updateRequest =
+            UpdateNamenodeRegistrationRequest.newInstance(
+                record.getNameserviceId(), record.getNamenodeId(), ACTIVE);
+        this.membershipInterface.updateNamenodeRegistration(updateRequest);
+      }
+    } catch (StateStoreUnavailableException e) {
+      LOG.error("Cannot update {} as active, State Store unavailable", address);
+    }
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext> getNamenodesForNameserviceId(
+      final String nsId) throws IOException {
+
+    List<? extends FederationNamenodeContext> ret = cacheNS.get(nsId);
+    if (ret == null) {
+      try {
+        MembershipState partial = MembershipState.newInstance();
+        partial.setNameserviceId(nsId);
+        GetNamenodeRegistrationsRequest request =
+            GetNamenodeRegistrationsRequest.newInstance(partial);
+
+        final List<MembershipState> result =
+            getRecentRegistrationForQuery(request, true, false);
+        if (result == null || result.isEmpty()) {
+          LOG.error("Cannot locate eligible NNs for {}", nsId);
+          return null;
+        } else {
+          cacheNS.put(nsId, result);
+          ret = result;
+        }
+      } catch (StateStoreUnavailableException e) {
+        LOG.error("Cannot get active NN for {}, State Store unavailable", nsId);
+      }
+    }
+    if (ret == null) {
+      return null;
+    }
+    return Collections.unmodifiableList(ret);
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
+      final String bpId) throws IOException {
+
+    List<? extends FederationNamenodeContext> ret = cacheBP.get(bpId);
+    if (ret == null) {
+      try {
+        MembershipState partial = MembershipState.newInstance();
+        partial.setBlockPoolId(bpId);
+        GetNamenodeRegistrationsRequest request =
+            GetNamenodeRegistrationsRequest.newInstance(partial);
+
+        final List<MembershipState> result =
+            getRecentRegistrationForQuery(request, true, false);
+        if (result == null || result.isEmpty()) {
+          LOG.error("Cannot locate eligible NNs for {}", bpId);
+        } else {
+          cacheBP.put(bpId, result);
+          ret = result;
+        }
+      } catch (StateStoreUnavailableException e) {
+        LOG.error("Cannot get active NN for {}, State Store unavailable", bpId);
+        return null;
+      }
+    }
+    if (ret == null) {
+      return null;
+    }
+    return Collections.unmodifiableList(ret);
+  }
+
+  @Override
+  public boolean registerNamenode(NamenodeStatusReport report)
+      throws IOException {
+
+    if (this.routerId == null) {
+      LOG.warn("Cannot register namenode, router ID is not known {}", report);
+      return false;
+    }
+
+    MembershipState record = MembershipState.newInstance(
+        routerId, report.getNameserviceId(), report.getNamenodeId(),
+        report.getClusterId(), report.getBlockPoolId(), report.getRpcAddress(),
+        report.getServiceAddress(), report.getLifelineAddress(),
+        report.getWebAddress(), report.getState(), report.getSafemode());
+
+    if (report.getState() != UNAVAILABLE) {
+      // Set/update our last contact time
+      record.setLastContact(Time.now());
+    }
+
+    NamenodeHeartbeatRequest request = NamenodeHeartbeatRequest.newInstance();
+    request.setNamenodeMembership(record);
+    return this.membershipInterface.namenodeHeartbeat(request).getResult();
+  }
+
+  @Override
+  public Set<FederationNamespaceInfo> getNamespaces() throws IOException {
+    GetNamespaceInfoRequest request = GetNamespaceInfoRequest.newInstance();
+    GetNamespaceInfoResponse response =
+        this.membershipInterface.getNamespaceInfo(request);
+    return response.getNamespaceInfo();
+  }
+
+  /**
+   * Picks the most relevant record registration that matches the query. Return
+   * registrations matching the query in this preference: 1) Most recently
+   * updated ACTIVE registration 2) Most recently updated STANDBY registration
+   * (if showStandby) 3) Most recently updated UNAVAILABLE registration (if
+   * showUnavailable). EXPIRED registrations are ignored.
+   *
+   * @param query The select query for NN registrations.
+   * @param excludes List of NNs to exclude from matching results.
+   * @param addUnavailable include UNAVAILABLE registrations.
+   * @param addExpired include EXPIRED registrations.
+   * @return List of memberships or null if no registrations that
+   *         both match the query AND the selected states.
+   * @throws IOException
+   */
+  private List<MembershipState> getRecentRegistrationForQuery(
+      GetNamenodeRegistrationsRequest request, boolean addUnavailable,
+      boolean addExpired) throws IOException {
+
+    // Retrieve a list of all registrations that match this query.
+    // This may include all NN records for a namespace/blockpool, including
+    // duplicate records for the same NN from different routers.
+    GetNamenodeRegistrationsResponse response =
+        this.membershipInterface.getNamenodeRegistrations(request);
+
+    List<MembershipState> memberships = response.getNamenodeMemberships();
+    if (!addExpired || !addUnavailable) {
+      Iterator<MembershipState> iterator = memberships.iterator();
+      while (iterator.hasNext()) {
+        MembershipState membership = iterator.next();
+        if (membership.getState() == EXPIRED && !addExpired) {
+          iterator.remove();
+        } else if (membership.getState() == UNAVAILABLE && !addUnavailable) {
+          iterator.remove();
+        }
+      }
+    }
+
+    List<MembershipState> priorityList = new ArrayList<>();
+    priorityList.addAll(memberships);
+    Collections.sort(priorityList, new NamenodePriorityComparator());
+
+    LOG.debug("Selected most recent NN {} for query", priorityList);
+    return priorityList;
+  }
+
+  @Override
+  public void setRouterId(String router) {
+    this.routerId = router;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
index 6e7e865..0129a37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
@@ -19,26 +19,57 @@ package org.apache.hadoop.hdfs.server.federation.router;
 
 import java.lang.reflect.Constructor;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utilities for managing HDFS federation.
  */
 public final class FederationUtil {
 
-  private static final Log LOG = LogFactory.getLog(FederationUtil.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationUtil.class);
 
   private FederationUtil() {
     // Utility Class
   }
 
   /**
+   * Create an instance of an interface with a constructor using a context.
+   *
+   * @param conf Configuration for the class names.
+   * @param context Context object to pass to the instance.
+   * @param contextClass Type of the context passed to the constructor.
+   * @param clazz Class of the object to return.
+   * @return New instance of the specified class that implements the desired
+   *         interface and a single parameter constructor containing a
+   *         StateStore reference.
+   */
+  private static <T, R> T newInstance(final Configuration conf,
+      final R context, final Class<R> contextClass, final Class<T> clazz) {
+    try {
+      if (contextClass == null) {
+        // Default constructor if no context
+        Constructor<T> constructor = clazz.getConstructor();
+        return constructor.newInstance();
+      } else {
+        // Constructor with context
+        Constructor<T> constructor = clazz.getConstructor(
+            Configuration.class, contextClass);
+        return constructor.newInstance(conf, context);
+      }
+    } catch (ReflectiveOperationException e) {
+      LOG.error("Could not instantiate: {}", clazz.getSimpleName(), e);
+      return null;
+    }
+  }
+
+  /**
    * Create an instance of an interface with a constructor using a state store
    * constructor.
    *
@@ -105,13 +136,14 @@ public final class FederationUtil {
    *
    * @param conf Configuration that defines the namenode resolver class.
    * @param obj Context object passed to class constructor.
-   * @return ActiveNamenodeResolver
+   * @return New active namenode resolver.
    */
   public static ActiveNamenodeResolver newActiveNamenodeResolver(
       Configuration conf, StateStoreService stateStore) {
-    return newInstance(conf, stateStore, StateStoreService.class,
+    Class<? extends ActiveNamenodeResolver> clazz = conf.getClass(
         DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
         DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT,
         ActiveNamenodeResolver.class);
+    return newInstance(conf, stateStore, StateStoreService.class, clazz);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java
new file mode 100644
index 0000000..90a6699
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java
@@ -0,0 +1,237 @@
+/**
+ * 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.store;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Record store that takes care of caching the records in memory.
+ *
+ * @param <R> Record to store by this interface.
+ */
+public abstract class CachedRecordStore<R extends BaseRecord>
+    extends RecordStore<R> implements StateStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CachedRecordStore.class);
+
+
+  /** Prevent loading the cache more than once every 500 ms. */
+  private static final long MIN_UPDATE_MS = 500;
+
+
+  /** Cached entries. */
+  private List<R> records = new ArrayList<>();
+
+  /** Time stamp of the cached entries. */
+  private long timestamp = -1;
+
+  /** If the cache is initialized. */
+  private boolean initialized = false;
+
+  /** Last time the cache was updated. */
+  private long lastUpdate = -1;
+
+  /** Lock to access the memory cache. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  /** If it should override the expired values when loading the cache. */
+  private boolean override = false;
+
+
+  /**
+   * Create a new cached record store.
+   *
+   * @param clazz Class of the record to store.
+   * @param driver State Store driver.
+   */
+  protected CachedRecordStore(Class<R> clazz, StateStoreDriver driver) {
+    this(clazz, driver, false);
+  }
+
+  /**
+   * Create a new cached record store.
+   *
+   * @param clazz Class of the record to store.
+   * @param driver State Store driver.
+   * @param override If the entries should be override if they expire
+   */
+  protected CachedRecordStore(
+      Class<R> clazz, StateStoreDriver driver, boolean over) {
+    super(clazz, driver);
+
+    this.override = over;
+  }
+
+  /**
+   * Check that the cache of the State Store information is available.
+   *
+   * @throws StateStoreUnavailableException If the cache is not initialized.
+   */
+  private void checkCacheAvailable() throws StateStoreUnavailableException {
+    if (!this.initialized) {
+      throw new StateStoreUnavailableException(
+          "Cached State Store not initialized, " +
+          getRecordClass().getSimpleName() + " records not valid");
+    }
+  }
+
+  @Override
+  public boolean loadCache(boolean force) throws IOException {
+    // Prevent loading the cache too frequently
+    if (force || isUpdateTime()) {
+      List<R> newRecords = null;
+      long t = -1;
+      try {
+        QueryResult<R> result = getDriver().get(getRecordClass());
+        newRecords = result.getRecords();
+        t = result.getTimestamp();
+
+        // If we have any expired record, update the State Store
+        if (this.override) {
+          overrideExpiredRecords(result);
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot get \"{}\" records from the State Store",
+            getRecordClass().getSimpleName());
+        this.initialized = false;
+        return false;
+      }
+
+      // Update cache atomically
+      writeLock.lock();
+      try {
+        this.records.clear();
+        this.records.addAll(newRecords);
+        this.timestamp = t;
+        this.initialized = true;
+      } finally {
+        writeLock.unlock();
+      }
+
+      lastUpdate = Time.monotonicNow();
+    }
+    return true;
+  }
+
+  /**
+   * Check if it's time to update the cache. Update it it was never updated.
+   *
+   * @return If it's time to update this cache.
+   */
+  private boolean isUpdateTime() {
+    return Time.monotonicNow() - lastUpdate > MIN_UPDATE_MS;
+  }
+
+  /**
+   * Updates the state store with any record overrides we detected, such as an
+   * expired state.
+   *
+   * @param query RecordQueryResult containing the data to be inspected.
+   * @param clazz Type of objects contained in the query.
+   * @throws IOException
+   */
+  public void overrideExpiredRecords(QueryResult<R> query) throws IOException {
+    List<R> commitRecords = new ArrayList<>();
+    List<R> newRecords = query.getRecords();
+    long currentDriverTime = query.getTimestamp();
+    if (newRecords == null || currentDriverTime <= 0) {
+      LOG.error("Cannot check overrides for record");
+      return;
+    }
+    for (R record : newRecords) {
+      if (record.checkExpired(currentDriverTime)) {
+        String recordName = StateStoreUtils.getRecordName(record.getClass());
+        LOG.info("Override State Store record {}: {}", recordName, record);
+        commitRecords.add(record);
+      }
+    }
+    if (commitRecords.size() > 0) {
+      getDriver().putAll(commitRecords, true, false);
+    }
+  }
+
+  /**
+   * Updates the state store with any record overrides we detected, such as an
+   * expired state.
+   *
+   * @param driver State store driver for the data store.
+   * @param record Record record to be updated.
+   * @param clazz Type of data record.
+   * @throws IOException
+   */
+  public void overrideExpiredRecord(R record) throws IOException {
+    List<R> newRecords = Collections.singletonList(record);
+    long time = getDriver().getTime();
+    QueryResult<R> query = new QueryResult<>(newRecords, time);
+    overrideExpiredRecords(query);
+  }
+
+  /**
+   * Get all the cached records.
+   *
+   * @return Copy of the cached records.
+   * @throws StateStoreUnavailableException If the State store is not available.
+   */
+  public List<R> getCachedRecords() throws StateStoreUnavailableException {
+    checkCacheAvailable();
+
+    List<R> ret = new LinkedList<R>();
+    this.readLock.lock();
+    try {
+      ret.addAll(this.records);
+    } finally {
+      this.readLock.unlock();
+    }
+    return ret;
+  }
+
+  /**
+   * Get all the cached records and the time stamp of the cache.
+   *
+   * @return Copy of the cached records and the time stamp.
+   * @throws StateStoreUnavailableException If the State store is not available.
+   */
+  protected QueryResult<R> getCachedRecordsAndTimeStamp()
+      throws StateStoreUnavailableException {
+    checkCacheAvailable();
+
+    this.readLock.lock();
+    try {
+      return new QueryResult<R>(this.records, this.timestamp);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MembershipStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MembershipStore.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MembershipStore.java
new file mode 100644
index 0000000..3e8ba6b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MembershipStore.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.federation.store;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+
+/**
+ * Management API for NameNode registrations stored in
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.MembershipState
+ * MembershipState} records. The {@link org.apache.hadoop.hdfs.server.
+ * federation.router.RouterHeartbeatService RouterHeartbeatService} periodically
+ * polls each NN to update the NameNode metadata(addresses, operational) and HA
+ * state(active, standby). Each NameNode may be polled by multiple
+ * {@link org.apache.hadoop.hdfs.server.federation.router.Router Router}
+ * instances.
+ * <p>
+ * Once fetched from the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver}, NameNode registrations are cached until the next query.
+ * The fetched registration data is aggregated using a quorum to determine the
+ * best/most accurate state for each NameNode. The cache is periodically updated
+ * by the @{link StateStoreCacheUpdateService}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class MembershipStore
+    extends CachedRecordStore<MembershipState> {
+
+  protected MembershipStore(StateStoreDriver driver) {
+    super(MembershipState.class, driver, true);
+  }
+
+  /**
+   * Inserts or updates a namenode membership entry into the table.
+   *
+   * @param request Fully populated NamenodeHeartbeatRequest request.
+   * @return True if successful, false otherwise.
+   * @throws StateStoreUnavailableException Throws exception if the data store
+   *           is not initialized.
+   * @throws IOException if the data store could not be queried or the query is
+   *           invalid.
+   */
+  public abstract NamenodeHeartbeatResponse namenodeHeartbeat(
+      NamenodeHeartbeatRequest request) throws IOException;
+
+  /**
+   * Queries for a single cached registration entry matching the given
+   * parameters. Possible keys are the names of data structure elements Possible
+   * values are matching SQL "LIKE" targets.
+   *
+   * @param request Fully populated GetNamenodeRegistrationsRequest request.
+   * @return Single matching FederationMembershipStateEntry or null if not found
+   *         or more than one entry matches.
+   * @throws StateStoreUnavailableException Throws exception if the data store
+   *           is not initialized.
+   * @throws IOException if the data store could not be queried or the query is
+   *           invalid.
+   */
+  public abstract GetNamenodeRegistrationsResponse getNamenodeRegistrations(
+      GetNamenodeRegistrationsRequest request) throws IOException;
+
+  /**
+   * Get the expired registrations from the registration cache.
+   *
+   * @return Expired registrations or zero-length list if none are found.
+   * @throws StateStoreUnavailableException Throws exception if the data store
+   *           is not initialized.
+   * @throws IOException if the data store could not be queried or the query is
+   *           invalid.
+   */
+  public abstract GetNamenodeRegistrationsResponse
+      getExpiredNamenodeRegistrations(GetNamenodeRegistrationsRequest request)
+          throws IOException;
+
+  /**
+   * Retrieves a list of registered nameservices and their associated info.
+   *
+   * @param request
+   * @return Collection of information for each registered nameservice.
+   * @throws IOException if the data store could not be queried or the query is
+   *           invalid.
+   */
+  public abstract GetNamespaceInfoResponse getNamespaceInfo(
+      GetNamespaceInfoRequest request) throws IOException;
+
+  /**
+   * Overrides a cached namenode state with an updated state.
+   *
+   * @param request Fully populated OverrideNamenodeRegistrationRequest request.
+   * @return OverrideNamenodeRegistrationResponse
+   * @throws StateStoreUnavailableException if the data store is not
+   *           initialized.
+   * @throws IOException if the data store could not be queried or the query is
+   *           invalid.
+   */
+  public abstract UpdateNamenodeRegistrationResponse updateNamenodeRegistration(
+      UpdateNamenodeRegistrationRequest request) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCache.java
new file mode 100644
index 0000000..83fc501
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCache.java
@@ -0,0 +1,36 @@
+/**
+ * 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.store;
+
+import java.io.IOException;
+
+/**
+ * Interface for a cached copy of the State Store.
+ */
+public interface StateStoreCache {
+
+  /**
+   * Load the cache from the State Store. Called by the cache update service
+   * when the data has been reloaded.
+   *
+   * @param force If we force the load.
+   * @return If the cache was loaded successfully.
+   * @throws IOException If there was an error loading the cache.
+   */
+  boolean loadCache(boolean force) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
new file mode 100644
index 0000000..bb8cfb0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreCacheUpdateService.java
@@ -0,0 +1,67 @@
+/**
+ * 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.store;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.router.PeriodicService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service to periodically update the {@link StateStoreService}
+ * cached information in the
+ * {@link org.apache.hadoop.hdfs.server.federation.router.Router Router}.
+ * This is for performance and removes the State Store from the critical path
+ * in common operations.
+ */
+public class StateStoreCacheUpdateService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreCacheUpdateService.class);
+
+  /** The service that manages the State Store connection. */
+  private final StateStoreService stateStore;
+
+
+  /**
+   * Create a new Cache update service.
+   *
+   * @param stateStore Implementation of the state store
+   */
+  public StateStoreCacheUpdateService(StateStoreService stateStore) {
+    super(StateStoreCacheUpdateService.class.getSimpleName());
+    this.stateStore = stateStore;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+
+    this.setIntervalMs(conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS,
+        DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT));
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    LOG.debug("Updating State Store cache");
+    stateStore.refreshCaches();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
index df207e0..73f607f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
@@ -18,17 +18,24 @@
 package org.apache.hadoop.hdfs.server.federation.store;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.impl.MembershipStoreImpl;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,6 +61,9 @@ import com.google.common.annotations.VisibleForTesting;
  * federation.
  * <li>{@link MountTableStore}: Mount table between to subclusters.
  * See {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
+ * <li>{@link RebalancerStore}: Log of the rebalancing operations.
+ * <li>{@link RouterStore}: Router state in the federation.
+ * <li>{@link TokenStore}: Tokens in the federation.
  * </ul>
  */
 @InterfaceAudience.Private
@@ -77,8 +87,30 @@ public class StateStoreService extends CompositeService {
   private StateStoreConnectionMonitorService monitorService;
 
 
+  /** Supported record stores. */
+  private final Map<
+      Class<? extends BaseRecord>, RecordStore<? extends BaseRecord>>
+          recordStores;
+
+  /** Service to maintain State Store caches. */
+  private StateStoreCacheUpdateService cacheUpdater;
+  /** Time the cache was last successfully updated. */
+  private long cacheLastUpdateTime;
+  /** List of internal caches to update. */
+  private final List<StateStoreCache> cachesToUpdateInternal;
+  /** List of external caches to update. */
+  private final List<StateStoreCache> cachesToUpdateExternal;
+
+
   public StateStoreService() {
     super(StateStoreService.class.getName());
+
+    // Records and stores supported by this implementation
+    this.recordStores = new HashMap<>();
+
+    // Caches to maintain
+    this.cachesToUpdateInternal = new ArrayList<>();
+    this.cachesToUpdateExternal = new ArrayList<>();
   }
 
   /**
@@ -102,10 +134,22 @@ public class StateStoreService extends CompositeService {
       throw new IOException("Cannot create driver for the State Store");
     }
 
+    // Add supported record stores
+    addRecordStore(MembershipStoreImpl.class);
+
     // Check the connection to the State Store periodically
     this.monitorService = new StateStoreConnectionMonitorService(this);
     this.addService(monitorService);
 
+    // Set expirations intervals for each record
+    MembershipState.setExpirationMs(conf.getLong(
+        DFSConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS,
+        DFSConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT));
+
+    // Cache update service
+    this.cacheUpdater = new StateStoreCacheUpdateService(this);
+    addService(this.cacheUpdater);
+
     super.serviceInit(this.conf);
   }
 
@@ -123,13 +167,56 @@ public class StateStoreService extends CompositeService {
   }
 
   /**
+   * Add a record store to the State Store. It includes adding the store, the
+   * supported record and the cache management.
+   *
+   * @param clazz Class of the record store to track.
+   * @return New record store.
+   * @throws ReflectiveOperationException
+   */
+  private <T extends RecordStore<?>> void addRecordStore(
+      final Class<T> clazz) throws ReflectiveOperationException {
+
+    assert this.getServiceState() == STATE.INITED :
+        "Cannot add record to the State Store once started";
+
+    T recordStore = RecordStore.newInstance(clazz, this.getDriver());
+    Class<? extends BaseRecord> recordClass = recordStore.getRecordClass();
+    this.recordStores.put(recordClass, recordStore);
+
+    // Subscribe for cache updates
+    if (recordStore instanceof StateStoreCache) {
+      StateStoreCache cachedRecordStore = (StateStoreCache) recordStore;
+      this.cachesToUpdateInternal.add(cachedRecordStore);
+    }
+  }
+
+  /**
+   * Get the record store in this State Store for a given interface.
+   *
+   * @param recordStoreClass Class of the record store.
+   * @return Registered record store or null if not found.
+   */
+  public <T extends RecordStore<?>> T getRegisteredRecordStore(
+      final Class<T> recordStoreClass) {
+    for (RecordStore<? extends BaseRecord> recordStore :
+        this.recordStores.values()) {
+      if (recordStoreClass.isInstance(recordStore)) {
+        @SuppressWarnings("unchecked")
+        T recordStoreChecked = (T) recordStore;
+        return recordStoreChecked;
+      }
+    }
+    return null;
+  }
+
+  /**
    * List of records supported by this State Store.
    *
    * @return List of supported record classes.
    */
   public Collection<Class<? extends BaseRecord>> getSupportedRecords() {
-    // TODO add list of records
-    return new LinkedList<>();
+    return this.recordStores.keySet();
   }
 
   /**
@@ -142,6 +229,7 @@ public class StateStoreService extends CompositeService {
         if (this.driver.init(conf, getIdentifier(), getSupportedRecords())) {
           LOG.info("Connection to the State Store driver {} is open and ready",
               driverName);
+          this.refreshCaches();
         } else {
           LOG.error("Cannot initialize State Store driver {}", driverName);
         }
@@ -198,4 +286,114 @@ public class StateStoreService extends CompositeService {
     this.identifier = id;
   }
 
+  //
+  // Cached state store data
+  //
+  /**
+   * The last time the state store cache was fully updated.
+   *
+   * @return Timestamp.
+   */
+  public long getCacheUpdateTime() {
+    return this.cacheLastUpdateTime;
+  }
+
+  /**
+   * Stops the cache update service.
+   */
+  @VisibleForTesting
+  public void stopCacheUpdateService() {
+    if (this.cacheUpdater != null) {
+      this.cacheUpdater.stop();
+      removeService(this.cacheUpdater);
+      this.cacheUpdater = null;
+    }
+  }
+
+  /**
+   * Register a cached record store for automatic periodic cache updates.
+   *
+   * @param client Client to the state store.
+   */
+  public void registerCacheExternal(StateStoreCache client) {
+    this.cachesToUpdateExternal.add(client);
+  }
+
+  /**
+   * Refresh the cache with information from the State Store. Called
+   * periodically by the CacheUpdateService to maintain data caches and
+   * versions.
+   */
+  public void refreshCaches() {
+    refreshCaches(false);
+  }
+
+  /**
+   * Refresh the cache with information from the State Store. Called
+   * periodically by the CacheUpdateService to maintain data caches and
+   * versions.
+   * @param force If we force the refresh.
+   */
+  public void refreshCaches(boolean force) {
+    boolean success = true;
+    if (isDriverReady()) {
+      List<StateStoreCache> cachesToUpdate = new LinkedList<>();
+      cachesToUpdate.addAll(cachesToUpdateInternal);
+      cachesToUpdate.addAll(cachesToUpdateExternal);
+      for (StateStoreCache cachedStore : cachesToUpdate) {
+        String cacheName = cachedStore.getClass().getSimpleName();
+        boolean result = false;
+        try {
+          result = cachedStore.loadCache(force);
+        } catch (IOException e) {
+          LOG.error("Error updating cache for {}", cacheName, e);
+          result = false;
+        }
+        if (!result) {
+          success = false;
+          LOG.error("Cache update failed for cache {}", cacheName);
+        }
+      }
+    } else {
+      success = false;
+      LOG.info("Skipping State Store cache update, driver is not ready.");
+    }
+    if (success) {
+      // Uses local time, not driver time.
+      this.cacheLastUpdateTime = Time.now();
+    }
+  }
+
+  /**
+   * Update the cache for a specific record store.
+   *
+   * @param clazz Class of the record store.
+   * @return If the cached was loaded.
+   * @throws IOException if the cache update failed.
+   */
+  public boolean loadCache(final Class<?> clazz) throws IOException {
+    return loadCache(clazz, false);
+  }
+
+  /**
+   * Update the cache for a specific record store.
+   *
+   * @param clazz Class of the record store.
+   * @param force Force the update ignoring cached periods.
+   * @return If the cached was loaded.
+   * @throws IOException if the cache update failed.
+   */
+  public boolean loadCache(Class<?> clazz, boolean force) throws IOException {
+    List<StateStoreCache> cachesToUpdate =
+        new LinkedList<StateStoreCache>();
+    cachesToUpdate.addAll(this.cachesToUpdateInternal);
+    cachesToUpdate.addAll(this.cachesToUpdateExternal);
+    for (StateStoreCache cachedStore : cachesToUpdate) {
+      if (clazz.isInstance(cachedStore)) {
+        return cachedStore.loadCache(force);
+      }
+    }
+    throw new IOException("Registered cache was not found for " + clazz);
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MembershipStoreImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MembershipStoreImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MembershipStoreImpl.java
new file mode 100644
index 0000000..c28131f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MembershipStoreImpl.java
@@ -0,0 +1,311 @@
+/**
+ * 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.store.impl;
+
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.filterMultiple;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreCache;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the {@link MembershipStore} State Store API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MembershipStoreImpl
+    extends MembershipStore implements StateStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MembershipStoreImpl.class);
+
+
+  /** Reported namespaces that are not decommissioned. */
+  private final Set<FederationNamespaceInfo> activeNamespaces;
+
+  /** Namenodes (after evaluating the quorum) that are active in the cluster. */
+  private final Map<String, MembershipState> activeRegistrations;
+  /** Namenode status reports (raw) that were discarded for being too old. */
+  private final Map<String, MembershipState> expiredRegistrations;
+
+  /** Lock to access the local memory cache. */
+  private final ReadWriteLock cacheReadWriteLock =
+      new ReentrantReadWriteLock();
+  private final Lock cacheReadLock = cacheReadWriteLock.readLock();
+  private final Lock cacheWriteLock = cacheReadWriteLock.writeLock();
+
+
+  public MembershipStoreImpl(StateStoreDriver driver) {
+    super(driver);
+
+    this.activeRegistrations = new HashMap<>();
+    this.expiredRegistrations = new HashMap<>();
+    this.activeNamespaces = new TreeSet<>();
+  }
+
+  @Override
+  public GetNamenodeRegistrationsResponse getExpiredNamenodeRegistrations(
+      GetNamenodeRegistrationsRequest request) throws IOException {
+
+    GetNamenodeRegistrationsResponse response =
+        GetNamenodeRegistrationsResponse.newInstance();
+    cacheReadLock.lock();
+    try {
+      Collection<MembershipState> vals = this.expiredRegistrations.values();
+      List<MembershipState> copyVals = new ArrayList<>(vals);
+      response.setNamenodeMemberships(copyVals);
+    } finally {
+      cacheReadLock.unlock();
+    }
+    return response;
+  }
+
+  @Override
+  public GetNamespaceInfoResponse getNamespaceInfo(
+      GetNamespaceInfoRequest request) throws IOException {
+
+    Set<FederationNamespaceInfo> namespaces = new HashSet<>();
+    try {
+      cacheReadLock.lock();
+      namespaces.addAll(activeNamespaces);
+    } finally {
+      cacheReadLock.unlock();
+    }
+
+    GetNamespaceInfoResponse response =
+        GetNamespaceInfoResponse.newInstance(namespaces);
+    return response;
+  }
+
+  @Override
+  public GetNamenodeRegistrationsResponse getNamenodeRegistrations(
+      final GetNamenodeRegistrationsRequest request) throws IOException {
+
+    // TODO Cache some common queries and sorts
+    List<MembershipState> ret = null;
+
+    cacheReadLock.lock();
+    try {
+      Collection<MembershipState> registrations = activeRegistrations.values();
+      MembershipState partialMembership = request.getPartialMembership();
+      if (partialMembership == null) {
+        ret = new ArrayList<>(registrations);
+      } else {
+        Query<MembershipState> query = new Query<>(partialMembership);
+        ret = filterMultiple(query, registrations);
+      }
+    } finally {
+      cacheReadLock.unlock();
+    }
+    // Sort in ascending update date order
+    Collections.sort(ret);
+
+    GetNamenodeRegistrationsResponse response =
+        GetNamenodeRegistrationsResponse.newInstance(ret);
+    return response;
+  }
+
+  @Override
+  public NamenodeHeartbeatResponse namenodeHeartbeat(
+      NamenodeHeartbeatRequest request) throws IOException {
+
+    MembershipState record = request.getNamenodeMembership();
+    String nnId = record.getNamenodeKey();
+    MembershipState existingEntry = null;
+    cacheReadLock.lock();
+    try {
+      existingEntry = this.activeRegistrations.get(nnId);
+    } finally {
+      cacheReadLock.unlock();
+    }
+
+    if (existingEntry != null) {
+      if (existingEntry.getState() != record.getState()) {
+        LOG.info("NN registration state has changed: {} -> {}",
+            existingEntry, record);
+      } else {
+        LOG.debug("Updating NN registration: {} -> {}", existingEntry, record);
+      }
+    } else {
+      LOG.info("Inserting new NN registration: {}", record);
+    }
+
+    boolean status = getDriver().put(record, true, false);
+
+    NamenodeHeartbeatResponse response =
+        NamenodeHeartbeatResponse.newInstance(status);
+    return response;
+  }
+
+  @Override
+  public boolean loadCache(boolean force) throws IOException {
+    super.loadCache(force);
+
+    // Update local cache atomically
+    cacheWriteLock.lock();
+    try {
+      this.activeRegistrations.clear();
+      this.expiredRegistrations.clear();
+      this.activeNamespaces.clear();
+
+      // Build list of NN registrations: nnId -> registration list
+      Map<String, List<MembershipState>> nnRegistrations = new HashMap<>();
+      List<MembershipState> cachedRecords = getCachedRecords();
+      for (MembershipState membership : cachedRecords) {
+        String nnId = membership.getNamenodeKey();
+        if (membership.getState() == FederationNamenodeServiceState.EXPIRED) {
+          // Expired, RPC service does not use these
+          String key = membership.getPrimaryKey();
+          this.expiredRegistrations.put(key, membership);
+        } else {
+          // This is a valid NN registration, build a list of all registrations
+          // using the NN id to use for the quorum calculation.
+          List<MembershipState> nnRegistration =
+              nnRegistrations.get(nnId);
+          if (nnRegistration == null) {
+            nnRegistration = new LinkedList<>();
+            nnRegistrations.put(nnId, nnRegistration);
+          }
+          nnRegistration.add(membership);
+          String bpId = membership.getBlockPoolId();
+          String cId = membership.getClusterId();
+          String nsId = membership.getNameserviceId();
+          FederationNamespaceInfo nsInfo =
+              new FederationNamespaceInfo(bpId, cId, nsId);
+          this.activeNamespaces.add(nsInfo);
+        }
+      }
+
+      // Calculate most representative entry for each active NN id
+      for (List<MembershipState> nnRegistration : nnRegistrations.values()) {
+        // Run quorum based on NN state
+        MembershipState representativeRecord =
+            getRepresentativeQuorum(nnRegistration);
+        String nnKey = representativeRecord.getNamenodeKey();
+        this.activeRegistrations.put(nnKey, representativeRecord);
+      }
+      LOG.debug("Refreshed {} NN registrations from State Store",
+          cachedRecords.size());
+    } finally {
+      cacheWriteLock.unlock();
+    }
+    return true;
+  }
+
+  @Override
+  public UpdateNamenodeRegistrationResponse updateNamenodeRegistration(
+      UpdateNamenodeRegistrationRequest request) throws IOException {
+
+    boolean status = false;
+    cacheWriteLock.lock();
+    try {
+      String namenode = MembershipState.getNamenodeKey(
+          request.getNameserviceId(), request.getNamenodeId());
+      MembershipState member = this.activeRegistrations.get(namenode);
+      if (member != null) {
+        member.setState(request.getState());
+        status = true;
+      }
+    } finally {
+      cacheWriteLock.unlock();
+    }
+    UpdateNamenodeRegistrationResponse response =
+        UpdateNamenodeRegistrationResponse.newInstance(status);
+    return response;
+  }
+
+  /**
+   * Picks the most recent entry in the subset that is most agreeable on the
+   * specified field. 1) If a majority of the collection has the same value for
+   * the field, the first sorted entry within the subset the matches the
+   * majority value 2) Otherwise the first sorted entry in the set of all
+   * entries
+   *
+   * @param entries - Collection of state store record objects of the same type
+   * @param fieldName - Field name for the value to compare
+   * @return record that is most representative of the field name
+   */
+  private MembershipState getRepresentativeQuorum(
+      Collection<MembershipState> records) {
+
+    // Collate objects by field value: field value -> order set of records
+    Map<FederationNamenodeServiceState, TreeSet<MembershipState>> occurenceMap =
+        new HashMap<>();
+    for (MembershipState record : records) {
+      FederationNamenodeServiceState state = record.getState();
+      TreeSet<MembershipState> matchingSet = occurenceMap.get(state);
+      if (matchingSet == null) {
+        // TreeSet orders elements by descending date via comparators
+        matchingSet = new TreeSet<>();
+        occurenceMap.put(state, matchingSet);
+      }
+      matchingSet.add(record);
+    }
+
+    // Select largest group
+    TreeSet<MembershipState> largestSet = new TreeSet<>();
+    for (TreeSet<MembershipState> matchingSet : occurenceMap.values()) {
+      if (largestSet.size() < matchingSet.size()) {
+        largestSet = matchingSet;
+      }
+    }
+
+    // If quorum, use the newest element here
+    if (largestSet.size() > records.size() / 2) {
+      return largestSet.first();
+      // Otherwise, return most recent by class comparator
+    } else if (records.size() > 0) {
+      TreeSet<MembershipState> sortedList = new TreeSet<>(records);
+      LOG.debug("Quorum failed, using most recent: {}", sortedList.first());
+      return sortedList.first();
+    } else {
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/package-info.java
new file mode 100644
index 0000000..1a50d15
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/package-info.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains implementations of the state store API interfaces. All classes
+ * derive from {@link
+ * org.apache.hadoop.hdfs.server.federation.store.RecordStore}. The API
+ * definitions are contained in the
+ * org.apache.hadoop.hdfs.server.federation.store package.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.store.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsRequest.java
new file mode 100644
index 0000000..568feaf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsRequest.java
@@ -0,0 +1,52 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+
+/**
+ * API request for listing namenode registrations present in the state store.
+ */
+public abstract class GetNamenodeRegistrationsRequest {
+
+  public static GetNamenodeRegistrationsRequest newInstance()
+      throws IOException {
+    return StateStoreSerializer.newRecord(
+        GetNamenodeRegistrationsRequest.class);
+  }
+
+  public static GetNamenodeRegistrationsRequest newInstance(
+      MembershipState member) throws IOException {
+    GetNamenodeRegistrationsRequest request = newInstance();
+    request.setPartialMembership(member);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract MembershipState getPartialMembership();
+
+  @Public
+  @Unstable
+  public abstract void setPartialMembership(MembershipState member);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsResponse.java
new file mode 100644
index 0000000..0d60c90
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamenodeRegistrationsResponse.java
@@ -0,0 +1,55 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+
+/**
+ * API response for listing namenode registrations present in the state store.
+ */
+public abstract class GetNamenodeRegistrationsResponse {
+
+  public static GetNamenodeRegistrationsResponse newInstance()
+      throws IOException {
+    return StateStoreSerializer.newRecord(
+        GetNamenodeRegistrationsResponse.class);
+  }
+
+  public static GetNamenodeRegistrationsResponse newInstance(
+      List<MembershipState> records) throws IOException {
+    GetNamenodeRegistrationsResponse response = newInstance();
+    response.setNamenodeMemberships(records);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract List<MembershipState> getNamenodeMemberships()
+      throws IOException;
+
+  @Public
+  @Unstable
+  public abstract void setNamenodeMemberships(
+      List<MembershipState> records) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoRequest.java
new file mode 100644
index 0000000..b5cc01b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoRequest.java
@@ -0,0 +1,30 @@
+/**
+ * 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.store.protocol;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for listing HDFS namespaces present in the state store.
+ */
+public abstract class GetNamespaceInfoRequest {
+
+  public static GetNamespaceInfoRequest newInstance() {
+    return StateStoreSerializer.newRecord(GetNamespaceInfoRequest.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoResponse.java
new file mode 100644
index 0000000..f541453
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetNamespaceInfoResponse.java
@@ -0,0 +1,52 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for listing HDFS namespaces present in the state store.
+ */
+public abstract class GetNamespaceInfoResponse {
+
+  public static GetNamespaceInfoResponse newInstance() {
+    return StateStoreSerializer.newRecord(GetNamespaceInfoResponse.class);
+  }
+
+  public static GetNamespaceInfoResponse newInstance(
+      Set<FederationNamespaceInfo> namespaces) throws IOException {
+    GetNamespaceInfoResponse response = newInstance();
+    response.setNamespaceInfo(namespaces);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract Set<FederationNamespaceInfo> getNamespaceInfo();
+
+  @Public
+  @Unstable
+  public abstract void setNamespaceInfo(
+      Set<FederationNamespaceInfo> namespaceInfo);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatRequest.java
new file mode 100644
index 0000000..9506026
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatRequest.java
@@ -0,0 +1,52 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+
+/**
+ * API request for registering a namenode with the state store.
+ */
+public abstract class NamenodeHeartbeatRequest {
+
+  public static NamenodeHeartbeatRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(NamenodeHeartbeatRequest.class);
+  }
+
+  public static NamenodeHeartbeatRequest newInstance(MembershipState namenode)
+      throws IOException {
+    NamenodeHeartbeatRequest request = newInstance();
+    request.setNamenodeMembership(namenode);
+    return request;
+  }
+
+  @Private
+  @Unstable
+  public abstract MembershipState getNamenodeMembership()
+      throws IOException;
+
+  @Private
+  @Unstable
+  public abstract void setNamenodeMembership(MembershipState report)
+      throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatResponse.java
new file mode 100644
index 0000000..acb7a6f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/NamenodeHeartbeatResponse.java
@@ -0,0 +1,49 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for registering a namenode with the state store.
+ */
+public abstract class NamenodeHeartbeatResponse {
+
+  public static NamenodeHeartbeatResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(NamenodeHeartbeatResponse.class);
+  }
+
+  public static NamenodeHeartbeatResponse newInstance(boolean status)
+      throws IOException {
+    NamenodeHeartbeatResponse response = newInstance();
+    response.setResult(status);
+    return response;
+  }
+
+  @Private
+  @Unstable
+  public abstract boolean getResult();
+
+  @Private
+  @Unstable
+  public abstract void setResult(boolean result);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationRequest.java
new file mode 100644
index 0000000..4459e33
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationRequest.java
@@ -0,0 +1,72 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for overriding an existing namenode registration in the state
+ * store.
+ */
+public abstract class UpdateNamenodeRegistrationRequest {
+
+  public static UpdateNamenodeRegistrationRequest newInstance()
+      throws IOException {
+    return StateStoreSerializer.newRecord(
+        UpdateNamenodeRegistrationRequest.class);
+  }
+
+  public static UpdateNamenodeRegistrationRequest newInstance(
+      String nameserviceId, String namenodeId,
+      FederationNamenodeServiceState state) throws IOException {
+    UpdateNamenodeRegistrationRequest request = newInstance();
+    request.setNameserviceId(nameserviceId);
+    request.setNamenodeId(namenodeId);
+    request.setState(state);
+    return request;
+  }
+
+  @Private
+  @Unstable
+  public abstract String getNameserviceId();
+
+  @Private
+  @Unstable
+  public abstract String getNamenodeId();
+
+  @Private
+  @Unstable
+  public abstract FederationNamenodeServiceState getState();
+
+  @Private
+  @Unstable
+  public abstract void setNameserviceId(String nsId);
+
+  @Private
+  @Unstable
+  public abstract void setNamenodeId(String nnId);
+
+  @Private
+  @Unstable
+  public abstract void setState(FederationNamenodeServiceState state);
+}
\ 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


[06/29] hadoop git commit: HADOOP-14839. DistCp log output should contain copied and deleted files and directories. Contributed by Yiqun Lin.

Posted by in...@apache.org.
HADOOP-14839. DistCp log output should contain copied and deleted files and directories. 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/63720ef5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/63720ef5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/63720ef5

Branch: refs/heads/HDFS-10467
Commit: 63720ef574df6d79a9bab0628edb915d1e3a7c30
Parents: 50506e9
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Sep 5 21:45:30 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Sep 5 23:34:55 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/tools/DistCpConstants.java    |  1 +
 .../apache/hadoop/tools/DistCpOptionSwitch.java |  7 ++
 .../org/apache/hadoop/tools/DistCpOptions.java  | 22 +++++
 .../org/apache/hadoop/tools/OptionsParser.java  |  4 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  | 32 +++++--
 .../src/site/markdown/DistCp.md.vm              |  1 +
 .../apache/hadoop/tools/TestDistCpOptions.java  | 21 ++++-
 .../hadoop/tools/mapred/TestCopyMapper.java     | 99 +++++++++++++++++++-
 .../hadoop/tools/DistCp_Counter.properties      |  1 +
 9 files changed, 177 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
index 8991e09..0bae5d5 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
@@ -45,6 +45,7 @@ public class DistCpConstants {
   public static final String CONF_LABEL_ATOMIC_COPY = "distcp.atomic.copy";
   public static final String CONF_LABEL_WORK_PATH = "distcp.work.path";
   public static final String CONF_LABEL_LOG_PATH = "distcp.log.path";
+  public static final String CONF_LABEL_VERBOSE_LOG = "distcp.verbose.log";
   public static final String CONF_LABEL_IGNORE_FAILURES = "distcp.ignore.failures";
   public static final String CONF_LABEL_PRESERVE_STATUS = "distcp.preserve.status";
   public static final String CONF_LABEL_PRESERVE_RAWXATTRS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
index 016172e0..e1c824e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
@@ -119,6 +119,13 @@ public enum DistCpOptionSwitch {
       new Option("log", true, "Folder on DFS where distcp execution logs are saved")),
 
   /**
+   * Log additional info (path, size) in the SKIP/COPY log.
+   */
+  VERBOSE_LOG(DistCpConstants.CONF_LABEL_VERBOSE_LOG,
+      new Option("v", false,
+          "Log additional info (path, size) in the SKIP/COPY log")),
+
+  /**
    * Copy strategy is use. This could be dynamic or uniform size etc.
    * DistCp would use an appropriate input format based on this.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
index af6cb8b..ece1a94 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
@@ -101,6 +101,9 @@ public final class DistCpOptions {
   // content at their s1, if src is not the same as tgt.
   private final boolean useRdiff;
 
+  /** Whether to log additional info (path, size) in the SKIP/COPY log. */
+  private final boolean verboseLog;
+
   // For both -diff and -rdiff, given the example command line switches, two
   // steps are taken:
   //   1. Sync Step. This step does renaming/deletion ops in the snapshot diff,
@@ -204,6 +207,7 @@ public final class DistCpOptions {
     this.blocksPerChunk = builder.blocksPerChunk;
 
     this.copyBufferSize = builder.copyBufferSize;
+    this.verboseLog = builder.verboseLog;
   }
 
   public Path getSourceFileListing() {
@@ -323,6 +327,10 @@ public final class DistCpOptions {
     return copyBufferSize;
   }
 
+  public boolean shouldVerboseLog() {
+    return verboseLog;
+  }
+
   /**
    * Add options to configuration. These will be used in the Mapper/committer
    *
@@ -361,6 +369,8 @@ public final class DistCpOptions {
         String.valueOf(blocksPerChunk));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.COPY_BUFFER_SIZE,
         String.valueOf(copyBufferSize));
+    DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.VERBOSE_LOG,
+        String.valueOf(verboseLog));
   }
 
   /**
@@ -396,6 +406,7 @@ public final class DistCpOptions {
         ", filtersFile='" + filtersFile + '\'' +
         ", blocksPerChunk=" + blocksPerChunk +
         ", copyBufferSize=" + copyBufferSize +
+        ", verboseLog=" + verboseLog +
         '}';
   }
 
@@ -420,6 +431,7 @@ public final class DistCpOptions {
     private boolean append = false;
     private boolean skipCRC = false;
     private boolean blocking = true;
+    private boolean verboseLog = false;
 
     private boolean useDiff = false;
     private boolean useRdiff = false;
@@ -552,6 +564,11 @@ public final class DistCpOptions {
         throw new IllegalArgumentException(
             "-diff and -rdiff are mutually exclusive");
       }
+
+      if (verboseLog && logPath == null) {
+        throw new IllegalArgumentException(
+            "-v is valid only with -log option");
+      }
     }
 
     @VisibleForTesting
@@ -685,6 +702,11 @@ public final class DistCpOptions {
               : DistCpConstants.COPY_BUFFER_SIZE_DEFAULT;
       return this;
     }
+
+    public Builder withVerboseLog(boolean newVerboseLog) {
+      this.verboseLog = newVerboseLog;
+      return this;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
index 2bfaccf..606ed32 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
@@ -111,7 +111,9 @@ public class OptionsParser {
         .withCRC(
             command.hasOption(DistCpOptionSwitch.SKIP_CRC.getSwitch()))
         .withBlocking(
-            !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch()));
+            !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch()))
+        .withVerboseLog(
+            command.hasOption(DistCpOptionSwitch.VERBOSE_LOG.getSwitch()));
 
     if (command.hasOption(DistCpOptionSwitch.DIFF.getSwitch())) {
       String[] snapshots = getVals(command,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
index d6b3ba8..faa4aa2 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
@@ -54,6 +54,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
    */
   public static enum Counter {
     COPY,         // Number of files received by the mapper for copy.
+    DIR_COPY,     // Number of directories received by the mapper for copy.
     SKIP,         // Number of files skipped.
     FAIL,         // Number of files that failed to be copied.
     BYTESCOPIED,  // Number of bytes actually copied by the copy-mapper, total.
@@ -82,6 +83,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   private boolean skipCrc = false;
   private boolean overWrite = false;
   private boolean append = false;
+  private boolean verboseLog = false;
   private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
 
   private FileSystem targetFS = null;
@@ -105,6 +107,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     skipCrc = conf.getBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(), false);
     overWrite = conf.getBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(), false);
     append = conf.getBoolean(DistCpOptionSwitch.APPEND.getConfigLabel(), false);
+    verboseLog = conf.getBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), false);
     preserve = DistCpUtils.unpackAttributes(conf.get(DistCpOptionSwitch.
         PRESERVE_STATUS.getConfigLabel()));
 
@@ -196,6 +200,13 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         updateSkipCounters(context, sourceCurrStatus);
         context.write(null, new Text("SKIP: " + sourceCurrStatus.getPath()));
 
+        if (verboseLog) {
+          context.write(null,
+              new Text("FILE_SKIPPED: source=" + sourceFileStatus.getPath()
+              + ", size=" + sourceFileStatus.getLen() + " --> "
+              + "target=" + target + ", size=" + (targetStatus == null ?
+                  0 : targetStatus.getLen())));
+        }
       } else {
         if (sourceCurrStatus.isSplit()) {
           tmpTarget = DistCpUtils.getSplitChunkPath(target, sourceCurrStatus);
@@ -203,8 +214,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         if (LOG.isDebugEnabled()) {
           LOG.debug("copying " + sourceCurrStatus + " " + tmpTarget);
         }
-        copyFileWithRetry(description, sourceCurrStatus, tmpTarget, context,
-            action, fileAttributes);
+        copyFileWithRetry(description, sourceCurrStatus, tmpTarget,
+            targetStatus, context, action, fileAttributes);
       }
       DistCpUtils.preserve(target.getFileSystem(conf), tmpTarget,
           sourceCurrStatus, fileAttributes, preserveRawXattrs);
@@ -235,9 +246,10 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   }
 
   private void copyFileWithRetry(String description,
-      CopyListingFileStatus sourceFileStatus, Path target, Context context,
-      FileAction action, EnumSet<DistCpOptions.FileAttribute> fileAttributes)
-      throws IOException {
+      CopyListingFileStatus sourceFileStatus, Path target,
+      FileStatus targrtFileStatus, Context context, FileAction action,
+      EnumSet<DistCpOptions.FileAttribute> fileAttributes)
+      throws IOException, InterruptedException {
     long bytesCopied;
     try {
       bytesCopied = (Long) new RetriableFileCopyCommand(skipCrc, description,
@@ -251,6 +263,14 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
     incrementCounter(context, Counter.COPY, 1);
     totalBytesCopied += bytesCopied;
+
+    if (verboseLog) {
+      context.write(null,
+          new Text("FILE_COPIED: source=" + sourceFileStatus.getPath() + ","
+          + " size=" + sourceFileStatus.getLen() + " --> "
+          + "target=" + target + ", size=" + (targrtFileStatus == null ?
+              0 : targrtFileStatus.getLen())));
+    }
   }
 
   private void createTargetDirsWithRetry(String description,
@@ -260,7 +280,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     } catch (Exception e) {
       throw new IOException("mkdir failed for " + target, e);
     }
-    incrementCounter(context, Counter.COPY, 1);
+    incrementCounter(context, Counter.DIR_COPY, 1);
   }
 
   private static void updateSkipCounters(Context context,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
index 925b24e..2cd01e2 100644
--- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
+++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
@@ -220,6 +220,7 @@ Flag              | Description                          | Notes
 `-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions. The option -pr is only valid if both source and target directory are not erasure coded. **Note:** If -p option's are not specified, then by default block size is preserved.
 `-i` | Ignore failures | As explained in the Appendix, this option will keep more accurate statistics about the copy than the default case. It also preserves logs from failed copies, which can be valuable for debugging. Finally, a failing map will not cause the job to fail before all splits are attempted.
 `-log <logdir>` | Write logs to \<logdir\> | DistCp keeps logs of each file it attempts to copy as map output. If a map fails, the log output will not be retained if it is re-executed.
+`-v` | Log additional info (path, size) in the SKIP/COPY log | This option can only be used with -log option.
 `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput.
 `-overwrite` | Overwrite destination | If a map fails and `-i` is not specified, all the files in the split, not only those that failed, will be recopied. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.
 `-update` | Overwrite if source and destination differ in size, blocksize, or checksum | As noted in the preceding, this is not a "sync" operation. The criteria examined are the source and destination file sizes, blocksizes, and checksums; if they differ, the source file replaces the destination file. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
index 6b59b97..dd8ec69 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
@@ -287,7 +287,7 @@ public class TestDistCpOptions {
         "mapBandwidth=0.0, copyStrategy='uniformsize', preserveStatus=[], " +
         "atomicWorkPath=null, logPath=null, sourceFileListing=abc, " +
         "sourcePaths=null, targetPath=xyz, filtersFile='null'," +
-        " blocksPerChunk=0, copyBufferSize=8192}";
+        " blocksPerChunk=0, copyBufferSize=8192, verboseLog=false}";
     String optionString = option.toString();
     Assert.assertEquals(val, optionString);
     Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
@@ -514,4 +514,23 @@ public class TestDistCpOptions {
     Assert.assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
         builder.build().getCopyBufferSize());
   }
+
+  @Test
+  public void testVerboseLog() {
+    final DistCpOptions.Builder builder = new DistCpOptions.Builder(
+        Collections.singletonList(new Path("hdfs://localhost:8020/source")),
+        new Path("hdfs://localhost:8020/target/"));
+    Assert.assertFalse(builder.build().shouldVerboseLog());
+
+    try {
+      builder.withVerboseLog(true).build();
+      fail("-v should fail if -log option is not specified");
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("-v is valid only with -log option", e);
+    }
+
+    final Path logPath = new Path("hdfs://localhost:8020/logs");
+    builder.withLogPath(logPath).withVerboseLog(true);
+    Assert.assertTrue(builder.build().shouldVerboseLog());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
index 866ad6e..fd998c8 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
@@ -255,7 +255,13 @@ public class TestCopyMapper {
     context.getConfiguration().setBoolean(
         DistCpOptionSwitch.APPEND.getConfigLabel(), true);
     copyMapper.setup(context);
+
+    int numFiles = 0;
     for (Path path: pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
       copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
               new CopyListingFileStatus(cluster.getFileSystem().getFileStatus(
                   path)), context);
@@ -266,7 +272,7 @@ public class TestCopyMapper {
     Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
         .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
         .getValue());
-    Assert.assertEquals(pathList.size(), stubContext.getReporter().
+    Assert.assertEquals(numFiles, stubContext.getReporter().
         getCounter(CopyMapper.Counter.COPY).getValue());
   }
 
@@ -295,7 +301,15 @@ public class TestCopyMapper {
 
     copyMapper.setup(context);
 
-    for (Path path: pathList) {
+    int numFiles = 0;
+    int numDirs = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isDirectory()) {
+        numDirs++;
+      } else {
+        numFiles++;
+      }
+
       copyMapper.map(
           new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
           new CopyListingFileStatus(fs.getFileStatus(path)), context);
@@ -303,8 +317,10 @@ public class TestCopyMapper {
 
     // Check that the maps worked.
     verifyCopy(fs, preserveChecksum);
-    Assert.assertEquals(pathList.size(), stubContext.getReporter()
+    Assert.assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
+    Assert.assertEquals(numDirs, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.DIR_COPY).getValue());
     if (!preserveChecksum) {
       Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
           .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
@@ -1118,4 +1134,81 @@ public class TestCopyMapper {
       e.printStackTrace();
     }
   }
+
+  @Test
+  public void testVerboseLogging() throws Exception {
+    deleteState();
+    createSourceData();
+
+    FileSystem fs = cluster.getFileSystem();
+    CopyMapper copyMapper = new CopyMapper();
+    StubContext stubContext = new StubContext(getConfiguration(), null, 0);
+    Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
+            = stubContext.getContext();
+    copyMapper.setup(context);
+
+    int numFiles = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    // Check that the maps worked.
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    testCopyingExistingFiles(fs, copyMapper, context);
+    // verify the verbose log
+    // we shouldn't print verbose log since this option is disabled
+    for (Text value : stubContext.getWriter().values()) {
+      Assert.assertTrue(!value.toString().startsWith("FILE_COPIED:"));
+      Assert.assertTrue(!value.toString().startsWith("FILE_SKIPPED:"));
+    }
+
+    // test with verbose logging
+    deleteState();
+    createSourceData();
+
+    stubContext = new StubContext(getConfiguration(), null, 0);
+    context = stubContext.getContext();
+    copyMapper.setup(context);
+
+    // enables verbose logging
+    context.getConfiguration().setBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), true);
+    copyMapper.setup(context);
+
+    for (Path path : pathList) {
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    // verify the verbose log of COPY log
+    int numFileCopied = 0;
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_COPIED:")) {
+        numFileCopied++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileCopied);
+
+    // verify the verbose log of SKIP log
+    int numFileSkipped = 0;
+    testCopyingExistingFiles(fs, copyMapper, context);
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_SKIPPED:")) {
+        numFileSkipped++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileSkipped);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63720ef5/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
index 2234619..77f6c02 100644
--- a/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
+++ b/hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties
@@ -15,6 +15,7 @@
 CounterGroupName=       distcp
 
 COPY.name=              Files copied
+DIR_COPY.name=          Directories copied
 SKIP.name=              Files skipped
 FAIL.name=              Files failed
 BYTESCOPIED.name=       Bytes copied


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


[12/29] hadoop git commit: HDFS-10687. Federation Membership State Store internal API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationResponse.java
new file mode 100644
index 0000000..1f0d556
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateNamenodeRegistrationResponse.java
@@ -0,0 +1,51 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for overriding an existing namenode registration in the state
+ * store.
+ */
+public abstract class UpdateNamenodeRegistrationResponse {
+
+  public static UpdateNamenodeRegistrationResponse newInstance() {
+    return StateStoreSerializer.newRecord(
+        UpdateNamenodeRegistrationResponse.class);
+  }
+
+  public static UpdateNamenodeRegistrationResponse newInstance(boolean status)
+      throws IOException {
+    UpdateNamenodeRegistrationResponse response = newInstance();
+    response.setResult(status);
+    return response;
+  }
+
+  @Private
+  @Unstable
+  public abstract boolean getResult();
+
+  @Private
+  @Unstable
+  public abstract void setResult(boolean result);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java
new file mode 100644
index 0000000..baad113
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java
@@ -0,0 +1,145 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+
+import org.apache.commons.codec.binary.Base64;
+
+import com.google.protobuf.GeneratedMessage;
+import com.google.protobuf.Message;
+import com.google.protobuf.Message.Builder;
+import com.google.protobuf.MessageOrBuilder;
+
+/**
+ * Helper class for setting/getting data elements in an object backed by a
+ * protobuf implementation.
+ */
+public class FederationProtocolPBTranslator<P extends GeneratedMessage,
+    B extends Builder, T extends MessageOrBuilder> {
+
+  /** Optional proto byte stream used to create this object. */
+  private P proto;
+  /** The class of the proto handler for this translator. */
+  private Class<P> protoClass;
+  /** Internal builder, used to store data that has been set. */
+  private B builder;
+
+  public FederationProtocolPBTranslator(Class<P> protoType) {
+    this.protoClass = protoType;
+  }
+
+  /**
+   * Called if this translator is to be created from an existing protobuf byte
+   * stream.
+   *
+   * @param p The existing proto object to use to initialize the translator.
+   * @throws IllegalArgumentException
+   */
+  @SuppressWarnings("unchecked")
+  public void setProto(Message p) {
+    if (protoClass.isInstance(p)) {
+      if (this.builder != null) {
+        // Merge with builder
+        this.builder.mergeFrom((P) p);
+      } else {
+        // Store proto
+        this.proto = (P) p;
+      }
+    } else {
+      throw new IllegalArgumentException(
+          "Cannot decode proto type " + p.getClass().getName());
+    }
+  }
+
+  /**
+   * Create or return the cached protobuf builder for this translator.
+   *
+   * @return cached Builder instance
+   */
+  @SuppressWarnings("unchecked")
+  public B getBuilder() {
+    if (this.builder == null) {
+      try {
+        Method method = protoClass.getMethod("newBuilder");
+        this.builder = (B) method.invoke(null);
+        if (this.proto != null) {
+          // Merge in existing immutable proto
+          this.builder.mergeFrom(this.proto);
+        }
+      } catch (ReflectiveOperationException e) {
+        this.builder = null;
+      }
+    }
+    return this.builder;
+  }
+
+  /**
+   * Get the serialized proto object. If the translator was created from a byte
+   * stream, returns the intitial byte stream. Otherwise creates a new byte
+   * stream from the cached builder.
+   *
+   * @return Protobuf message object
+   */
+  @SuppressWarnings("unchecked")
+  public P build() {
+    if (this.builder != null) {
+      // serialize from builder (mutable) first
+      Message m = this.builder.build();
+      return (P) m;
+    } else if (this.proto != null) {
+      // Use immutable message source, message is unchanged
+      return this.proto;
+    }
+    return null;
+  }
+
+  /**
+   * Returns an interface to access data stored within this object. The object
+   * may have been initialized either via a builder or by an existing protobuf
+   * byte stream.
+   *
+   * @return MessageOrBuilder protobuf interface for the requested class.
+   */
+  @SuppressWarnings("unchecked")
+  public T getProtoOrBuilder() {
+    if (this.builder != null) {
+      // Use mutable builder if it exists
+      return (T) this.builder;
+    } else if (this.proto != null) {
+      // Use immutable message source
+      return (T) this.proto;
+    } else {
+      // Construct empty builder
+      return (T) this.getBuilder();
+    }
+  }
+
+  /**
+   * Read instance from base64 data.
+   * @param base64String
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public void readInstance(String base64String) throws IOException {
+    byte[] bytes = Base64.decodeBase64(base64String);
+    Message msg = getBuilder().mergeFrom(bytes).build();
+    this.proto = (P) msg;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsRequestPBImpl.java
new file mode 100644
index 0000000..4f7fee1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsRequestPBImpl.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.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamenodeRegistrationsRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamenodeRegistrationsRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.MembershipStatePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetNamenodeRegistrationsRequest.
+ */
+public class GetNamenodeRegistrationsRequestPBImpl
+    extends GetNamenodeRegistrationsRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetNamenodeRegistrationsRequestProto,
+      GetNamenodeRegistrationsRequestProto.Builder,
+      GetNamenodeRegistrationsRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              GetNamenodeRegistrationsRequestProto,
+              GetNamenodeRegistrationsRequestProto.Builder,
+              GetNamenodeRegistrationsRequestProtoOrBuilder>(
+                  GetNamenodeRegistrationsRequestProto.class);
+
+  public GetNamenodeRegistrationsRequestPBImpl() {
+  }
+
+  public GetNamenodeRegistrationsRequestPBImpl(
+      GetNamenodeRegistrationsRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public GetNamenodeRegistrationsRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public MembershipState getPartialMembership() {
+    GetNamenodeRegistrationsRequestProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasMembership()){
+      return null;
+    }
+    NamenodeMembershipRecordProto memberProto = proto.getMembership();
+    return new MembershipStatePBImpl(memberProto);
+  }
+
+  @Override
+  public void setPartialMembership(MembershipState member) {
+    MembershipStatePBImpl memberPB = (MembershipStatePBImpl)member;
+    this.translator.getBuilder().setMembership(memberPB.getProto());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsResponsePBImpl.java
new file mode 100644
index 0000000..f6be11d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamenodeRegistrationsResponsePBImpl.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamenodeRegistrationsResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamenodeRegistrationsResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.MembershipStatePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetNamenodeRegistrationsResponse.
+ */
+public class GetNamenodeRegistrationsResponsePBImpl
+    extends GetNamenodeRegistrationsResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetNamenodeRegistrationsResponseProto,
+      GetNamenodeRegistrationsResponseProto.Builder,
+      GetNamenodeRegistrationsResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              GetNamenodeRegistrationsResponseProto,
+              GetNamenodeRegistrationsResponseProto.Builder,
+              GetNamenodeRegistrationsResponseProtoOrBuilder>(
+                  GetNamenodeRegistrationsResponseProto.class);
+
+  public GetNamenodeRegistrationsResponsePBImpl() {
+  }
+
+  public GetNamenodeRegistrationsResponsePBImpl(
+      GetNamenodeRegistrationsResponseProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public GetNamenodeRegistrationsResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public List<MembershipState> getNamenodeMemberships()
+      throws IOException {
+
+    List<MembershipState> ret = new ArrayList<MembershipState>();
+    List<NamenodeMembershipRecordProto> memberships =
+        this.translator.getProtoOrBuilder().getNamenodeMembershipsList();
+    for (NamenodeMembershipRecordProto memberProto : memberships) {
+      MembershipState membership = new MembershipStatePBImpl(memberProto);
+      ret.add(membership);
+    }
+
+    return ret;
+  }
+
+  @Override
+  public void setNamenodeMemberships(List<MembershipState> records)
+      throws IOException {
+    for (MembershipState member : records) {
+      if (member instanceof MembershipStatePBImpl) {
+        MembershipStatePBImpl memberPB = (MembershipStatePBImpl)member;
+        this.translator.getBuilder().addNamenodeMemberships(
+            memberPB.getProto());
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoRequestPBImpl.java
new file mode 100644
index 0000000..5f3e186
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoRequestPBImpl.java
@@ -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.
+ */
+package org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamespaceInfoRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamespaceInfoRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamespaceInfoRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetNamespaceInfoRequest.
+ */
+public class GetNamespaceInfoRequestPBImpl extends GetNamespaceInfoRequest
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetNamespaceInfoRequestProto,
+      Builder, GetNamespaceInfoRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<GetNamespaceInfoRequestProto,
+              Builder, GetNamespaceInfoRequestProtoOrBuilder>(
+                  GetNamespaceInfoRequestProto.class);
+
+  public GetNamespaceInfoRequestPBImpl() {
+  }
+
+  @Override
+  public GetNamespaceInfoRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message protocol) {
+    this.translator.setProto(protocol);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoResponsePBImpl.java
new file mode 100644
index 0000000..be1b184
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetNamespaceInfoResponsePBImpl.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.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.FederationNamespaceInfoProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamespaceInfoResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetNamespaceInfoResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetNamespaceInfoResponse.
+ */
+public class GetNamespaceInfoResponsePBImpl
+    extends GetNamespaceInfoResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetNamespaceInfoResponseProto,
+      GetNamespaceInfoResponseProto.Builder,
+      GetNamespaceInfoResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<GetNamespaceInfoResponseProto,
+              GetNamespaceInfoResponseProto.Builder,
+              GetNamespaceInfoResponseProtoOrBuilder>(
+                  GetNamespaceInfoResponseProto.class);
+
+  public GetNamespaceInfoResponsePBImpl() {
+  }
+
+  @Override
+  public GetNamespaceInfoResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message protocol) {
+    this.translator.setProto(protocol);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public Set<FederationNamespaceInfo> getNamespaceInfo() {
+
+    Set<FederationNamespaceInfo> ret = new HashSet<FederationNamespaceInfo>();
+    List<FederationNamespaceInfoProto> namespaceList =
+        this.translator.getProtoOrBuilder().getNamespaceInfosList();
+    for (FederationNamespaceInfoProto ns : namespaceList) {
+      FederationNamespaceInfo info = new FederationNamespaceInfo(
+          ns.getBlockPoolId(), ns.getClusterId(), ns.getNameserviceId());
+      ret.add(info);
+    }
+    return ret;
+  }
+
+  @Override
+  public void setNamespaceInfo(Set<FederationNamespaceInfo> namespaceInfo) {
+    int index = 0;
+    for (FederationNamespaceInfo item : namespaceInfo) {
+      FederationNamespaceInfoProto.Builder itemBuilder =
+          FederationNamespaceInfoProto.newBuilder();
+      itemBuilder.setClusterId(item.getClusterId());
+      itemBuilder.setBlockPoolId(item.getBlockPoolId());
+      itemBuilder.setNameserviceId(item.getNameserviceId());
+      this.translator.getBuilder().addNamespaceInfos(index,
+          itemBuilder.build());
+      index++;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatRequestPBImpl.java
new file mode 100644
index 0000000..d1fc73f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatRequestPBImpl.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.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeHeartbeatRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeHeartbeatRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeHeartbeatRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.MembershipStatePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * NamenodeHeartbeatRequest.
+ */
+public class NamenodeHeartbeatRequestPBImpl
+    extends NamenodeHeartbeatRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<NamenodeHeartbeatRequestProto, Builder,
+      NamenodeHeartbeatRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<NamenodeHeartbeatRequestProto,
+              Builder,
+              NamenodeHeartbeatRequestProtoOrBuilder>(
+                  NamenodeHeartbeatRequestProto.class);
+
+  public NamenodeHeartbeatRequestPBImpl() {
+  }
+
+  @Override
+  public NamenodeHeartbeatRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public MembershipState getNamenodeMembership() throws IOException {
+    NamenodeMembershipRecordProto membershipProto =
+        this.translator.getProtoOrBuilder().getNamenodeMembership();
+    MembershipState membership =
+        StateStoreSerializer.newRecord(MembershipState.class);
+    if (membership instanceof MembershipStatePBImpl) {
+      MembershipStatePBImpl membershipPB = (MembershipStatePBImpl)membership;
+      membershipPB.setProto(membershipProto);
+      return membershipPB;
+    } else {
+      throw new IOException("Cannot get membership from request");
+    }
+  }
+
+  @Override
+  public void setNamenodeMembership(MembershipState membership)
+      throws IOException {
+    if (membership instanceof MembershipStatePBImpl) {
+      MembershipStatePBImpl membershipPB = (MembershipStatePBImpl)membership;
+      NamenodeMembershipRecordProto membershipProto =
+          (NamenodeMembershipRecordProto)membershipPB.getProto();
+      this.translator.getBuilder().setNamenodeMembership(membershipProto);
+    } else {
+      throw new IOException("Cannot set mount table entry");
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatResponsePBImpl.java
new file mode 100644
index 0000000..c243a6f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/NamenodeHeartbeatResponsePBImpl.java
@@ -0,0 +1,71 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeHeartbeatResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeHeartbeatResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * NamenodeHeartbeatResponse.
+ */
+public class NamenodeHeartbeatResponsePBImpl extends NamenodeHeartbeatResponse
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<NamenodeHeartbeatResponseProto,
+      NamenodeHeartbeatResponseProto.Builder,
+      NamenodeHeartbeatResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<NamenodeHeartbeatResponseProto,
+              NamenodeHeartbeatResponseProto.Builder,
+              NamenodeHeartbeatResponseProtoOrBuilder>(
+                  NamenodeHeartbeatResponseProto.class);
+
+  public NamenodeHeartbeatResponsePBImpl() {
+  }
+
+  @Override
+  public NamenodeHeartbeatResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getResult() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setResult(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationRequestPBImpl.java
new file mode 100644
index 0000000..5091360
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationRequestPBImpl.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.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateNamenodeRegistrationRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateNamenodeRegistrationRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * OverrideNamenodeRegistrationRequest.
+ */
+public class UpdateNamenodeRegistrationRequestPBImpl
+    extends UpdateNamenodeRegistrationRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<
+      UpdateNamenodeRegistrationRequestProto,
+      UpdateNamenodeRegistrationRequestProto.Builder,
+      UpdateNamenodeRegistrationRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              UpdateNamenodeRegistrationRequestProto,
+              UpdateNamenodeRegistrationRequestProto.Builder,
+              UpdateNamenodeRegistrationRequestProtoOrBuilder>(
+                  UpdateNamenodeRegistrationRequestProto.class);
+
+  public UpdateNamenodeRegistrationRequestPBImpl() {
+  }
+
+  @Override
+  public UpdateNamenodeRegistrationRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message protocol) {
+    this.translator.setProto(protocol);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public String getNameserviceId() {
+    return this.translator.getProtoOrBuilder().getNameserviceId();
+  }
+
+  @Override
+  public String getNamenodeId() {
+    return this.translator.getProtoOrBuilder().getNamenodeId();
+  }
+
+  @Override
+  public FederationNamenodeServiceState getState() {
+    return FederationNamenodeServiceState
+        .valueOf(this.translator.getProtoOrBuilder().getState());
+  }
+
+  @Override
+  public void setNameserviceId(String nsId) {
+    this.translator.getBuilder().setNameserviceId(nsId);
+  }
+
+  @Override
+  public void setNamenodeId(String nnId) {
+    this.translator.getBuilder().setNamenodeId(nnId);
+  }
+
+  @Override
+  public void setState(FederationNamenodeServiceState state) {
+    this.translator.getBuilder().setState(state.toString());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationResponsePBImpl.java
new file mode 100644
index 0000000..4558f06
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateNamenodeRegistrationResponsePBImpl.java
@@ -0,0 +1,73 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateNamenodeRegistrationResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateNamenodeRegistrationResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * OverrideNamenodeRegistrationResponse.
+ */
+public class UpdateNamenodeRegistrationResponsePBImpl
+    extends UpdateNamenodeRegistrationResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<
+      UpdateNamenodeRegistrationResponseProto,
+      UpdateNamenodeRegistrationResponseProto.Builder,
+      UpdateNamenodeRegistrationResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              UpdateNamenodeRegistrationResponseProto,
+              UpdateNamenodeRegistrationResponseProto.Builder,
+              UpdateNamenodeRegistrationResponseProtoOrBuilder>(
+                  UpdateNamenodeRegistrationResponseProto.class);
+
+  public UpdateNamenodeRegistrationResponsePBImpl() {
+  }
+
+  @Override
+  public UpdateNamenodeRegistrationResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getResult() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setResult(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/package-info.java
new file mode 100644
index 0000000..43c94be
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/package-info.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.
+ */
+
+/**
+ * Protobuf implementations of FederationProtocolBase request/response objects
+ * used by state store APIs. Each state store API is defined in the
+ * org.apache.hadoop.hdfs.server.federation.store.protocol package.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java
new file mode 100644
index 0000000..ab0ff0a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java
@@ -0,0 +1,329 @@
+/**
+ * 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.store.records;
+
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.ACTIVE;
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.EXPIRED;
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.UNAVAILABLE;
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * Data schema for storing NN registration information in the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.StateStoreService
+ * FederationStateStoreService}.
+ */
+public abstract class MembershipState extends BaseRecord
+    implements FederationNamenodeContext {
+
+  /** Expiration time in ms for this entry. */
+  private static long expirationMs;
+
+
+  /** Comparator based on the name.*/
+  public static final Comparator<MembershipState> NAME_COMPARATOR =
+      new Comparator<MembershipState>() {
+        public int compare(MembershipState m1, MembershipState m2) {
+          return m1.compareNameTo(m2);
+        }
+      };
+
+
+  /**
+   * Constructors.
+   */
+  public MembershipState() {
+    super();
+  }
+
+  /**
+   * Create a new membership instance.
+   * @return Membership instance.
+   * @throws IOException
+   */
+  public static MembershipState newInstance() {
+    MembershipState record =
+        StateStoreSerializer.newRecord(MembershipState.class);
+    record.init();
+    return record;
+  }
+
+  /**
+   * Create a new membership instance.
+   *
+   * @param router Identifier of the router.
+   * @param nameservice Identifier of the nameservice.
+   * @param namenode Identifier of the namenode.
+   * @param clusterId Identifier of the cluster.
+   * @param blockPoolId Identifier of the blockpool.
+   * @param rpcAddress RPC address.
+   * @param serviceAddress Service RPC address.
+   * @param lifelineAddress Lifeline RPC address.
+   * @param webAddress HTTP address.
+   * @param state State of the federation.
+   * @param safemode If the safe mode is enabled.
+   * @return Membership instance.
+   * @throws IOException If we cannot create the instance.
+   */
+  public static MembershipState newInstance(String router, String nameservice,
+      String namenode, String clusterId, String blockPoolId, String rpcAddress,
+      String serviceAddress, String lifelineAddress, String webAddress,
+      FederationNamenodeServiceState state, boolean safemode) {
+
+    MembershipState record = MembershipState.newInstance();
+    record.setRouterId(router);
+    record.setNameserviceId(nameservice);
+    record.setNamenodeId(namenode);
+    record.setRpcAddress(rpcAddress);
+    record.setServiceAddress(serviceAddress);
+    record.setLifelineAddress(lifelineAddress);
+    record.setWebAddress(webAddress);
+    record.setIsSafeMode(safemode);
+    record.setState(state);
+    record.setClusterId(clusterId);
+    record.setBlockPoolId(blockPoolId);
+    record.validate();
+    return record;
+  }
+
+  public abstract void setRouterId(String routerId);
+
+  public abstract String getRouterId();
+
+  public abstract void setNameserviceId(String nameserviceId);
+
+  public abstract void setNamenodeId(String namenodeId);
+
+  public abstract void setWebAddress(String webAddress);
+
+  public abstract void setRpcAddress(String rpcAddress);
+
+  public abstract void setServiceAddress(String serviceAddress);
+
+  public abstract void setLifelineAddress(String lifelineAddress);
+
+  public abstract void setIsSafeMode(boolean isSafeMode);
+
+  public abstract void setClusterId(String clusterId);
+
+  public abstract void setBlockPoolId(String blockPoolId);
+
+  public abstract void setState(FederationNamenodeServiceState state);
+
+  public abstract String getNameserviceId();
+
+  public abstract String getNamenodeId();
+
+  public abstract String getClusterId();
+
+  public abstract String getBlockPoolId();
+
+  public abstract String getRpcAddress();
+
+  public abstract String getServiceAddress();
+
+  public abstract String getLifelineAddress();
+
+  public abstract String getWebAddress();
+
+  public abstract boolean getIsSafeMode();
+
+  public abstract FederationNamenodeServiceState getState();
+
+  public abstract void setStats(MembershipStats stats);
+
+  public abstract MembershipStats getStats() throws IOException;
+
+  public abstract void setLastContact(long contact);
+
+  public abstract long getLastContact();
+
+  @Override
+  public boolean like(BaseRecord o) {
+    if (o instanceof MembershipState) {
+      MembershipState other = (MembershipState)o;
+      if (getRouterId() != null &&
+          !getRouterId().equals(other.getRouterId())) {
+        return false;
+      }
+      if (getNameserviceId() != null &&
+          !getNameserviceId().equals(other.getNameserviceId())) {
+        return false;
+      }
+      if (getNamenodeId() != null &&
+          !getNamenodeId().equals(other.getNamenodeId())) {
+        return false;
+      }
+      if (getRpcAddress() != null &&
+          !getRpcAddress().equals(other.getRpcAddress())) {
+        return false;
+      }
+      if (getClusterId() != null &&
+          !getClusterId().equals(other.getClusterId())) {
+        return false;
+      }
+      if (getBlockPoolId() != null &&
+          !getBlockPoolId().equals(other.getBlockPoolId())) {
+        return false;
+      }
+      if (getState() != null &&
+          !getState().equals(other.getState())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getRouterId() + "->" + getNameserviceId() + ":" + getNamenodeId()
+        + ":" + getRpcAddress() + "-" + getState();
+  }
+
+  @Override
+  public SortedMap<String, String> getPrimaryKeys() {
+    SortedMap<String, String> map = new TreeMap<String, String>();
+    map.put("routerId", getRouterId());
+    map.put("nameserviceId", getNameserviceId());
+    map.put("namenodeId", getNamenodeId());
+    return map;
+  }
+
+  /**
+   * Check if the namenode is available.
+   *
+   * @return If the namenode is available.
+   */
+  public boolean isAvailable() {
+    return getState() == ACTIVE;
+  }
+
+  /**
+   * Validates the entry. Throws an IllegalArgementException if the data record
+   * is missing required information.
+   */
+  @Override
+  public boolean validate() {
+    boolean ret = super.validate();
+    if (getNameserviceId() == null || getNameserviceId().length() == 0) {
+      //LOG.error("Invalid registration, no nameservice specified " + this);
+      ret = false;
+    }
+    if (getWebAddress() == null || getWebAddress().length() == 0) {
+      //LOG.error("Invalid registration, no web address specified " + this);
+      ret = false;
+    }
+    if (getRpcAddress() == null || getRpcAddress().length() == 0) {
+      //LOG.error("Invalid registration, no rpc address specified " + this);
+      ret = false;
+    }
+    if (!isBadState() &&
+        (getBlockPoolId().isEmpty() || getBlockPoolId().length() == 0)) {
+      //LOG.error("Invalid registration, no block pool specified " + this);
+      ret = false;
+    }
+    return ret;
+  }
+
+
+  /**
+   * Overrides the cached getBlockPoolId() with an update. The state will be
+   * reset when the cache is flushed
+   *
+   * @param newState Service state of the namenode.
+   */
+  public void overrideState(FederationNamenodeServiceState newState) {
+    this.setState(newState);
+  }
+
+  /**
+   * Sort by nameservice, namenode, and router.
+   *
+   * @param other Another membership to compare to.
+   * @return If this object goes before the parameter.
+   */
+  public int compareNameTo(MembershipState other) {
+    int ret = this.getNameserviceId().compareTo(other.getNameserviceId());
+    if (ret == 0) {
+      ret = this.getNamenodeId().compareTo(other.getNamenodeId());
+    }
+    if (ret == 0) {
+      ret = this.getRouterId().compareTo(other.getRouterId());
+    }
+    return ret;
+  }
+
+  /**
+   * Get the identifier of this namenode registration.
+   * @return Identifier of the namenode.
+   */
+  public String getNamenodeKey() {
+    return getNamenodeKey(this.getNameserviceId(), this.getNamenodeId());
+  }
+
+  /**
+   * Generate the identifier for a Namenode in the HDFS federation.
+   *
+   * @param nsId Nameservice of the Namenode.
+   * @param nnId Namenode within the Nameservice (HA).
+   * @return Namenode identifier within the federation.
+   */
+  public static String getNamenodeKey(String nsId, String nnId) {
+    return nsId + "-" + nnId;
+  }
+
+  /**
+   * Check if the membership is in a bad state (expired or unavailable).
+   * @return If the membership is in a bad state (expired or unavailable).
+   */
+  private boolean isBadState() {
+    return this.getState() == EXPIRED || this.getState() == UNAVAILABLE;
+  }
+
+  @Override
+  public boolean checkExpired(long currentTime) {
+    if (super.checkExpired(currentTime)) {
+      this.setState(EXPIRED);
+      // Commit it
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public long getExpirationMs() {
+    return MembershipState.expirationMs;
+  }
+
+  /**
+   * Set the expiration time for this class.
+   *
+   * @param time Expiration time in milliseconds.
+   */
+  public static void setExpirationMs(long time) {
+    MembershipState.expirationMs = time;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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
new file mode 100644
index 0000000..0bd19d9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.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.federation.store.records;
+
+import java.io.IOException;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * Data schema for storing NN stats in the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.StateStoreService
+ * StateStoreService}.
+ */
+public abstract class MembershipStats extends BaseRecord {
+
+  public static MembershipStats newInstance() throws IOException {
+    MembershipStats record =
+        StateStoreSerializer.newRecord(MembershipStats.class);
+    record.init();
+    return record;
+  }
+
+  public abstract void setTotalSpace(long space);
+
+  public abstract long getTotalSpace();
+
+  public abstract void setAvailableSpace(long space);
+
+  public abstract long getAvailableSpace();
+
+  public abstract void setNumOfFiles(long files);
+
+  public abstract long getNumOfFiles();
+
+  public abstract void setNumOfBlocks(long blocks);
+
+  public abstract long getNumOfBlocks();
+
+  public abstract void setNumOfBlocksMissing(long blocks);
+
+  public abstract long getNumOfBlocksMissing();
+
+  public abstract void setNumOfBlocksPendingReplication(long blocks);
+
+  public abstract long getNumOfBlocksPendingReplication();
+
+  public abstract void setNumOfBlocksUnderReplicated(long blocks);
+
+  public abstract long getNumOfBlocksUnderReplicated();
+
+  public abstract void setNumOfBlocksPendingDeletion(long blocks);
+
+  public abstract long getNumOfBlocksPendingDeletion();
+
+  public abstract void setNumOfActiveDatanodes(int nodes);
+
+  public abstract int getNumOfActiveDatanodes();
+
+  public abstract void setNumOfDeadDatanodes(int nodes);
+
+  public abstract int getNumOfDeadDatanodes();
+
+  public abstract void setNumOfDecommissioningDatanodes(int nodes);
+
+  public abstract int getNumOfDecommissioningDatanodes();
+
+  public abstract void setNumOfDecomActiveDatanodes(int nodes);
+
+  public abstract int getNumOfDecomActiveDatanodes();
+
+  public abstract void setNumOfDecomDeadDatanodes(int nodes);
+
+  public abstract int getNumOfDecomDeadDatanodes();
+
+  @Override
+  public SortedMap<String, String> getPrimaryKeys() {
+    // This record is not stored directly, no key needed
+    SortedMap<String, String> map = new TreeMap<String, String>();
+    return map;
+  }
+
+  @Override
+  public long getExpirationMs() {
+    // This record is not stored directly, no expiration needed
+    return -1;
+  }
+
+  @Override
+  public void setDateModified(long time) {
+    // We don't store this record directly
+  }
+
+  @Override
+  public long getDateModified() {
+    // We don't store this record directly
+    return 0;
+  }
+
+  @Override
+  public void setDateCreated(long time) {
+    // We don't store this record directly
+  }
+
+  @Override
+  public long getDateCreated() {
+    // We don't store this record directly
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatePBImpl.java
new file mode 100644
index 0000000..805c2af
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatePBImpl.java
@@ -0,0 +1,334 @@
+/**
+ * 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.store.records.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipRecordProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipRecordProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipRecordProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipStatsRecordProto;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the MembershipState record.
+ */
+public class MembershipStatePBImpl extends MembershipState implements PBRecord {
+
+  private FederationProtocolPBTranslator<NamenodeMembershipRecordProto, Builder,
+      NamenodeMembershipRecordProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<NamenodeMembershipRecordProto,
+              Builder, NamenodeMembershipRecordProtoOrBuilder>(
+                  NamenodeMembershipRecordProto.class);
+
+  public MembershipStatePBImpl() {
+  }
+
+  public MembershipStatePBImpl(NamenodeMembershipRecordProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public NamenodeMembershipRecordProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public void setRouterId(String routerId) {
+    Builder builder = this.translator.getBuilder();
+    if (routerId == null) {
+      builder.clearRouterId();
+    } else {
+      builder.setRouterId(routerId);
+    }
+  }
+
+  @Override
+  public void setNameserviceId(String nameserviceId) {
+    Builder builder = this.translator.getBuilder();
+    if (nameserviceId == null) {
+      builder.clearNameserviceId();
+    } else {
+      builder.setNameserviceId(nameserviceId);
+    }
+  }
+
+  @Override
+  public void setNamenodeId(String namenodeId) {
+    Builder builder = this.translator.getBuilder();
+    if (namenodeId == null) {
+      builder.clearNamenodeId();
+    } else {
+      builder.setNamenodeId(namenodeId);
+    }
+  }
+
+  @Override
+  public void setWebAddress(String webAddress) {
+    Builder builder = this.translator.getBuilder();
+    if (webAddress == null) {
+      builder.clearWebAddress();
+    } else {
+      builder.setWebAddress(webAddress);
+    }
+  }
+
+  @Override
+  public void setRpcAddress(String rpcAddress) {
+    Builder builder = this.translator.getBuilder();
+    if (rpcAddress == null) {
+      builder.clearRpcAddress();
+    } else {
+      builder.setRpcAddress(rpcAddress);
+    }
+  }
+
+  @Override
+  public void setServiceAddress(String serviceAddress) {
+    this.translator.getBuilder().setServiceAddress(serviceAddress);
+  }
+
+  @Override
+  public void setLifelineAddress(String lifelineAddress) {
+    Builder builder = this.translator.getBuilder();
+    if (lifelineAddress == null) {
+      builder.clearLifelineAddress();
+    } else {
+      builder.setLifelineAddress(lifelineAddress);
+    }
+  }
+
+  @Override
+  public void setIsSafeMode(boolean isSafeMode) {
+    Builder builder = this.translator.getBuilder();
+    builder.setIsSafeMode(isSafeMode);
+  }
+
+  @Override
+  public void setClusterId(String clusterId) {
+    Builder builder = this.translator.getBuilder();
+    if (clusterId == null) {
+      builder.clearClusterId();
+    } else {
+      builder.setClusterId(clusterId);
+    }
+  }
+
+  @Override
+  public void setBlockPoolId(String blockPoolId) {
+    Builder builder = this.translator.getBuilder();
+    if (blockPoolId == null) {
+      builder.clearBlockPoolId();
+    } else {
+      builder.setBlockPoolId(blockPoolId);
+    }
+  }
+
+  @Override
+  public void setState(FederationNamenodeServiceState state) {
+    Builder builder = this.translator.getBuilder();
+    if (state == null) {
+      builder.clearState();
+    } else {
+      builder.setState(state.toString());
+    }
+  }
+
+  @Override
+  public String getRouterId() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasRouterId()) {
+      return null;
+    }
+    return proto.getRouterId();
+  }
+
+  @Override
+  public String getNameserviceId() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasNameserviceId()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getNameserviceId();
+  }
+
+  @Override
+  public String getNamenodeId() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasNamenodeId()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getNamenodeId();
+  }
+
+  @Override
+  public String getClusterId() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasClusterId()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getClusterId();
+  }
+
+  @Override
+  public String getBlockPoolId() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasBlockPoolId()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getBlockPoolId();
+  }
+
+  @Override
+  public String getRpcAddress() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasRpcAddress()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getRpcAddress();
+  }
+
+  @Override
+  public String getServiceAddress() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasServiceAddress()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getServiceAddress();
+  }
+
+  @Override
+  public String getWebAddress() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasWebAddress()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getWebAddress();
+  }
+
+  @Override
+  public String getLifelineAddress() {
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasLifelineAddress()) {
+      return null;
+    }
+    return this.translator.getProtoOrBuilder().getLifelineAddress();
+  }
+
+  @Override
+  public boolean getIsSafeMode() {
+    return this.translator.getProtoOrBuilder().getIsSafeMode();
+  }
+
+  @Override
+  public FederationNamenodeServiceState getState() {
+    FederationNamenodeServiceState ret =
+        FederationNamenodeServiceState.UNAVAILABLE;
+    NamenodeMembershipRecordProtoOrBuilder proto =
+        this.translator.getProtoOrBuilder();
+    if (!proto.hasState()) {
+      return null;
+    }
+    try {
+      ret = FederationNamenodeServiceState.valueOf(proto.getState());
+    } catch (IllegalArgumentException e) {
+      // Ignore this error
+    }
+    return ret;
+  }
+
+  @Override
+  public void setStats(MembershipStats stats) {
+    if (stats instanceof MembershipStatsPBImpl) {
+      MembershipStatsPBImpl statsPB = (MembershipStatsPBImpl)stats;
+      NamenodeMembershipStatsRecordProto statsProto =
+          (NamenodeMembershipStatsRecordProto)statsPB.getProto();
+      this.translator.getBuilder().setStats(statsProto);
+    }
+  }
+
+  @Override
+  public MembershipStats getStats() throws IOException {
+    NamenodeMembershipStatsRecordProto statsProto =
+        this.translator.getProtoOrBuilder().getStats();
+    MembershipStats stats =
+        StateStoreSerializer.newRecord(MembershipStats.class);
+    if (stats instanceof MembershipStatsPBImpl) {
+      MembershipStatsPBImpl statsPB = (MembershipStatsPBImpl)stats;
+      statsPB.setProto(statsProto);
+      return statsPB;
+    } else {
+      throw new IOException("Cannot get stats for the membership");
+    }
+  }
+
+  @Override
+  public void setLastContact(long contact) {
+    this.translator.getBuilder().setLastContact(contact);
+  }
+
+  @Override
+  public long getLastContact() {
+    return this.translator.getProtoOrBuilder().getLastContact();
+  }
+
+  @Override
+  public void setDateModified(long time) {
+    this.translator.getBuilder().setDateModified(time);
+  }
+
+  @Override
+  public long getDateModified() {
+    return this.translator.getProtoOrBuilder().getDateModified();
+  }
+
+  @Override
+  public void setDateCreated(long time) {
+    this.translator.getBuilder().setDateCreated(time);
+  }
+
+  @Override
+  public long getDateCreated() {
+    return this.translator.getProtoOrBuilder().getDateCreated();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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
new file mode 100644
index 0000000..9f0a167
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
@@ -0,0 +1,191 @@
+/**
+ * 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.store.records.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipStatsRecordProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipStatsRecordProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.NamenodeMembershipStatsRecordProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the MembershipStats record.
+ */
+public class MembershipStatsPBImpl extends MembershipStats
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<NamenodeMembershipStatsRecordProto,
+      Builder, NamenodeMembershipStatsRecordProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<NamenodeMembershipStatsRecordProto,
+          Builder, NamenodeMembershipStatsRecordProtoOrBuilder>(
+              NamenodeMembershipStatsRecordProto.class);
+
+  public MembershipStatsPBImpl() {
+  }
+
+  @Override
+  public NamenodeMembershipStatsRecordProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public void setTotalSpace(long space) {
+    this.translator.getBuilder().setTotalSpace(space);
+  }
+
+  @Override
+  public long getTotalSpace() {
+    return this.translator.getProtoOrBuilder().getTotalSpace();
+  }
+
+  @Override
+  public void setAvailableSpace(long space) {
+    this.translator.getBuilder().setAvailableSpace(space);
+  }
+
+  @Override
+  public long getAvailableSpace() {
+    return this.translator.getProtoOrBuilder().getAvailableSpace();
+  }
+
+  @Override
+  public void setNumOfFiles(long files) {
+    this.translator.getBuilder().setNumOfFiles(files);
+  }
+
+  @Override
+  public long getNumOfFiles() {
+    return this.translator.getProtoOrBuilder().getNumOfFiles();
+  }
+
+  @Override
+  public void setNumOfBlocks(long blocks) {
+    this.translator.getBuilder().setNumOfBlocks(blocks);
+  }
+
+  @Override
+  public long getNumOfBlocks() {
+    return this.translator.getProtoOrBuilder().getNumOfBlocks();
+  }
+
+  @Override
+  public void setNumOfBlocksMissing(long blocks) {
+    this.translator.getBuilder().setNumOfBlocksMissing(blocks);
+  }
+
+  @Override
+  public long getNumOfBlocksMissing() {
+    return this.translator.getProtoOrBuilder().getNumOfBlocksMissing();
+  }
+
+  @Override
+  public void setNumOfBlocksPendingReplication(long blocks) {
+    this.translator.getBuilder().setNumOfBlocksPendingReplication(blocks);
+  }
+
+  @Override
+  public long getNumOfBlocksPendingReplication() {
+    return this.translator.getProtoOrBuilder()
+        .getNumOfBlocksPendingReplication();
+  }
+
+  @Override
+  public void setNumOfBlocksUnderReplicated(long blocks) {
+    this.translator.getBuilder().setNumOfBlocksUnderReplicated(blocks);
+  }
+
+  @Override
+  public long getNumOfBlocksUnderReplicated() {
+    return this.translator.getProtoOrBuilder().getNumOfBlocksUnderReplicated();
+  }
+
+  @Override
+  public void setNumOfBlocksPendingDeletion(long blocks) {
+    this.translator.getBuilder().setNumOfBlocksPendingDeletion(blocks);
+  }
+
+  @Override
+  public long getNumOfBlocksPendingDeletion() {
+    return this.translator.getProtoOrBuilder().getNumOfBlocksPendingDeletion();
+  }
+
+  @Override
+  public void setNumOfActiveDatanodes(int nodes) {
+    this.translator.getBuilder().setNumOfActiveDatanodes(nodes);
+  }
+
+  @Override
+  public int getNumOfActiveDatanodes() {
+    return this.translator.getProtoOrBuilder().getNumOfActiveDatanodes();
+  }
+
+  @Override
+  public void setNumOfDeadDatanodes(int nodes) {
+    this.translator.getBuilder().setNumOfDeadDatanodes(nodes);
+  }
+
+  @Override
+  public int getNumOfDeadDatanodes() {
+    return this.translator.getProtoOrBuilder().getNumOfDeadDatanodes();
+  }
+
+  @Override
+  public void setNumOfDecommissioningDatanodes(int nodes) {
+    this.translator.getBuilder().setNumOfDecommissioningDatanodes(nodes);
+  }
+
+  @Override
+  public int getNumOfDecommissioningDatanodes() {
+    return this.translator.getProtoOrBuilder()
+        .getNumOfDecommissioningDatanodes();
+  }
+
+  @Override
+  public void setNumOfDecomActiveDatanodes(int nodes) {
+    this.translator.getBuilder().setNumOfDecomActiveDatanodes(nodes);
+  }
+
+  @Override
+  public int getNumOfDecomActiveDatanodes() {
+    return this.translator.getProtoOrBuilder().getNumOfDecomActiveDatanodes();
+  }
+
+  @Override
+  public void setNumOfDecomDeadDatanodes(int nodes) {
+    this.translator.getBuilder().setNumOfDecomDeadDatanodes(nodes);
+  }
+
+  @Override
+  public int getNumOfDecomDeadDatanodes() {
+    return this.translator.getProtoOrBuilder().getNumOfDecomDeadDatanodes();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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
new file mode 100644
index 0000000..487fe46
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
@@ -0,0 +1,107 @@
+/**
+ * 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 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.federation.protocol.proto";
+option java_outer_classname = "HdfsServerFederationProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+
+/////////////////////////////////////////////////
+// Membership
+/////////////////////////////////////////////////
+
+message NamenodeMembershipStatsRecordProto {
+  optional uint64 totalSpace = 1;
+  optional uint64 availableSpace = 2;
+
+  optional uint64 numOfFiles = 10;
+  optional uint64 numOfBlocks = 11;
+  optional uint64 numOfBlocksMissing = 12;
+  optional uint64 numOfBlocksPendingReplication = 13;
+  optional uint64 numOfBlocksUnderReplicated = 14;
+  optional uint64 numOfBlocksPendingDeletion = 15;
+
+  optional uint32 numOfActiveDatanodes = 20;
+  optional uint32 numOfDeadDatanodes = 21;
+  optional uint32 numOfDecommissioningDatanodes = 22;
+  optional uint32 numOfDecomActiveDatanodes = 23;
+  optional uint32 numOfDecomDeadDatanodes = 24;
+}
+
+message NamenodeMembershipRecordProto {
+  optional uint64 dateCreated = 1;
+  optional uint64 dateModified = 2;
+  optional uint64 lastContact = 3;
+  optional string routerId = 4;
+  optional string nameserviceId = 5;
+  optional string namenodeId = 6;
+  optional string clusterId = 7;
+  optional string blockPoolId = 8;
+  optional string webAddress = 9;
+  optional string rpcAddress = 10;
+  optional string serviceAddress = 11;
+  optional string lifelineAddress = 12;
+  optional string state = 13;
+  optional bool isSafeMode = 14;
+
+  optional NamenodeMembershipStatsRecordProto stats = 15;
+}
+
+message FederationNamespaceInfoProto {
+  optional string blockPoolId = 1;
+  optional string clusterId = 2;
+  optional string nameserviceId = 3;
+}
+
+message GetNamenodeRegistrationsRequestProto {
+  optional NamenodeMembershipRecordProto membership = 1;
+}
+
+message GetNamenodeRegistrationsResponseProto {
+  repeated NamenodeMembershipRecordProto namenodeMemberships = 1;
+}
+
+message GetExpiredRegistrationsRequestProto {
+}
+
+message GetNamespaceInfoRequestProto {
+}
+
+message GetNamespaceInfoResponseProto {
+  repeated FederationNamespaceInfoProto namespaceInfos = 1;
+}
+
+message UpdateNamenodeRegistrationRequestProto {
+  optional string nameserviceId = 1;
+  optional string namenodeId = 2;
+  optional string state = 3;
+}
+
+message UpdateNamenodeRegistrationResponseProto {
+  optional bool status = 1;
+}
+
+message NamenodeHeartbeatRequestProto {
+  optional NamenodeMembershipRecordProto namenodeMembership = 1;
+}
+
+message NamenodeHeartbeatResponseProto {
+  optional bool status = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/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 33fff0f..57a0cd8 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
@@ -4724,7 +4724,7 @@
 
   <property>
     <name>dfs.federation.router.namenode.resolver.client.class</name>
-    <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
+    <value>org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver</value>
     <description>
       Class to resolve the namenode for a subcluster.
     </description>
@@ -4754,4 +4754,20 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.cache.ttl</name>
+    <value>60000</value>
+    <description>
+      How often to refresh the State Store caches in milliseconds.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.store.membership.expiration</name>
+    <value>300000</value>
+    <description>
+      Expiration time in milliseconds for a membership record.
+    </description>
+  </property>
+
 </configuration>


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


[11/29] hadoop git commit: HDFS-10687. Federation Membership State Store internal API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java
new file mode 100644
index 0000000..2d74505
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java
@@ -0,0 +1,284 @@
+/**
+ * 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.resolver;
+
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMESERVICES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.ROUTERS;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.createNamenodeReport;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyException;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.clearRecords;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.newStateStore;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.waitStateStore;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the basic {@link ActiveNamenodeResolver} functionality.
+ */
+public class TestNamenodeResolver {
+
+  private static StateStoreService stateStore;
+  private static ActiveNamenodeResolver namenodeResolver;
+
+  @BeforeClass
+  public static void create() throws Exception {
+
+    Configuration conf = getStateStoreConfiguration();
+
+    // Reduce expirations to 5 seconds
+    conf.setLong(
+        DFSConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS,
+        TimeUnit.SECONDS.toMillis(5));
+
+    stateStore = newStateStore(conf);
+    assertNotNull(stateStore);
+
+    namenodeResolver = new MembershipNamenodeResolver(conf, stateStore);
+    namenodeResolver.setRouterId(ROUTERS[0]);
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    stateStore.stop();
+    stateStore.close();
+  }
+
+  @Before
+  public void setup() throws IOException, InterruptedException {
+    // Wait for state store to connect
+    stateStore.loadDriver();
+    waitStateStore(stateStore, 10000);
+
+    // Clear NN registrations
+    boolean cleared = clearRecords(stateStore, MembershipState.class);
+    assertTrue(cleared);
+  }
+
+  @Test
+  public void testStateStoreDisconnected() throws Exception {
+
+    // Add an entry to the store
+    NamenodeStatusReport report = createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[0], HAServiceState.ACTIVE);
+    assertTrue(namenodeResolver.registerNamenode(report));
+
+    // Close the data store driver
+    stateStore.closeDriver();
+    assertFalse(stateStore.isDriverReady());
+
+    // Flush the caches
+    stateStore.refreshCaches(true);
+
+    // Verify commands fail due to no cached data and no state store
+    // connectivity.
+    List<? extends FederationNamenodeContext> nns =
+        namenodeResolver.getNamenodesForBlockPoolId(NAMESERVICES[0]);
+    assertNull(nns);
+
+    verifyException(namenodeResolver, "registerNamenode",
+        StateStoreUnavailableException.class,
+        new Class[] {NamenodeStatusReport.class}, new Object[] {report});
+  }
+
+  /**
+   * Verify the first registration on the resolver.
+   *
+   * @param nsId Nameservice identifier.
+   * @param nnId Namenode identifier within the nemeservice.
+   * @param resultsCount Number of results expected.
+   * @param state Expected state for the first one.
+   * @throws IOException If we cannot get the namenodes.
+   */
+  private void verifyFirstRegistration(String nsId, String nnId,
+      int resultsCount, FederationNamenodeServiceState state)
+          throws IOException {
+    List<? extends FederationNamenodeContext> namenodes =
+        namenodeResolver.getNamenodesForNameserviceId(nsId);
+    if (resultsCount == 0) {
+      assertNull(namenodes);
+    } else {
+      assertEquals(resultsCount, namenodes.size());
+      if (namenodes.size() > 0) {
+        FederationNamenodeContext namenode = namenodes.get(0);
+        assertEquals(state, namenode.getState());
+        assertEquals(nnId, namenode.getNamenodeId());
+      }
+    }
+  }
+
+  @Test
+  public void testRegistrationExpired()
+      throws InterruptedException, IOException {
+
+    // Populate the state store with a single NN element
+    // 1) ns0:nn0 - Active
+    // Wait for the entry to expire without heartbeating
+    // Verify the NN entry is not accessible once expired.
+    NamenodeStatusReport report = createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[0], HAServiceState.ACTIVE);
+    assertTrue(namenodeResolver.registerNamenode(report));
+
+    // Load cache
+    stateStore.refreshCaches(true);
+
+    // Verify
+    verifyFirstRegistration(
+        NAMESERVICES[0], NAMENODES[0], 1,
+        FederationNamenodeServiceState.ACTIVE);
+
+    // Wait past expiration (set in conf to 5 seconds)
+    Thread.sleep(6000);
+    // Reload cache
+    stateStore.refreshCaches(true);
+
+    // Verify entry is now expired and is no longer in the cache
+    verifyFirstRegistration(
+        NAMESERVICES[0], NAMENODES[0], 0,
+        FederationNamenodeServiceState.ACTIVE);
+
+    // Heartbeat again, updates dateModified
+    assertTrue(namenodeResolver.registerNamenode(report));
+    // Reload cache
+    stateStore.refreshCaches(true);
+
+    // Verify updated entry is marked active again and accessible to RPC server
+    verifyFirstRegistration(
+        NAMESERVICES[0], NAMENODES[0], 1,
+        FederationNamenodeServiceState.ACTIVE);
+  }
+
+  @Test
+  public void testRegistrationNamenodeSelection()
+      throws InterruptedException, IOException {
+
+    // 1) ns0:nn0 - Active
+    // 2) ns0:nn1 - Standby (newest)
+    // Verify the selected entry is the active entry
+    assertTrue(namenodeResolver.registerNamenode(
+        createNamenodeReport(
+            NAMESERVICES[0], NAMENODES[0], HAServiceState.ACTIVE)));
+    Thread.sleep(100);
+    assertTrue(namenodeResolver.registerNamenode(
+        createNamenodeReport(
+            NAMESERVICES[0], NAMENODES[1], HAServiceState.STANDBY)));
+
+    stateStore.refreshCaches(true);
+
+    verifyFirstRegistration(
+        NAMESERVICES[0], NAMENODES[0], 2,
+        FederationNamenodeServiceState.ACTIVE);
+
+    // 1) ns0:nn0 - Expired (stale)
+    // 2) ns0:nn1 - Standby (newest)
+    // Verify the selected entry is the standby entry as the active entry is
+    // stale
+    assertTrue(namenodeResolver.registerNamenode(
+        createNamenodeReport(
+            NAMESERVICES[0], NAMENODES[0], HAServiceState.ACTIVE)));
+
+    // Expire active registration
+    Thread.sleep(6000);
+
+    // Refresh standby registration
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[1], HAServiceState.STANDBY)));
+
+    // Verify that standby is selected (active is now expired)
+    stateStore.refreshCaches(true);
+    verifyFirstRegistration(NAMESERVICES[0], NAMENODES[1], 1,
+        FederationNamenodeServiceState.STANDBY);
+
+    // 1) ns0:nn0 - Active
+    // 2) ns0:nn1 - Unavailable (newest)
+    // Verify the selected entry is the active entry
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[0], HAServiceState.ACTIVE)));
+    Thread.sleep(100);
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[1], null)));
+    stateStore.refreshCaches(true);
+    verifyFirstRegistration(NAMESERVICES[0], NAMENODES[0], 2,
+        FederationNamenodeServiceState.ACTIVE);
+
+    // 1) ns0:nn0 - Unavailable (newest)
+    // 2) ns0:nn1 - Standby
+    // Verify the selected entry is the standby entry
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[1], HAServiceState.STANDBY)));
+    Thread.sleep(1000);
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[0], null)));
+
+    stateStore.refreshCaches(true);
+    verifyFirstRegistration(NAMESERVICES[0], NAMENODES[1], 2,
+        FederationNamenodeServiceState.STANDBY);
+
+    // 1) ns0:nn0 - Active (oldest)
+    // 2) ns0:nn1 - Standby
+    // 3) ns0:nn2 - Active (newest)
+    // Verify the selected entry is the newest active entry
+    assertTrue(namenodeResolver.registerNamenode(
+        createNamenodeReport(NAMESERVICES[0], NAMENODES[0], null)));
+    Thread.sleep(100);
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[1], HAServiceState.STANDBY)));
+    Thread.sleep(100);
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[2], HAServiceState.ACTIVE)));
+
+    stateStore.refreshCaches(true);
+    verifyFirstRegistration(NAMESERVICES[0], NAMENODES[2], 3,
+        FederationNamenodeServiceState.ACTIVE);
+
+    // 1) ns0:nn0 - Standby (oldest)
+    // 2) ns0:nn1 - Standby (newest)
+    // 3) ns0:nn2 - Standby
+    // Verify the selected entry is the newest standby entry
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[0], HAServiceState.STANDBY)));
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[2], HAServiceState.STANDBY)));
+    Thread.sleep(1500);
+    assertTrue(namenodeResolver.registerNamenode(createNamenodeReport(
+        NAMESERVICES[0], NAMENODES[1], HAServiceState.STANDBY)));
+
+    stateStore.refreshCaches(true);
+    verifyFirstRegistration(NAMESERVICES[0], NAMENODES[1], 3,
+        FederationNamenodeServiceState.STANDBY);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
index fc5aebd..598b9cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
@@ -34,9 +34,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileBaseImpl;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
 import org.apache.hadoop.util.Time;
 
 /**
@@ -96,7 +99,7 @@ public final class FederationStateStoreTestUtils {
    * @throws IOException If it cannot create the State Store.
    * @throws InterruptedException If we cannot wait for the store to start.
    */
-  public static StateStoreService getStateStore(
+  public static StateStoreService newStateStore(
       Configuration configuration) throws IOException, InterruptedException {
 
     StateStoreService stateStore = new StateStoreService();
@@ -205,6 +208,7 @@ public final class FederationStateStoreTestUtils {
     if (!synchronizeRecords(store, emptyList, recordClass)) {
       return false;
     }
+    store.refreshCaches(true);
     return true;
   }
 
@@ -229,4 +233,21 @@ public final class FederationStateStoreTestUtils {
     }
     return false;
   }
+
+  public static MembershipState createMockRegistrationForNamenode(
+      String nameserviceId, String namenodeId,
+      FederationNamenodeServiceState state) throws IOException {
+    MembershipState entry = MembershipState.newInstance(
+        "routerId", nameserviceId, namenodeId, "clusterId", "test",
+        "0.0.0.0:0", "0.0.0.0:0", "0.0.0.0:0", "0.0.0.0:0", state, false);
+    MembershipStats stats = MembershipStats.newInstance();
+    stats.setNumOfActiveDatanodes(100);
+    stats.setNumOfDeadDatanodes(10);
+    stats.setNumOfDecommissioningDatanodes(20);
+    stats.setNumOfDecomActiveDatanodes(15);
+    stats.setNumOfDecomDeadDatanodes(5);
+    stats.setNumOfBlocks(10);
+    entry.setStats(stats);
+    return entry;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreBase.java
new file mode 100644
index 0000000..7f6704e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreBase.java
@@ -0,0 +1,81 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.newStateStore;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.waitStateStore;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+/**
+ * Test the basic {@link StateStoreService} {@link MountTableStore}
+ * functionality.
+ */
+public class TestStateStoreBase {
+
+  private static StateStoreService stateStore;
+  private static Configuration conf;
+
+  protected static StateStoreService getStateStore() {
+    return stateStore;
+  }
+
+  protected static Configuration getConf() {
+    return conf;
+  }
+
+  @BeforeClass
+  public static void createBase() throws IOException, InterruptedException {
+
+    conf = getStateStoreConfiguration();
+
+    // Disable auto-reconnect to data store
+    conf.setLong(DFSConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS,
+        TimeUnit.HOURS.toMillis(1));
+  }
+
+  @AfterClass
+  public static void destroyBase() throws Exception {
+    if (stateStore != null) {
+      stateStore.stop();
+      stateStore.close();
+      stateStore = null;
+    }
+  }
+
+  @Before
+  public void setupBase() throws IOException, InterruptedException,
+      InstantiationException, IllegalAccessException {
+    if (stateStore == null) {
+      stateStore = newStateStore(conf);
+      assertNotNull(stateStore);
+    }
+    // Wait for state store to connect
+    stateStore.loadDriver();
+    waitStateStore(stateStore, TimeUnit.SECONDS.toMillis(10));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java
new file mode 100644
index 0000000..26f081b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java
@@ -0,0 +1,463 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMESERVICES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.ROUTERS;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyException;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.clearRecords;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.createMockRegistrationForNamenode;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.synchronizeRecords;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.util.Time;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the basic {@link MembershipStore} membership functionality.
+ */
+public class TestStateStoreMembershipState extends TestStateStoreBase {
+
+  private static MembershipStore membershipStore;
+
+  @BeforeClass
+  public static void create() {
+    // Reduce expirations to 5 seconds
+    getConf().setLong(
+        DFSConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS,
+        TimeUnit.SECONDS.toMillis(5));
+  }
+
+  @Before
+  public void setup() throws IOException, InterruptedException {
+
+    membershipStore =
+        getStateStore().getRegisteredRecordStore(MembershipStore.class);
+
+    // Clear NN registrations
+    assertTrue(clearRecords(getStateStore(), MembershipState.class));
+  }
+
+  @Test
+  public void testNamenodeStateOverride() throws Exception {
+    // Populate the state store
+    // 1) ns0:nn0 - Standby
+    String ns = "ns0";
+    String nn = "nn0";
+    MembershipState report = createRegistration(
+        ns, nn, ROUTERS[1], FederationNamenodeServiceState.STANDBY);
+    assertTrue(namenodeHeartbeat(report));
+
+    // Load data into cache and calculate quorum
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+
+    MembershipState existingState = getNamenodeRegistration(ns, nn);
+    assertEquals(
+        FederationNamenodeServiceState.STANDBY, existingState.getState());
+
+    // Override cache
+    UpdateNamenodeRegistrationRequest request =
+        UpdateNamenodeRegistrationRequest.newInstance(
+            ns, nn, FederationNamenodeServiceState.ACTIVE);
+    assertTrue(membershipStore.updateNamenodeRegistration(request).getResult());
+
+    MembershipState newState = getNamenodeRegistration(ns, nn);
+    assertEquals(FederationNamenodeServiceState.ACTIVE, newState.getState());
+  }
+
+  @Test
+  public void testStateStoreDisconnected() throws Exception {
+
+    // Close the data store driver
+    getStateStore().closeDriver();
+    assertFalse(getStateStore().isDriverReady());
+
+    NamenodeHeartbeatRequest hbRequest = NamenodeHeartbeatRequest.newInstance();
+    hbRequest.setNamenodeMembership(
+        createMockRegistrationForNamenode(
+            "test", "test", FederationNamenodeServiceState.UNAVAILABLE));
+    verifyException(membershipStore, "namenodeHeartbeat",
+        StateStoreUnavailableException.class,
+        new Class[] {NamenodeHeartbeatRequest.class},
+        new Object[] {hbRequest });
+
+    // Information from cache, no exception should be triggered for these
+    // TODO - should cached info expire at some point?
+    GetNamenodeRegistrationsRequest getRequest =
+        GetNamenodeRegistrationsRequest.newInstance();
+    verifyException(membershipStore,
+        "getNamenodeRegistrations", null,
+        new Class[] {GetNamenodeRegistrationsRequest.class},
+        new Object[] {getRequest});
+
+    verifyException(membershipStore,
+        "getExpiredNamenodeRegistrations", null,
+        new Class[] {GetNamenodeRegistrationsRequest.class},
+        new Object[] {getRequest});
+
+    UpdateNamenodeRegistrationRequest overrideRequest =
+        UpdateNamenodeRegistrationRequest.newInstance();
+    verifyException(membershipStore,
+        "updateNamenodeRegistration", null,
+        new Class[] {UpdateNamenodeRegistrationRequest.class},
+        new Object[] {overrideRequest});
+  }
+
+  private void registerAndLoadRegistrations(
+      List<MembershipState> registrationList) throws IOException {
+    // Populate
+    assertTrue(synchronizeRecords(
+        getStateStore(), registrationList, MembershipState.class));
+
+    // Load into cache
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+  }
+
+  private MembershipState createRegistration(String ns, String nn,
+      String router, FederationNamenodeServiceState state) throws IOException {
+    MembershipState record = MembershipState.newInstance(
+        router, ns,
+        nn, "testcluster", "testblock-" + ns, "testrpc-"+ ns + nn,
+        "testservice-"+ ns + nn, "testlifeline-"+ ns + nn,
+        "testweb-" + ns + nn, state, false);
+    return record;
+  }
+
+  @Test
+  public void testRegistrationMajorityQuorum()
+      throws InterruptedException, IOException {
+
+    // Populate the state store with a set of non-matching elements
+    // 1) ns0:nn0 - Standby (newest)
+    // 2) ns0:nn0 - Active (oldest)
+    // 3) ns0:nn0 - Active (2nd oldest)
+    // 4) ns0:nn0 - Active (3nd oldest element, newest active element)
+    // Verify the selected entry is the newest majority opinion (4)
+    String ns = "ns0";
+    String nn = "nn0";
+
+    // Active - oldest
+    MembershipState report = createRegistration(
+        ns, nn, ROUTERS[1], FederationNamenodeServiceState.ACTIVE);
+    assertTrue(namenodeHeartbeat(report));
+    Thread.sleep(1000);
+
+    // Active - 2nd oldest
+    report = createRegistration(
+        ns, nn, ROUTERS[2], FederationNamenodeServiceState.ACTIVE);
+    assertTrue(namenodeHeartbeat(report));
+    Thread.sleep(1000);
+
+    // Active - 3rd oldest, newest active element
+    report = createRegistration(
+        ns, nn, ROUTERS[3], FederationNamenodeServiceState.ACTIVE);
+    assertTrue(namenodeHeartbeat(report));
+
+    // standby - newest overall
+    report = createRegistration(
+        ns, nn, ROUTERS[0], FederationNamenodeServiceState.STANDBY);
+    assertTrue(namenodeHeartbeat(report));
+
+    // Load and calculate quorum
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+
+    // Verify quorum entry
+    MembershipState quorumEntry = getNamenodeRegistration(
+        report.getNameserviceId(), report.getNamenodeId());
+    assertNotNull(quorumEntry);
+    assertEquals(quorumEntry.getRouterId(), ROUTERS[3]);
+  }
+
+  @Test
+  public void testRegistrationQuorumExcludesExpired()
+      throws InterruptedException, IOException {
+
+    // Populate the state store with some expired entries and verify the expired
+    // entries are ignored.
+    // 1) ns0:nn0 - Active
+    // 2) ns0:nn0 - Expired
+    // 3) ns0:nn0 - Expired
+    // 4) ns0:nn0 - Expired
+    // Verify the selected entry is the active entry
+    List<MembershipState> registrationList = new ArrayList<>();
+    String ns = "ns0";
+    String nn = "nn0";
+    String rpcAddress = "testrpcaddress";
+    String serviceAddress = "testserviceaddress";
+    String lifelineAddress = "testlifelineaddress";
+    String blockPoolId = "testblockpool";
+    String clusterId = "testcluster";
+    String webAddress = "testwebaddress";
+    boolean safemode = false;
+
+    // Active
+    MembershipState record = MembershipState.newInstance(
+        ROUTERS[0], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.ACTIVE, safemode);
+    registrationList.add(record);
+
+    // Expired
+    record = MembershipState.newInstance(
+        ROUTERS[1], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    registrationList.add(record);
+
+    // Expired
+    record = MembershipState.newInstance(
+        ROUTERS[2], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    registrationList.add(record);
+
+    // Expired
+    record = MembershipState.newInstance(
+        ROUTERS[3], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    registrationList.add(record);
+    registerAndLoadRegistrations(registrationList);
+
+    // Verify quorum entry chooses active membership
+    MembershipState quorumEntry = getNamenodeRegistration(
+        record.getNameserviceId(), record.getNamenodeId());
+    assertNotNull(quorumEntry);
+    assertEquals(ROUTERS[0], quorumEntry.getRouterId());
+  }
+
+  @Test
+  public void testRegistrationQuorumAllExpired() throws IOException {
+
+    // 1) ns0:nn0 - Expired (oldest)
+    // 2) ns0:nn0 - Expired
+    // 3) ns0:nn0 - Expired
+    // 4) ns0:nn0 - Expired
+    // Verify no entry is either selected or cached
+    List<MembershipState> registrationList = new ArrayList<>();
+    String ns = NAMESERVICES[0];
+    String nn = NAMENODES[0];
+    String rpcAddress = "testrpcaddress";
+    String serviceAddress = "testserviceaddress";
+    String lifelineAddress = "testlifelineaddress";
+    String blockPoolId = "testblockpool";
+    String clusterId = "testcluster";
+    String webAddress = "testwebaddress";
+    boolean safemode = false;
+    long startingTime = Time.now();
+
+    // Expired
+    MembershipState record = MembershipState.newInstance(
+        ROUTERS[0], ns, nn, clusterId, blockPoolId,
+        rpcAddress, webAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    record.setDateModified(startingTime - 10000);
+    registrationList.add(record);
+
+    // Expired
+    record = MembershipState.newInstance(
+        ROUTERS[1], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    record.setDateModified(startingTime);
+    registrationList.add(record);
+
+    // Expired
+    record = MembershipState.newInstance(
+        ROUTERS[2], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    record.setDateModified(startingTime);
+    registrationList.add(record);
+
+    // Expired
+    record = MembershipState.newInstance(
+        ROUTERS[3], ns, nn, clusterId, blockPoolId,
+        rpcAddress, serviceAddress, lifelineAddress, webAddress,
+        FederationNamenodeServiceState.EXPIRED, safemode);
+    record.setDateModified(startingTime);
+    registrationList.add(record);
+
+    registerAndLoadRegistrations(registrationList);
+
+    // Verify no entry is found for this nameservice
+    assertNull(getNamenodeRegistration(
+        record.getNameserviceId(), record.getNamenodeId()));
+  }
+
+  @Test
+  public void testRegistrationNoQuorum()
+      throws InterruptedException, IOException {
+
+    // Populate the state store with a set of non-matching elements
+    // 1) ns0:nn0 - Standby (newest)
+    // 2) ns0:nn0 - Standby (oldest)
+    // 3) ns0:nn0 - Active (2nd oldest)
+    // 4) ns0:nn0 - Active (3nd oldest element, newest active element)
+    // Verify the selected entry is the newest entry (1)
+    MembershipState report1 = createRegistration(
+        NAMESERVICES[0], NAMENODES[0], ROUTERS[1],
+        FederationNamenodeServiceState.STANDBY);
+    assertTrue(namenodeHeartbeat(report1));
+    Thread.sleep(100);
+    MembershipState report2 = createRegistration(
+        NAMESERVICES[0], NAMENODES[0], ROUTERS[2],
+        FederationNamenodeServiceState.ACTIVE);
+    assertTrue(namenodeHeartbeat(report2));
+    Thread.sleep(100);
+    MembershipState report3 = createRegistration(
+        NAMESERVICES[0], NAMENODES[0], ROUTERS[3],
+        FederationNamenodeServiceState.ACTIVE);
+    assertTrue(namenodeHeartbeat(report3));
+    Thread.sleep(100);
+    MembershipState report4 = createRegistration(
+        NAMESERVICES[0], NAMENODES[0], ROUTERS[0],
+        FederationNamenodeServiceState.STANDBY);
+    assertTrue(namenodeHeartbeat(report4));
+
+    // Load and calculate quorum
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+
+    // Verify quorum entry uses the newest data, even though it is standby
+    MembershipState quorumEntry = getNamenodeRegistration(
+        report1.getNameserviceId(), report1.getNamenodeId());
+    assertNotNull(quorumEntry);
+    assertEquals(ROUTERS[0], quorumEntry.getRouterId());
+    assertEquals(
+        FederationNamenodeServiceState.STANDBY, quorumEntry.getState());
+  }
+
+  @Test
+  public void testRegistrationExpired()
+      throws InterruptedException, IOException {
+
+    // Populate the state store with a single NN element
+    // 1) ns0:nn0 - Active
+    // Wait for the entry to expire without heartbeating
+    // Verify the NN entry is populated as EXPIRED internally in the state store
+
+    MembershipState report = createRegistration(
+        NAMESERVICES[0], NAMENODES[0], ROUTERS[0],
+        FederationNamenodeServiceState.ACTIVE);
+    assertTrue(namenodeHeartbeat(report));
+
+    // Load cache
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+
+    // Verify quorum and entry
+    MembershipState quorumEntry = getNamenodeRegistration(
+        report.getNameserviceId(), report.getNamenodeId());
+    assertNotNull(quorumEntry);
+    assertEquals(ROUTERS[0], quorumEntry.getRouterId());
+    assertEquals(FederationNamenodeServiceState.ACTIVE, quorumEntry.getState());
+
+    // Wait past expiration (set in conf to 5 seconds)
+    Thread.sleep(6000);
+    // Reload cache
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+
+    // Verify entry is now expired and is no longer in the cache
+    quorumEntry = getNamenodeRegistration(NAMESERVICES[0], NAMENODES[0]);
+    assertNull(quorumEntry);
+
+    // Verify entry is now expired and can't be used by RPC service
+    quorumEntry = getNamenodeRegistration(
+        report.getNameserviceId(), report.getNamenodeId());
+    assertNull(quorumEntry);
+
+    // Heartbeat again, updates dateModified
+    assertTrue(namenodeHeartbeat(report));
+    // Reload cache
+    assertTrue(getStateStore().loadCache(MembershipStore.class, true));
+
+    // Verify updated entry marked as active and is accessible to RPC server
+    quorumEntry = getNamenodeRegistration(
+        report.getNameserviceId(), report.getNamenodeId());
+    assertNotNull(quorumEntry);
+    assertEquals(ROUTERS[0], quorumEntry.getRouterId());
+    assertEquals(FederationNamenodeServiceState.ACTIVE, quorumEntry.getState());
+  }
+
+  /**
+   * Get a single namenode membership record from the store.
+   *
+   * @param nsId The HDFS nameservice ID to search for
+   * @param nnId The HDFS namenode ID to search for
+   * @return The single NamenodeMembershipRecord that matches the query or null
+   *         if not found.
+   * @throws IOException if the query could not be executed.
+   */
+  private MembershipState getNamenodeRegistration(
+      final String nsId, final String nnId) throws IOException {
+
+    MembershipState partial = MembershipState.newInstance();
+    partial.setNameserviceId(nsId);
+    partial.setNamenodeId(nnId);
+    GetNamenodeRegistrationsRequest request =
+        GetNamenodeRegistrationsRequest.newInstance(partial);
+    GetNamenodeRegistrationsResponse response =
+        membershipStore.getNamenodeRegistrations(request);
+
+    List<MembershipState> results = response.getNamenodeMemberships();
+    if (results != null && results.size() == 1) {
+      MembershipState record = results.get(0);
+      return record;
+    }
+    return null;
+  }
+
+  /**
+   * Register a namenode heartbeat with the state store.
+   *
+   * @param store FederationMembershipStateStore instance to retrieve the
+   *          membership data records.
+   * @param namenode A fully populated namenode membership record to be
+   *          committed to the data store.
+   * @return True if successful, false otherwise.
+   * @throws IOException if the state store query could not be performed.
+   */
+  private boolean namenodeHeartbeat(MembershipState namenode)
+      throws IOException {
+
+    NamenodeHeartbeatRequest request =
+        NamenodeHeartbeatRequest.newInstance(namenode);
+    NamenodeHeartbeatResponse response =
+        membershipStore.namenodeHeartbeat(request);
+    return response.getResult();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
index 7f0b36a..dc51ee9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
@@ -31,11 +31,14 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
 import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
 import org.apache.hadoop.hdfs.server.federation.store.records.Query;
 import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
 import org.junit.AfterClass;
@@ -54,6 +57,8 @@ public class TestStateStoreDriverBase {
   private static StateStoreService stateStore;
   private static Configuration conf;
 
+  private static final Random RANDOM = new Random();
+
 
   /**
    * Get the State Store driver.
@@ -78,29 +83,47 @@ public class TestStateStoreDriverBase {
    */
   public static void getStateStore(Configuration config) throws Exception {
     conf = config;
-    stateStore = FederationStateStoreTestUtils.getStateStore(conf);
+    stateStore = FederationStateStoreTestUtils.newStateStore(conf);
+  }
+
+  private String generateRandomString() {
+    String randomString = "/randomString-" + RANDOM.nextInt();
+    return randomString;
+  }
+
+  @SuppressWarnings("rawtypes")
+  private <T extends Enum> T generateRandomEnum(Class<T> enumClass) {
+    int x = RANDOM.nextInt(enumClass.getEnumConstants().length);
+    T data = enumClass.getEnumConstants()[x];
+    return data;
   }
 
+  @SuppressWarnings("unchecked")
   private <T extends BaseRecord> T generateFakeRecord(Class<T> recordClass)
       throws IllegalArgumentException, IllegalAccessException, IOException {
 
-    // TODO add record
+    if (recordClass == MembershipState.class) {
+      return (T) MembershipState.newInstance(generateRandomString(),
+          generateRandomString(), generateRandomString(),
+          generateRandomString(), generateRandomString(),
+          generateRandomString(), generateRandomString(),
+          generateRandomString(), generateRandomString(),
+          generateRandomEnum(FederationNamenodeServiceState.class), false);
+    }
+
     return null;
   }
 
   /**
    * Validate if a record is the same.
    *
-   * @param original
-   * @param committed
+   * @param original Original record.
+   * @param committed Committed record.
    * @param assertEquals Assert if the records are equal or just return.
-   * @return
-   * @throws IllegalArgumentException
-   * @throws IllegalAccessException
+   * @return If the record is successfully validated.
    */
   private boolean validateRecord(
-      BaseRecord original, BaseRecord committed, boolean assertEquals)
-          throws IllegalArgumentException, IllegalAccessException {
+      BaseRecord original, BaseRecord committed, boolean assertEquals) {
 
     boolean ret = true;
 
@@ -131,7 +154,7 @@ public class TestStateStoreDriverBase {
   }
 
   public static void removeAll(StateStoreDriver driver) throws IOException {
-    // TODO add records to remove
+    driver.removeAll(MembershipState.class);
   }
 
   public <T extends BaseRecord> void testInsert(
@@ -139,17 +162,20 @@ public class TestStateStoreDriverBase {
           throws IllegalArgumentException, IllegalAccessException, IOException {
 
     assertTrue(driver.removeAll(recordClass));
-    QueryResult<T> records = driver.get(recordClass);
-    assertTrue(records.getRecords().isEmpty());
+    QueryResult<T> queryResult0 = driver.get(recordClass);
+    List<T> records0 = queryResult0.getRecords();
+    assertTrue(records0.isEmpty());
 
     // Insert single
     BaseRecord record = generateFakeRecord(recordClass);
     driver.put(record, true, false);
 
     // Verify
-    records = driver.get(recordClass);
-    assertEquals(1, records.getRecords().size());
-    validateRecord(record, records.getRecords().get(0), true);
+    QueryResult<T> queryResult1 = driver.get(recordClass);
+    List<T> records1 = queryResult1.getRecords();
+    assertEquals(1, records1.size());
+    T record0 = records1.get(0);
+    validateRecord(record, record0, true);
 
     // Insert multiple
     List<T> insertList = new ArrayList<>();
@@ -160,8 +186,9 @@ public class TestStateStoreDriverBase {
     driver.putAll(insertList, true, false);
 
     // Verify
-    records = driver.get(recordClass);
-    assertEquals(11, records.getRecords().size());
+    QueryResult<T> queryResult2 = driver.get(recordClass);
+    List<T> records2 = queryResult2.getRecords();
+    assertEquals(11, records2.size());
   }
 
   public <T extends BaseRecord> void testFetchErrors(StateStoreDriver driver,
@@ -319,23 +346,23 @@ public class TestStateStoreDriverBase {
 
   public void testInsert(StateStoreDriver driver)
       throws IllegalArgumentException, IllegalAccessException, IOException {
-    // TODO add records
+    testInsert(driver, MembershipState.class);
   }
 
   public void testPut(StateStoreDriver driver)
       throws IllegalArgumentException, ReflectiveOperationException,
       IOException, SecurityException {
-    // TODO add records
+    testPut(driver, MembershipState.class);
   }
 
   public void testRemove(StateStoreDriver driver)
       throws IllegalArgumentException, IllegalAccessException, IOException {
-    // TODO add records
+    testRemove(driver, MembershipState.class);
   }
 
   public void testFetchErrors(StateStoreDriver driver)
       throws IllegalArgumentException, IllegalAccessException, IOException {
-    // TODO add records
+    testFetchErrors(driver, MembershipState.class);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79fce5e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMembershipState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMembershipState.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMembershipState.java
new file mode 100644
index 0000000..d922414
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMembershipState.java
@@ -0,0 +1,129 @@
+/**
+ * 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.store.records;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.junit.Test;
+
+/**
+ * Test the Membership State records.
+ */
+public class TestMembershipState {
+
+  private static final String ROUTER = "router";
+  private static final String NAMESERVICE = "nameservice";
+  private static final String NAMENODE = "namenode";
+  private static final String CLUSTER_ID = "cluster";
+  private static final String BLOCKPOOL_ID = "blockpool";
+  private static final String RPC_ADDRESS = "rpcaddress";
+  private static final String SERVICE_ADDRESS = "serviceaddress";
+  private static final String LIFELINE_ADDRESS = "lifelineaddress";
+  private static final String WEB_ADDRESS = "webaddress";
+  private static final boolean SAFE_MODE = false;
+
+  private static final long DATE_CREATED = 100;
+  private static final long DATE_MODIFIED = 200;
+
+  private static final long NUM_BLOCKS = 300;
+  private static final long NUM_FILES = 400;
+  private static final int NUM_DEAD = 500;
+  private static final int NUM_ACTIVE = 600;
+  private static final int NUM_DECOM = 700;
+  private static final int NUM_DECOM_ACTIVE = 800;
+  private static final int NUM_DECOM_DEAD = 900;
+  private static final long NUM_BLOCK_MISSING = 1000;
+
+  private static final long TOTAL_SPACE = 1100;
+  private static final long AVAILABLE_SPACE = 1200;
+
+  private static final FederationNamenodeServiceState STATE =
+      FederationNamenodeServiceState.ACTIVE;
+
+  private MembershipState createRecord() throws IOException {
+
+    MembershipState record = MembershipState.newInstance(
+        ROUTER, NAMESERVICE, NAMENODE, CLUSTER_ID,
+        BLOCKPOOL_ID, RPC_ADDRESS, SERVICE_ADDRESS, LIFELINE_ADDRESS,
+        WEB_ADDRESS, STATE, SAFE_MODE);
+    record.setDateCreated(DATE_CREATED);
+    record.setDateModified(DATE_MODIFIED);
+
+    MembershipStats stats = MembershipStats.newInstance();
+    stats.setNumOfBlocks(NUM_BLOCKS);
+    stats.setNumOfFiles(NUM_FILES);
+    stats.setNumOfActiveDatanodes(NUM_ACTIVE);
+    stats.setNumOfDeadDatanodes(NUM_DEAD);
+    stats.setNumOfDecommissioningDatanodes(NUM_DECOM);
+    stats.setNumOfDecomActiveDatanodes(NUM_DECOM_ACTIVE);
+    stats.setNumOfDecomDeadDatanodes(NUM_DECOM_DEAD);
+    stats.setNumOfBlocksMissing(NUM_BLOCK_MISSING);
+    stats.setTotalSpace(TOTAL_SPACE);
+    stats.setAvailableSpace(AVAILABLE_SPACE);
+    record.setStats(stats);
+    return record;
+  }
+
+  private void validateRecord(MembershipState record) throws IOException {
+
+    assertEquals(ROUTER, record.getRouterId());
+    assertEquals(NAMESERVICE, record.getNameserviceId());
+    assertEquals(CLUSTER_ID, record.getClusterId());
+    assertEquals(BLOCKPOOL_ID, record.getBlockPoolId());
+    assertEquals(RPC_ADDRESS, record.getRpcAddress());
+    assertEquals(WEB_ADDRESS, record.getWebAddress());
+    assertEquals(STATE, record.getState());
+    assertEquals(SAFE_MODE, record.getIsSafeMode());
+    assertEquals(DATE_CREATED, record.getDateCreated());
+    assertEquals(DATE_MODIFIED, record.getDateModified());
+
+    MembershipStats stats = record.getStats();
+    assertEquals(NUM_BLOCKS, stats.getNumOfBlocks());
+    assertEquals(NUM_FILES, stats.getNumOfFiles());
+    assertEquals(NUM_ACTIVE, stats.getNumOfActiveDatanodes());
+    assertEquals(NUM_DEAD, stats.getNumOfDeadDatanodes());
+    assertEquals(NUM_DECOM, stats.getNumOfDecommissioningDatanodes());
+    assertEquals(NUM_DECOM_ACTIVE, stats.getNumOfDecomActiveDatanodes());
+    assertEquals(NUM_DECOM_DEAD, stats.getNumOfDecomDeadDatanodes());
+    assertEquals(TOTAL_SPACE, stats.getTotalSpace());
+    assertEquals(AVAILABLE_SPACE, stats.getAvailableSpace());
+  }
+
+  @Test
+  public void testGetterSetter() throws IOException {
+    MembershipState record = createRecord();
+    validateRecord(record);
+  }
+
+  @Test
+  public void testSerialization() throws IOException {
+
+    MembershipState record = createRecord();
+
+    StateStoreSerializer serializer = StateStoreSerializer.getSerializer();
+    String serializedString = serializer.serializeString(record);
+    MembershipState newRecord =
+        serializer.deserialize(serializedString, MembershipState.class);
+
+    validateRecord(newRecord);
+  }
+}
\ 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


[23/29] hadoop git commit: HDFS-10630. Federation State Store FS Implementation. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
new file mode 100644
index 0000000..7f0b36a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
@@ -0,0 +1,483 @@
+/**
+ * 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.store.driver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.junit.AfterClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base tests for the driver. The particular implementations will use this to
+ * test their functionality.
+ */
+public class TestStateStoreDriverBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestStateStoreDriverBase.class);
+
+  private static StateStoreService stateStore;
+  private static Configuration conf;
+
+
+  /**
+   * Get the State Store driver.
+   * @return State Store driver.
+   */
+  protected StateStoreDriver getStateStoreDriver() {
+    return stateStore.getDriver();
+  }
+
+  @AfterClass
+  public static void tearDownCluster() {
+    if (stateStore != null) {
+      stateStore.stop();
+    }
+  }
+
+  /**
+   * Get a new State Store using this configuration.
+   *
+   * @param config Configuration for the State Store.
+   * @throws Exception If we cannot get the State Store.
+   */
+  public static void getStateStore(Configuration config) throws Exception {
+    conf = config;
+    stateStore = FederationStateStoreTestUtils.getStateStore(conf);
+  }
+
+  private <T extends BaseRecord> T generateFakeRecord(Class<T> recordClass)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+
+    // TODO add record
+    return null;
+  }
+
+  /**
+   * Validate if a record is the same.
+   *
+   * @param original
+   * @param committed
+   * @param assertEquals Assert if the records are equal or just return.
+   * @return
+   * @throws IllegalArgumentException
+   * @throws IllegalAccessException
+   */
+  private boolean validateRecord(
+      BaseRecord original, BaseRecord committed, boolean assertEquals)
+          throws IllegalArgumentException, IllegalAccessException {
+
+    boolean ret = true;
+
+    Map<String, Class<?>> fields = getFields(original);
+    for (String key : fields.keySet()) {
+      if (key.equals("dateModified") ||
+          key.equals("dateCreated") ||
+          key.equals("proto")) {
+        // Fields are updated/set on commit and fetch and may not match
+        // the fields that are initialized in a non-committed object.
+        continue;
+      }
+      Object data1 = getField(original, key);
+      Object data2 = getField(committed, key);
+      if (assertEquals) {
+        assertEquals("Field " + key + " does not match", data1, data2);
+      } else if (!data1.equals(data2)) {
+        ret = false;
+      }
+    }
+
+    long now = stateStore.getDriver().getTime();
+    assertTrue(
+        committed.getDateCreated() <= now && committed.getDateCreated() > 0);
+    assertTrue(committed.getDateModified() >= committed.getDateCreated());
+
+    return ret;
+  }
+
+  public static void removeAll(StateStoreDriver driver) throws IOException {
+    // TODO add records to remove
+  }
+
+  public <T extends BaseRecord> void testInsert(
+      StateStoreDriver driver, Class<T> recordClass)
+          throws IllegalArgumentException, IllegalAccessException, IOException {
+
+    assertTrue(driver.removeAll(recordClass));
+    QueryResult<T> records = driver.get(recordClass);
+    assertTrue(records.getRecords().isEmpty());
+
+    // Insert single
+    BaseRecord record = generateFakeRecord(recordClass);
+    driver.put(record, true, false);
+
+    // Verify
+    records = driver.get(recordClass);
+    assertEquals(1, records.getRecords().size());
+    validateRecord(record, records.getRecords().get(0), true);
+
+    // Insert multiple
+    List<T> insertList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      T newRecord = generateFakeRecord(recordClass);
+      insertList.add(newRecord);
+    }
+    driver.putAll(insertList, true, false);
+
+    // Verify
+    records = driver.get(recordClass);
+    assertEquals(11, records.getRecords().size());
+  }
+
+  public <T extends BaseRecord> void testFetchErrors(StateStoreDriver driver,
+      Class<T> clazz) throws IllegalAccessException, IOException {
+
+    // Fetch empty list
+    driver.removeAll(clazz);
+    QueryResult<T> result0 = driver.get(clazz);
+    assertNotNull(result0);
+    List<T> records0 = result0.getRecords();
+    assertEquals(records0.size(), 0);
+
+    // Insert single
+    BaseRecord record = generateFakeRecord(clazz);
+    assertTrue(driver.put(record, true, false));
+
+    // Verify
+    QueryResult<T> result1 = driver.get(clazz);
+    List<T> records1 = result1.getRecords();
+    assertEquals(1, records1.size());
+    validateRecord(record, records1.get(0), true);
+
+    // Test fetch single object with a bad query
+    final T fakeRecord = generateFakeRecord(clazz);
+    final Query<T> query = new Query<T>(fakeRecord);
+    T getRecord = driver.get(clazz, query);
+    assertNull(getRecord);
+
+    // Test fetch multiple objects does not exist returns empty list
+    assertEquals(driver.getMultiple(clazz, query).size(), 0);
+  }
+
+  public <T extends BaseRecord> void testPut(
+      StateStoreDriver driver, Class<T> clazz)
+          throws IllegalArgumentException, ReflectiveOperationException,
+          IOException, SecurityException {
+
+    driver.removeAll(clazz);
+    QueryResult<T> records = driver.get(clazz);
+    assertTrue(records.getRecords().isEmpty());
+
+    // Insert multiple
+    List<T> insertList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      T newRecord = generateFakeRecord(clazz);
+      insertList.add(newRecord);
+    }
+
+    // Verify
+    assertTrue(driver.putAll(insertList, false, true));
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 10);
+
+    // Generate a new record with the same PK fields as an existing record
+    BaseRecord updatedRecord = generateFakeRecord(clazz);
+    BaseRecord existingRecord = records.getRecords().get(0);
+    Map<String, String> primaryKeys = existingRecord.getPrimaryKeys();
+    for (Entry<String, String> entry : primaryKeys.entrySet()) {
+      String key = entry.getKey();
+      String value = entry.getValue();
+      Class<?> fieldType = getFieldType(existingRecord, key);
+      Object field = fromString(value, fieldType);
+      assertTrue(setField(updatedRecord, key, field));
+    }
+
+    // Attempt an update of an existing entry, but it is not allowed.
+    assertFalse(driver.put(updatedRecord, false, true));
+
+    // Verify no update occurred, all original records are unchanged
+    QueryResult<T> newRecords = driver.get(clazz);
+    assertTrue(newRecords.getRecords().size() == 10);
+    assertEquals("A single entry was improperly updated in the store", 10,
+        countMatchingEntries(records.getRecords(), newRecords.getRecords()));
+
+    // Update the entry (allowing updates)
+    assertTrue(driver.put(updatedRecord, true, false));
+
+    // Verify that one entry no longer matches the original set
+    newRecords = driver.get(clazz);
+    assertEquals(10, newRecords.getRecords().size());
+    assertEquals(
+        "Record of type " + clazz + " not updated in the store", 9,
+        countMatchingEntries(records.getRecords(), newRecords.getRecords()));
+  }
+
+  private int countMatchingEntries(
+      Collection<? extends BaseRecord> committedList,
+      Collection<? extends BaseRecord> matchList) {
+
+    int matchingCount = 0;
+    for (BaseRecord committed : committedList) {
+      for (BaseRecord match : matchList) {
+        try {
+          if (match.getPrimaryKey().equals(committed.getPrimaryKey())) {
+            if (validateRecord(match, committed, false)) {
+              matchingCount++;
+            }
+            break;
+          }
+        } catch (Exception ex) {
+        }
+      }
+    }
+    return matchingCount;
+  }
+
+  public <T extends BaseRecord> void testRemove(
+      StateStoreDriver driver, Class<T> clazz)
+          throws IllegalArgumentException, IllegalAccessException, IOException {
+
+    // Remove all
+    assertTrue(driver.removeAll(clazz));
+    QueryResult<T> records = driver.get(clazz);
+    assertTrue(records.getRecords().isEmpty());
+
+    // Insert multiple
+    List<T> insertList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      T newRecord = generateFakeRecord(clazz);
+      insertList.add(newRecord);
+    }
+
+    // Verify
+    assertTrue(driver.putAll(insertList, false, true));
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 10);
+
+    // Remove Single
+    assertTrue(driver.remove(records.getRecords().get(0)));
+
+    // Verify
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 9);
+
+    // Remove with filter
+    final T firstRecord = records.getRecords().get(0);
+    final Query<T> query0 = new Query<T>(firstRecord);
+    assertTrue(driver.remove(clazz, query0) > 0);
+
+    final T secondRecord = records.getRecords().get(1);
+    final Query<T> query1 = new Query<T>(secondRecord);
+    assertTrue(driver.remove(clazz, query1) > 0);
+
+    // Verify
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 7);
+
+    // Remove all
+    assertTrue(driver.removeAll(clazz));
+
+    // Verify
+    records = driver.get(clazz);
+    assertTrue(records.getRecords().isEmpty());
+  }
+
+  public void testInsert(StateStoreDriver driver)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    // TODO add records
+  }
+
+  public void testPut(StateStoreDriver driver)
+      throws IllegalArgumentException, ReflectiveOperationException,
+      IOException, SecurityException {
+    // TODO add records
+  }
+
+  public void testRemove(StateStoreDriver driver)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    // TODO add records
+  }
+
+  public void testFetchErrors(StateStoreDriver driver)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    // TODO add records
+  }
+
+  /**
+   * Sets the value of a field on the object.
+   *
+   * @param fieldName The string name of the field.
+   * @param data The data to pass to the field's setter.
+   *
+   * @return True if successful, fails if failed.
+   */
+  private static boolean setField(
+      BaseRecord record, String fieldName, Object data) {
+
+    Method m = locateSetter(record, fieldName);
+    if (m != null) {
+      try {
+        m.invoke(record, data);
+      } catch (Exception e) {
+        LOG.error("Cannot set field " + fieldName + " on object "
+            + record.getClass().getName() + " to data " + data + " of type "
+            + data.getClass(), e);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Finds the appropriate setter for a field name.
+   *
+   * @param fieldName The legacy name of the field.
+   * @return The matching setter or null if not found.
+   */
+  private static Method locateSetter(BaseRecord record, String fieldName) {
+    for (Method m : record.getClass().getMethods()) {
+      if (m.getName().equalsIgnoreCase("set" + fieldName)) {
+        return m;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Returns all serializable fields in the object.
+   *
+   * @return Map with the fields.
+   */
+  private static Map<String, Class<?>> getFields(BaseRecord record) {
+    Map<String, Class<?>> getters = new HashMap<>();
+    for (Method m : record.getClass().getDeclaredMethods()) {
+      if (m.getName().startsWith("get")) {
+        try {
+          Class<?> type = m.getReturnType();
+          char[] c = m.getName().substring(3).toCharArray();
+          c[0] = Character.toLowerCase(c[0]);
+          String key = new String(c);
+          getters.put(key, type);
+        } catch (Exception e) {
+          LOG.error("Cannot execute getter " + m.getName()
+              + " on object " + record);
+        }
+      }
+    }
+    return getters;
+  }
+
+  /**
+   * Get the type of a field.
+   *
+   * @param fieldName
+   * @return Field type
+   */
+  private static Class<?> getFieldType(BaseRecord record, String fieldName) {
+    Method m = locateGetter(record, fieldName);
+    return m.getReturnType();
+  }
+
+  /**
+   * Fetches the value for a field name.
+   *
+   * @param fieldName the legacy name of the field.
+   * @return The field data or null if not found.
+   */
+  private static Object getField(BaseRecord record, String fieldName) {
+    Object result = null;
+    Method m = locateGetter(record, fieldName);
+    if (m != null) {
+      try {
+        result = m.invoke(record);
+      } catch (Exception e) {
+        LOG.error("Cannot get field " + fieldName + " on object " + record);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Finds the appropriate getter for a field name.
+   *
+   * @param fieldName The legacy name of the field.
+   * @return The matching getter or null if not found.
+   */
+  private static Method locateGetter(BaseRecord record, String fieldName) {
+    for (Method m : record.getClass().getMethods()) {
+      if (m.getName().equalsIgnoreCase("get" + fieldName)) {
+        return m;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Expands a data object from the store into an record object. Default store
+   * data type is a String. Override if additional serialization is required.
+   *
+   * @param data Object containing the serialized data. Only string is
+   *          supported.
+   * @param clazz Target object class to hold the deserialized data.
+   * @return An instance of the target data object initialized with the
+   *         deserialized data.
+   */
+  @Deprecated
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  private static <T> T fromString(String data, Class<T> clazz) {
+
+    if (data.equals("null")) {
+      return null;
+    } else if (clazz == String.class) {
+      return (T) data;
+    } else if (clazz == Long.class || clazz == long.class) {
+      return (T) Long.valueOf(data);
+    } else if (clazz == Integer.class || clazz == int.class) {
+      return (T) Integer.valueOf(data);
+    } else if (clazz == Double.class || clazz == double.class) {
+      return (T) Double.valueOf(data);
+    } else if (clazz == Float.class || clazz == float.class) {
+      return (T) Float.valueOf(data);
+    } else if (clazz == Boolean.class || clazz == boolean.class) {
+      return (T) Boolean.valueOf(data);
+    } else if (clazz.isEnum()) {
+      return (T) Enum.valueOf((Class<Enum>) clazz, data);
+    }
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java
new file mode 100644
index 0000000..920e280
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.store.driver;
+
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the FileSystem (e.g., HDFS) implementation of the State Store driver.
+ */
+public class TestStateStoreFile extends TestStateStoreDriverBase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = getStateStoreConfiguration(StateStoreFileImpl.class);
+    getStateStore(conf);
+  }
+
+  @Before
+  public void startup() throws IOException {
+    removeAll(getStateStoreDriver());
+  }
+
+  @Test
+  public void testInsert()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testUpdate()
+      throws IllegalArgumentException, ReflectiveOperationException,
+      IOException, SecurityException {
+    testPut(getStateStoreDriver());
+  }
+
+  @Test
+  public void testDelete()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testRemove(getStateStoreDriver());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java
new file mode 100644
index 0000000..da2e51d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.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.federation.store.driver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileSystemImpl;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the FileSystem (e.g., HDFS) implementation of the State Store driver.
+ */
+public class TestStateStoreFileSystem extends TestStateStoreDriverBase {
+
+  private static MiniDFSCluster dfsCluster;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = FederationStateStoreTestUtils
+        .getStateStoreConfiguration(StateStoreFileSystemImpl.class);
+    conf.set(StateStoreFileSystemImpl.FEDERATION_STORE_FS_PATH,
+        "/hdfs-federation/");
+
+    // Create HDFS cluster to back the state tore
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+    builder.numDataNodes(1);
+    dfsCluster = builder.build();
+    dfsCluster.waitClusterUp();
+    getStateStore(conf);
+  }
+
+  @AfterClass
+  public static void tearDownCluster() {
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+  }
+
+  @Before
+  public void startup() throws IOException {
+    removeAll(getStateStoreDriver());
+  }
+
+  @Test
+  public void testInsert()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testUpdate()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testDelete()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testFetchErrors()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testFetchErrors(getStateStoreDriver());
+  }
+}
\ 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


[07/29] hadoop git commit: HADOOP-12077. Provide a multi-URI replication Inode for ViewFs. Contributed by Gera Shegalov

Posted by in...@apache.org.
HADOOP-12077. Provide a multi-URI replication Inode for ViewFs. Contributed by Gera Shegalov


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

Branch: refs/heads/HDFS-10467
Commit: 1f3bc63e6772be81bc9a6a7d93ed81d2a9e066c0
Parents: 63720ef
Author: Chris Douglas <cd...@apache.org>
Authored: Tue Sep 5 23:30:18 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Tue Sep 5 23:51:51 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/viewfs/ConfigUtil.java |  27 +
 .../org/apache/hadoop/fs/viewfs/Constants.java  |   8 +-
 .../org/apache/hadoop/fs/viewfs/InodeTree.java  |  64 +-
 .../apache/hadoop/fs/viewfs/NflyFSystem.java    | 951 +++++++++++++++++++
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |  37 +-
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     |  10 +-
 .../TestViewFileSystemLocalFileSystem.java      |  77 +-
 .../hadoop/fs/viewfs/TestViewFsConfig.java      |  13 +-
 .../fs/viewfs/TestViewFileSystemHdfs.java       | 151 ++-
 9 files changed, 1275 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
index 6900df2..a5fc62e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.viewfs;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Utilities for config variables of the viewFs See {@link ViewFs}
@@ -69,6 +70,32 @@ public class ConfigUtil {
   }
   
   /**
+   *
+   * @param conf
+   * @param mountTableName
+   * @param src
+   * @param settings
+   * @param targets
+   */
+  public static void addLinkNfly(Configuration conf, String mountTableName,
+      String src, String settings, final URI ... targets) {
+
+    settings = settings == null
+        ? "minReplication=2,repairOnRead=true"
+        : settings;
+
+    conf.set(getConfigViewFsPrefix(mountTableName) + "." +
+            Constants.CONFIG_VIEWFS_LINK_NFLY + "." + settings + "." + src,
+        StringUtils.uriToString(targets));
+  }
+
+  public static void addLinkNfly(final Configuration conf, final String src,
+      final URI ... targets) {
+    addLinkNfly(conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, src, null,
+        targets);
+  }
+
+  /**
    * Add config variable for homedir for default mount table
    * @param conf - add to this conf
    * @param homedir - the home dir path starting with slash

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
index 9882a8e..1a07c10 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
@@ -57,7 +57,13 @@ public interface Constants {
    * Config variable for specifying a merge link
    */
   public static final String CONFIG_VIEWFS_LINK_MERGE = "linkMerge";
-  
+
+  /**
+   * Config variable for specifying an nfly link. Nfly writes to multiple
+   * locations, and allows reads from the closest one.
+   */
+  String CONFIG_VIEWFS_LINK_NFLY = "linkNfly";
+
   /**
    * Config variable for specifying a merge of the root of the mount-table
    *  with the root of another file system. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
index c62d5cc..665c9c9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
@@ -133,6 +133,12 @@ abstract class InodeTree<T> {
     }
   }
 
+  enum LinkType {
+    SINGLE,
+    MERGE,
+    NFLY
+  }
+
   /**
    * An internal class to represent a mount link.
    * A mount link can be single dir link or a merge dir link.
@@ -146,19 +152,17 @@ abstract class InodeTree<T> {
    * is changed later it is then ignored (a dir with null entries)
    */
   static class INodeLink<T> extends INode<T> {
-    final boolean isMergeLink; // true if MergeLink
     final URI[] targetDirLinkList;
     final T targetFileSystem;   // file system object created from the link.
 
     /**
-     * Construct a mergeLink.
+     * Construct a mergeLink or nfly.
      */
     INodeLink(final String pathToNode, final UserGroupInformation aUgi,
         final T targetMergeFs, final URI[] aTargetDirLinkList) {
       super(pathToNode, aUgi);
       targetFileSystem = targetMergeFs;
       targetDirLinkList = aTargetDirLinkList;
-      isMergeLink = true;
     }
 
     /**
@@ -170,7 +174,6 @@ abstract class InodeTree<T> {
       targetFileSystem = targetFs;
       targetDirLinkList = new URI[1];
       targetDirLinkList[0] = aTargetDirLink;
-      isMergeLink = false;
     }
 
     /**
@@ -188,7 +191,9 @@ abstract class InodeTree<T> {
   }
 
   private void createLink(final String src, final String target,
-      final boolean isLinkMerge, final UserGroupInformation aUgi)
+      final LinkType linkType, final String settings,
+      final UserGroupInformation aUgi,
+      final Configuration config)
       throws URISyntaxException, IOException,
       FileAlreadyExistsException, UnsupportedFileSystemException {
     // Validate that src is valid absolute path
@@ -235,18 +240,20 @@ abstract class InodeTree<T> {
     final INodeLink<T> newLink;
     final String fullPath = curInode.fullPath + (curInode == root ? "" : "/")
         + iPath;
-    if (isLinkMerge) { // Target is list of URIs
-      String[] targetsList = StringUtils.getStrings(target);
-      URI[] targetsListURI = new URI[targetsList.length];
-      int k = 0;
-      for (String itarget : targetsList) {
-        targetsListURI[k++] = new URI(itarget);
-      }
-      newLink = new INodeLink<T>(fullPath, aUgi,
-          getTargetFileSystem(targetsListURI), targetsListURI);
-    } else {
+    switch (linkType) {
+    case SINGLE:
       newLink = new INodeLink<T>(fullPath, aUgi,
           getTargetFileSystem(new URI(target)), new URI(target));
+      break;
+    case MERGE:
+    case NFLY:
+      final URI[] targetUris = StringUtils.stringToURI(
+          StringUtils.getStrings(target));
+      newLink = new INodeLink<T>(fullPath, aUgi,
+            getTargetFileSystem(settings, targetUris), targetUris);
+      break;
+    default:
+      throw new IllegalArgumentException(linkType + ": Infeasible linkType");
     }
     curInode.addLink(iPath, newLink);
     mountPoints.add(new MountPoint<T>(src, newLink));
@@ -257,14 +264,14 @@ abstract class InodeTree<T> {
    * 3 abstract methods.
    * @throws IOException
    */
-  protected abstract T getTargetFileSystem(final URI uri)
+  protected abstract T getTargetFileSystem(URI uri)
       throws UnsupportedFileSystemException, URISyntaxException, IOException;
 
-  protected abstract T getTargetFileSystem(final INodeDir<T> dir)
+  protected abstract T getTargetFileSystem(INodeDir<T> dir)
       throws URISyntaxException;
 
-  protected abstract T getTargetFileSystem(final URI[] mergeFsURIList)
-      throws UnsupportedFileSystemException, URISyntaxException;
+  protected abstract T getTargetFileSystem(String settings, URI[] mergeFsURIs)
+      throws UnsupportedFileSystemException, URISyntaxException, IOException;
 
   /**
    * Create Inode Tree from the specified mount-table specified in Config
@@ -298,8 +305,9 @@ abstract class InodeTree<T> {
       final String key = si.getKey();
       if (key.startsWith(mtPrefix)) {
         gotMountTableEntry = true;
-        boolean isMergeLink = false;
+        LinkType linkType = LinkType.SINGLE;
         String src = key.substring(mtPrefix.length());
+        String settings = null;
         if (src.startsWith(linkPrefix)) {
           src = src.substring(linkPrefix.length());
           if (src.equals(SlashPath.toString())) {
@@ -309,8 +317,20 @@ abstract class InodeTree<T> {
                 + "supported yet.");
           }
         } else if (src.startsWith(linkMergePrefix)) { // A merge link
-          isMergeLink = true;
+          linkType = LinkType.MERGE;
           src = src.substring(linkMergePrefix.length());
+        } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_NFLY)) {
+          // prefix.settings.src
+          src = src.substring(Constants.CONFIG_VIEWFS_LINK_NFLY.length() + 1);
+          // settings.src
+          settings = src.substring(0, src.indexOf('.'));
+          // settings
+
+          // settings.src
+          src = src.substring(settings.length() + 1);
+          // src
+
+          linkType = LinkType.NFLY;
         } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) {
           // ignore - we set home dir from config
           continue;
@@ -319,7 +339,7 @@ abstract class InodeTree<T> {
               "Mount table in config: " + src);
         }
         final String target = si.getValue(); // link or merge link
-        createLink(src, target, isMergeLink, ugi);
+        createLink(src, target, linkType, settings, ugi, config);
       }
     }
     if (!gotMountTableEntry) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
new file mode 100644
index 0000000..53966b8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NflyFSystem.java
@@ -0,0 +1,951 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.viewfs;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.net.ScriptBasedMapping;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Nfly is a multi filesystem mount point.
+ */
+@Private
+final class NflyFSystem extends FileSystem {
+  private static final Log LOG = LogFactory.getLog(NflyFSystem.class);
+  private static final String NFLY_TMP_PREFIX = "_nfly_tmp_";
+
+  enum NflyKey {
+    // minimum replication, if local filesystem is included +1 is recommended
+    minReplication,
+
+    // forces to check all the replicas and fetch the one with the most recent
+    // time stamp
+    //
+    readMostRecent,
+
+    // create missing replica from far to near, including local?
+    repairOnRead
+  }
+
+  private static final int DEFAULT_MIN_REPLICATION = 2;
+  private static URI nflyURI = URI.create("nfly:///");
+
+  private final NflyNode[] nodes;
+  private final int minReplication;
+  private final EnumSet<NflyKey> nflyFlags;
+  private final Node myNode;
+  private final NetworkTopology topology;
+
+  /**
+   * URI's authority is used as an approximation of the distance from the
+   * client. It's sufficient for DC but not accurate because worker nodes can be
+   * closer.
+   */
+  private static class NflyNode extends NodeBase {
+    private final ChRootedFileSystem fs;
+    NflyNode(String hostName, String rackName, URI uri,
+        Configuration conf) throws IOException {
+      this(hostName, rackName, new ChRootedFileSystem(uri, conf));
+    }
+
+    NflyNode(String hostName, String rackName, ChRootedFileSystem fs) {
+      super(hostName, rackName);
+      this.fs = fs;
+    }
+
+    ChRootedFileSystem getFs() {
+      return fs;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      // satisfy findbugs
+      return super.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      // satisfy findbugs
+      return super.hashCode();
+    }
+
+  }
+
+  private static final class MRNflyNode
+      extends NflyNode implements Comparable<MRNflyNode> {
+
+    private FileStatus status;
+
+    private MRNflyNode(NflyNode n) {
+      super(n.getName(), n.getNetworkLocation(), n.fs);
+    }
+
+    private void updateFileStatus(Path f) throws IOException {
+      final FileStatus tmpStatus = getFs().getFileStatus(f);
+      status = tmpStatus == null
+          ? notFoundStatus(f)
+          : tmpStatus;
+    }
+
+    // TODO allow configurable error margin for FileSystems with different
+    // timestamp precisions
+    @Override
+    public int compareTo(MRNflyNode other) {
+      if (status == null) {
+        return other.status == null ? 0 : 1; // move non-null towards head
+      } else if (other.status == null) {
+        return -1; // move this towards head
+      } else {
+        final long mtime = status.getModificationTime();
+        final long their = other.status.getModificationTime();
+        return Long.compare(their, mtime); // move more recent towards head
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof MRNflyNode)) {
+        return false;
+      }
+      MRNflyNode other = (MRNflyNode) o;
+      return 0 == compareTo(other);
+    }
+
+    @Override
+    public int hashCode() {
+      // satisfy findbugs
+      return super.hashCode();
+    }
+
+    private FileStatus nflyStatus() throws IOException {
+      return new NflyStatus(getFs(), status);
+    }
+
+    private FileStatus cloneStatus() throws IOException {
+      return new FileStatus(status.getLen(),
+          status.isDirectory(),
+          status.getReplication(),
+          status.getBlockSize(),
+          status.getModificationTime(),
+          status.getAccessTime(),
+          null, null, null,
+          status.isSymlink() ? status.getSymlink() : null,
+          status.getPath());
+    }
+  }
+
+  private MRNflyNode[] workSet() {
+    final MRNflyNode[] res = new MRNflyNode[nodes.length];
+    for (int i = 0; i < res.length; i++) {
+      res[i] = new MRNflyNode(nodes[i]);
+    }
+    return res;
+  }
+
+
+  /**
+   * Utility to replace null with DEFAULT_RACK.
+   *
+   * @param rackString rack value, can be null
+   * @return non-null rack string
+   */
+  private static String getRack(String rackString) {
+    return rackString == null ? NetworkTopology.DEFAULT_RACK : rackString;
+  }
+
+  /**
+   * Creates a new Nfly instance.
+   *
+   * @param uris the list of uris in the mount point
+   * @param conf configuration object
+   * @param minReplication minimum copies to commit a write op
+   * @param nflyFlags modes such readMostRecent
+   * @throws IOException
+   */
+  private NflyFSystem(URI[] uris, Configuration conf, int minReplication,
+      EnumSet<NflyKey> nflyFlags) throws IOException {
+    if (uris.length < minReplication) {
+      throw new IOException(minReplication + " < " + uris.length
+          + ": Minimum replication < #destinations");
+    }
+    setConf(conf);
+    final String localHostName = InetAddress.getLocalHost().getHostName();
+
+    // build a list for topology resolution
+    final List<String> hostStrings = new ArrayList<String>(uris.length + 1);
+    for (URI uri : uris) {
+      final String uriHost = uri.getHost();
+      // assume local file system or another closest filesystem if no authority
+      hostStrings.add(uriHost == null ? localHostName : uriHost);
+    }
+    // resolve the client node
+    hostStrings.add(localHostName);
+
+    final DNSToSwitchMapping tmpDns = ReflectionUtils.newInstance(conf.getClass(
+        CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+        ScriptBasedMapping.class, DNSToSwitchMapping.class), conf);
+
+    // this is an ArrayList
+    final List<String> rackStrings = tmpDns.resolve(hostStrings);
+    nodes = new NflyNode[uris.length];
+    final Iterator<String> rackIter = rackStrings.iterator();
+    for (int i = 0; i < nodes.length; i++) {
+      nodes[i] = new NflyNode(hostStrings.get(i), rackIter.next(), uris[i],
+          conf);
+    }
+    // sort all the uri's by distance from myNode, the local file system will
+    // automatically be the the first one.
+    //
+    myNode = new NodeBase(localHostName, getRack(rackIter.next()));
+    topology = NetworkTopology.getInstance(conf);
+    topology.sortByDistance(myNode, nodes, nodes.length);
+
+    this.minReplication = minReplication;
+    this.nflyFlags = nflyFlags;
+    statistics = getStatistics(nflyURI.getScheme(), getClass());
+  }
+
+  /**
+   * Transactional output stream. When creating path /dir/file
+   * 1) create invisible /real/dir_i/_nfly_tmp_file
+   * 2) when more than min replication was written, write is committed by
+   *   renaming all successfully written files to /real/dir_i/file
+   */
+  private final class NflyOutputStream extends OutputStream {
+    // actual path
+    private final Path nflyPath;
+    // tmp path before commit
+    private final Path tmpPath;
+    // broadcast set
+    private final FSDataOutputStream[] outputStreams;
+    // status set: 1 working, 0 problem
+    private final BitSet opSet;
+    private final boolean useOverwrite;
+
+    private NflyOutputStream(Path f, FsPermission permission, boolean overwrite,
+        int bufferSize, short replication, long blockSize,
+        Progressable progress) throws IOException {
+      nflyPath = f;
+      tmpPath = getNflyTmpPath(f);
+      outputStreams = new FSDataOutputStream[nodes.length];
+      for (int i = 0; i < outputStreams.length; i++) {
+        outputStreams[i] = nodes[i].fs.create(tmpPath, permission, true,
+            bufferSize, replication, blockSize, progress);
+      }
+      opSet = new BitSet(outputStreams.length);
+      opSet.set(0, outputStreams.length);
+      useOverwrite = false;
+    }
+
+    //
+    // TODO consider how to clean up and throw an exception early when the clear
+    // bits under min replication
+    //
+
+    private void mayThrow(List<IOException> ioExceptions) throws IOException {
+      final IOException ioe = MultipleIOException
+          .createIOException(ioExceptions);
+      if (opSet.cardinality() < minReplication) {
+        throw ioe;
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Exceptions occurred: " + ioe);
+        }
+      }
+    }
+
+
+    @Override
+    public void write(int d) throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >=0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].write(d);
+        } catch (Throwable t) {
+          osException(i, "write", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    private void osException(int i, String op, Throwable t,
+        List<IOException> ioExceptions) {
+      opSet.clear(i);
+      processThrowable(nodes[i], op, t, ioExceptions, tmpPath, nflyPath);
+    }
+
+    @Override
+    public void write(byte[] bytes, int offset, int len) throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].write(bytes, offset, len);
+        } catch (Throwable t) {
+          osException(i, "write", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    @Override
+    public void flush() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].flush();
+        } catch (Throwable t) {
+          osException(i, "flush", t, ioExceptions);
+        }
+      }
+      mayThrow(ioExceptions);
+    }
+
+    @Override
+    public void close() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        try {
+          outputStreams[i].close();
+        } catch (Throwable t) {
+          osException(i, "close", t, ioExceptions);
+        }
+      }
+      if (opSet.cardinality() < minReplication) {
+        cleanupAllTmpFiles();
+        throw new IOException("Failed to sufficiently replicate: min="
+            + minReplication + " actual=" + opSet.cardinality());
+      } else {
+        commit();
+      }
+    }
+
+    private void cleanupAllTmpFiles() throws IOException {
+      for (int i = 0; i < outputStreams.length; i++) {
+        try {
+          nodes[i].fs.delete(tmpPath);
+        } catch (Throwable t) {
+          processThrowable(nodes[i], "delete", t, null, tmpPath);
+        }
+      }
+    }
+
+    private void commit() throws IOException {
+      final List<IOException> ioExceptions = new ArrayList<IOException>();
+      for (int i = opSet.nextSetBit(0);
+           i >= 0;
+           i = opSet.nextSetBit(i + 1)) {
+        final NflyNode nflyNode = nodes[i];
+        try {
+          if (useOverwrite) {
+            nflyNode.fs.delete(nflyPath);
+          }
+          nflyNode.fs.rename(tmpPath, nflyPath);
+
+        } catch (Throwable t) {
+          osException(i, "commit", t, ioExceptions);
+        }
+      }
+
+      if (opSet.cardinality() < minReplication) {
+        // cleanup should be done outside. If rename failed, it's unlikely that
+        // delete will work either. It's the same kind of metadata-only op
+        //
+        throw MultipleIOException.createIOException(ioExceptions);
+      }
+
+      // best effort to have a consistent timestamp
+      final long commitTime = System.currentTimeMillis();
+      for (int i = opSet.nextSetBit(0);
+          i >= 0;
+          i = opSet.nextSetBit(i + 1)) {
+        try {
+          nodes[i].fs.setTimes(nflyPath, commitTime, commitTime);
+        } catch (Throwable t) {
+          LOG.info("Failed to set timestamp: " + nodes[i] + " " + nflyPath);
+        }
+      }
+    }
+  }
+
+  private Path getNflyTmpPath(Path f) {
+    return new Path(f.getParent(), NFLY_TMP_PREFIX + f.getName());
+  }
+
+  /**
+   * // TODO
+   * Some file status implementations have expensive deserialization or metadata
+   * retrieval. This probably does not go beyond RawLocalFileSystem. Wrapping
+   * the the real file status to preserve this behavior. Otherwise, calling
+   * realStatus getters in constructor defeats this design.
+   */
+  static final class NflyStatus extends FileStatus {
+    private static final long serialVersionUID = 0x21f276d8;
+
+    private final FileStatus realStatus;
+    private final String strippedRoot;
+
+    private NflyStatus(ChRootedFileSystem realFs, FileStatus realStatus)
+        throws IOException {
+      this.realStatus = realStatus;
+      this.strippedRoot = realFs.stripOutRoot(realStatus.getPath());
+    }
+
+    String stripRoot() throws IOException {
+      return strippedRoot;
+    }
+
+    @Override
+    public long getLen() {
+      return realStatus.getLen();
+    }
+
+    @Override
+    public boolean isFile() {
+      return realStatus.isFile();
+    }
+
+    @Override
+    public boolean isDirectory() {
+      return realStatus.isDirectory();
+    }
+
+    @Override
+    public boolean isSymlink() {
+      return realStatus.isSymlink();
+    }
+
+    @Override
+    public long getBlockSize() {
+      return realStatus.getBlockSize();
+    }
+
+    @Override
+    public short getReplication() {
+      return realStatus.getReplication();
+    }
+
+    @Override
+    public long getModificationTime() {
+      return realStatus.getModificationTime();
+    }
+
+    @Override
+    public long getAccessTime() {
+      return realStatus.getAccessTime();
+    }
+
+    @Override
+    public FsPermission getPermission() {
+      return realStatus.getPermission();
+    }
+
+    @Override
+    public String getOwner() {
+      return realStatus.getOwner();
+    }
+
+    @Override
+    public String getGroup() {
+      return realStatus.getGroup();
+    }
+
+    @Override
+    public Path getPath() {
+      return realStatus.getPath();
+    }
+
+    @Override
+    public void setPath(Path p) {
+      realStatus.setPath(p);
+    }
+
+    @Override
+    public Path getSymlink() throws IOException {
+      return realStatus.getSymlink();
+    }
+
+    @Override
+    public void setSymlink(Path p) {
+      realStatus.setSymlink(p);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return realStatus.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      return realStatus.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return realStatus.toString();
+    }
+  }
+
+  @Override
+  public URI getUri() {
+    return nflyURI;
+  }
+
+  /**
+   * Category: READ.
+   *
+   * @param f the file name to open
+   * @param bufferSize the size of the buffer to be used.
+   * @return input stream according to nfly flags (closest, most recent)
+   * @throws IOException
+   * @throws FileNotFoundException iff all destinations generate this exception
+   */
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    // TODO proxy stream for reads
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+    int numNotFounds = 0;
+    final MRNflyNode[] mrNodes = workSet();
+
+    // naively iterate until one can be opened
+    //
+    for (final MRNflyNode nflyNode : mrNodes) {
+      try {
+        if (nflyFlags.contains(NflyKey.repairOnRead)
+            || nflyFlags.contains(NflyKey.readMostRecent)) {
+          // calling file status to avoid pulling bytes prematurely
+          nflyNode.updateFileStatus(f);
+        } else {
+          return nflyNode.getFs().open(f, bufferSize);
+        }
+      } catch (FileNotFoundException fnfe) {
+        nflyNode.status = notFoundStatus(f);
+        numNotFounds++;
+        processThrowable(nflyNode, "open", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "open", t, ioExceptions, f);
+      }
+    }
+
+    if (nflyFlags.contains(NflyKey.readMostRecent)) {
+      // sort from most recent to least recent
+      Arrays.sort(mrNodes);
+    }
+
+    final FSDataInputStream fsdisAfterRepair = repairAndOpen(mrNodes, f,
+        bufferSize);
+
+    if (fsdisAfterRepair != null) {
+      return fsdisAfterRepair;
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  private static FileStatus notFoundStatus(Path f) {
+    return new FileStatus(-1, false, 0, 0, 0, f);
+  }
+
+  /**
+   * Iterate all available nodes in the proximity order to attempt repair of all
+   * FileNotFound nodes.
+   *
+   * @param mrNodes work set copy of nodes
+   * @param f path to repair and open
+   * @param bufferSize buffer size for read RPC
+   * @return the closest/most recent replica stream AFTER repair
+   */
+  private FSDataInputStream repairAndOpen(MRNflyNode[] mrNodes, Path f,
+      int bufferSize) {
+    long maxMtime = 0L;
+    for (final MRNflyNode srcNode : mrNodes) {
+      if (srcNode.status == null  // not available
+          || srcNode.status.getLen() < 0L) { // not found
+        continue; // not available
+      }
+      if (srcNode.status.getModificationTime() > maxMtime) {
+        maxMtime = srcNode.status.getModificationTime();
+      }
+
+      // attempt to repair all notFound nodes with srcNode
+      //
+      for (final MRNflyNode dstNode : mrNodes) {
+        if (dstNode.status == null // not available
+            || srcNode.compareTo(dstNode) == 0) { // same mtime
+          continue;
+        }
+
+        try {
+          // status is absolute from the underlying mount, making it chrooted
+          //
+          final FileStatus srcStatus = srcNode.cloneStatus();
+          srcStatus.setPath(f);
+          final Path tmpPath = getNflyTmpPath(f);
+          FileUtil.copy(srcNode.getFs(), srcStatus, dstNode.getFs(), tmpPath,
+              false,                // don't delete
+              true,                 // overwrite
+              getConf());
+          dstNode.getFs().delete(f, false);
+          if (dstNode.getFs().rename(tmpPath, f)) {
+            try {
+              dstNode.getFs().setTimes(f, srcNode.status.getModificationTime(),
+                  srcNode.status.getAccessTime());
+            } finally {
+              // save getFileStatus rpc
+              srcStatus.setPath(dstNode.getFs().makeQualified(f));
+              dstNode.status = srcStatus;
+            }
+          }
+        } catch (IOException ioe) {
+          // can blame the source by statusSet.clear(ai), however, it would
+          // cost an extra RPC, so just rely on the loop below that will attempt
+          // an open anyhow
+          //
+          LOG.info(f + " " + srcNode + "->" + dstNode + ": Failed to repair",
+                ioe);
+        }
+      }
+    }
+
+    // Since Java7, QuickSort is used instead of MergeSort.
+    // QuickSort may not be stable and thus the equal most recent nodes, may no
+    // longer appear in the NetworkTopology order.
+    //
+    if (maxMtime > 0) {
+      final List<MRNflyNode> mrList = new ArrayList<MRNflyNode>();
+      for (final MRNflyNode openNode : mrNodes) {
+        if (openNode.status != null && openNode.status.getLen() >= 0L) {
+          if (openNode.status.getModificationTime() == maxMtime) {
+            mrList.add(openNode);
+          }
+        }
+      }
+      // assert mrList.size > 0
+      final MRNflyNode[] readNodes = mrList.toArray(new MRNflyNode[0]);
+      topology.sortByDistance(myNode, readNodes, readNodes.length);
+      for (final MRNflyNode rNode : readNodes) {
+        try {
+          return rNode.getFs().open(f, bufferSize);
+        } catch (IOException e) {
+          LOG.info(f + ": Failed to open at " + rNode.getFs().getUri());
+        }
+      }
+    }
+    return null;
+  }
+
+  private void mayThrowFileNotFound(List<IOException> ioExceptions,
+      int numNotFounds) throws FileNotFoundException {
+    if (numNotFounds == nodes.length) {
+      throw (FileNotFoundException)ioExceptions.get(nodes.length - 1);
+    }
+  }
+
+  // WRITE
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return new FSDataOutputStream(new NflyOutputStream(f, permission, overwrite,
+        bufferSize, replication, blockSize, progress), statistics);
+  }
+
+  // WRITE
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    return null;
+  }
+
+  // WRITE
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    final List<IOException> ioExceptions = new ArrayList<IOException>();
+    int numNotFounds = 0;
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      try {
+        succ &= nflyNode.fs.rename(src, dst);
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "rename", fnfe, ioExceptions, src, dst);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "rename", t, ioExceptions, src, dst);
+        succ = false;
+      }
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+
+    // if all destinations threw exceptions throw, otherwise return
+    //
+    if (ioExceptions.size() == nodes.length) {
+      throw MultipleIOException.createIOException(ioExceptions);
+    }
+
+    return succ;
+  }
+
+  // WRITE
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    final List<IOException> ioExceptions = new ArrayList<IOException>();
+    int numNotFounds = 0;
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      try {
+        succ &= nflyNode.fs.delete(f);
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "delete", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "delete", t, ioExceptions, f);
+        succ = false;
+      }
+    }
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+
+    // if all destinations threw exceptions throw, otherwise return
+    //
+    if (ioExceptions.size() == nodes.length) {
+      throw MultipleIOException.createIOException(ioExceptions);
+    }
+
+    return succ;
+  }
+
+
+  /**
+   * Returns the closest non-failing destination's result.
+   *
+   * @param f given path
+   * @return array of file statuses according to nfly modes
+   * @throws FileNotFoundException
+   * @throws IOException
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws FileNotFoundException,
+      IOException {
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+
+    final MRNflyNode[] mrNodes = workSet();
+    if (nflyFlags.contains(NflyKey.readMostRecent)) {
+      int numNotFounds = 0;
+      for (final MRNflyNode nflyNode : mrNodes) {
+        try {
+          nflyNode.updateFileStatus(f);
+        } catch (FileNotFoundException fnfe) {
+          numNotFounds++;
+          processThrowable(nflyNode, "listStatus", fnfe, ioExceptions, f);
+        } catch (Throwable t) {
+          processThrowable(nflyNode, "listStatus", t, ioExceptions, f);
+        }
+      }
+      mayThrowFileNotFound(ioExceptions, numNotFounds);
+      Arrays.sort(mrNodes);
+    }
+
+    int numNotFounds = 0;
+    for (final MRNflyNode nflyNode : mrNodes) {
+      try {
+        final FileStatus[] realStats = nflyNode.getFs().listStatus(f);
+        final FileStatus[] nflyStats = new FileStatus[realStats.length];
+        for (int i = 0; i < realStats.length; i++) {
+          nflyStats[i] = new NflyStatus(nflyNode.getFs(), realStats[i]);
+        }
+        return nflyStats;
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "listStatus", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "listStatus", t, ioExceptions, f);
+      }
+    }
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  @Override
+  public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
+      throws FileNotFoundException, IOException {
+    // TODO important for splits
+    return super.listLocatedStatus(f);
+  }
+
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    for (final NflyNode nflyNode : nodes) {
+      nflyNode.fs.setWorkingDirectory(newDir);
+    }
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return nodes[0].fs.getWorkingDirectory(); // 0 is as good as any
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    boolean succ = true;
+    for (final NflyNode nflyNode : nodes) {
+      succ &= nflyNode.fs.mkdirs(f, permission);
+    }
+    return succ;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    // TODO proxy stream for reads
+    final List<IOException> ioExceptions =
+        new ArrayList<IOException>(nodes.length);
+    int numNotFounds = 0;
+    final MRNflyNode[] mrNodes = workSet();
+
+    long maxMtime = Long.MIN_VALUE;
+    int maxMtimeIdx = Integer.MIN_VALUE;
+
+    // naively iterate until one can be returned
+    //
+    for (int i = 0; i < mrNodes.length; i++) {
+      MRNflyNode nflyNode = mrNodes[i];
+      try {
+        nflyNode.updateFileStatus(f);
+        if (nflyFlags.contains(NflyKey.readMostRecent)) {
+          final long nflyTime = nflyNode.status.getModificationTime();
+          if (nflyTime > maxMtime) {
+            maxMtime = nflyTime;
+            maxMtimeIdx = i;
+          }
+        } else {
+          return nflyNode.nflyStatus();
+        }
+      } catch (FileNotFoundException fnfe) {
+        numNotFounds++;
+        processThrowable(nflyNode, "getFileStatus", fnfe, ioExceptions, f);
+      } catch (Throwable t) {
+        processThrowable(nflyNode, "getFileStatus", t, ioExceptions, f);
+      }
+    }
+
+    if (maxMtimeIdx >= 0) {
+      return mrNodes[maxMtimeIdx].nflyStatus();
+    }
+
+    mayThrowFileNotFound(ioExceptions, numNotFounds);
+    throw MultipleIOException.createIOException(ioExceptions);
+  }
+
+  private static void processThrowable(NflyNode nflyNode, String op,
+      Throwable t, List<IOException> ioExceptions,
+      Path... f) {
+    final String errMsg = Arrays.toString(f)
+        + ": failed to " + op + " " + nflyNode.fs.getUri();
+    final IOException ioex;
+    if (t instanceof FileNotFoundException) {
+      ioex = new FileNotFoundException(errMsg);
+      ioex.initCause(t);
+    } else {
+      ioex = new IOException(errMsg, t);
+    }
+
+    if (ioExceptions != null) {
+      ioExceptions.add(ioex);
+    }
+  }
+
+  /**
+   * Initializes an nfly mountpoint in viewfs.
+   *
+   * @param uris destinations to replicate writes to
+   * @param conf file system configuration
+   * @param settings comma-separated list of k=v pairs.
+   * @return an Nfly filesystem
+   * @throws IOException
+   */
+  static FileSystem createFileSystem(URI[] uris, Configuration conf,
+      String settings) throws IOException {
+    // assert settings != null
+    int minRepl = DEFAULT_MIN_REPLICATION;
+    EnumSet<NflyKey> nflyFlags = EnumSet.noneOf(NflyKey.class);
+    final String[] kvPairs = StringUtils.split(settings);
+    for (String kv : kvPairs) {
+      final String[] kvPair = StringUtils.split(kv, '=');
+      if (kvPair.length != 2) {
+        throw new IllegalArgumentException(kv);
+      }
+      NflyKey nflyKey = NflyKey.valueOf(kvPair[0]);
+      switch (nflyKey) {
+      case minReplication:
+        minRepl = Integer.parseInt(kvPair[1]);
+        break;
+      case repairOnRead:
+      case readMostRecent:
+        if (Boolean.valueOf(kvPair[1])) {
+          nflyFlags.add(nflyKey);
+        }
+        break;
+      default:
+        throw new IllegalArgumentException(nflyKey + ": Infeasible");
+      }
+    }
+    return new NflyFSystem(uris, conf, minRepl, nflyFlags);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 158b099..ca1380a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -186,25 +185,21 @@ public class ViewFileSystem extends FileSystem {
       fsState = new InodeTree<FileSystem>(conf, authority) {
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(final URI uri)
+        protected FileSystem getTargetFileSystem(final URI uri)
           throws URISyntaxException, IOException {
             return new ChRootedFileSystem(uri, config);
         }
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
+        protected FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
           throws URISyntaxException {
           return new InternalDirOfViewFs(dir, creationTime, ugi, myUri, config);
         }
 
         @Override
-        protected
-        FileSystem getTargetFileSystem(URI[] mergeFsURIList)
-            throws URISyntaxException, UnsupportedFileSystemException {
-          throw new UnsupportedFileSystemException("mergefs not implemented");
-          // return MergeFs.createMergeFs(mergeFsURIList, config);
+        protected FileSystem getTargetFileSystem(final String settings,
+            final URI[] uris) throws URISyntaxException, IOException {
+          return NflyFSystem.createFileSystem(uris, config, settings);
         }
       };
       workingDir = this.getHomeDirectory();
@@ -455,8 +450,13 @@ public class ViewFileSystem extends FileSystem {
 
   private Path getChrootedPath(InodeTree.ResolveResult<FileSystem> res,
       FileStatus status, Path f) throws IOException {
-    final String suffix = ((ChRootedFileSystem)res.targetFileSystem)
-        .stripOutRoot(status.getPath());
+    final String suffix;
+    if (res.targetFileSystem instanceof ChRootedFileSystem) {
+      suffix = ((ChRootedFileSystem)res.targetFileSystem)
+          .stripOutRoot(status.getPath());
+    } else { // nfly
+      suffix = ((NflyFSystem.NflyStatus)status).stripRoot();
+    }
     return this.makeQualified(
         suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix));
   }
@@ -501,10 +501,15 @@ public class ViewFileSystem extends FileSystem {
     verifyRenameStrategy(srcUri, dstUri,
         resSrc.targetFileSystem == resDst.targetFileSystem, renameStrategy);
 
-    ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
-    ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
-    return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
-        dstFS.fullPath(resDst.remainingPath));
+    if (resSrc.targetFileSystem instanceof ChRootedFileSystem &&
+        resDst.targetFileSystem instanceof ChRootedFileSystem) {
+      ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
+      ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
+      return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
+          dstFS.fullPath(resDst.remainingPath));
+    } else {
+      return resSrc.targetFileSystem.rename(resSrc.remainingPath, resDst.remainingPath);
+    }
   }
 
   static void verifyRenameStrategy(URI srcUri, URI dstUri,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
index 364485f..6a89f27 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
@@ -212,8 +212,7 @@ public class ViewFs extends AbstractFileSystem {
     fsState = new InodeTree<AbstractFileSystem>(conf, authority) {
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(final URI uri)
+      protected AbstractFileSystem getTargetFileSystem(final URI uri)
         throws URISyntaxException, UnsupportedFileSystemException {
           String pathString = uri.getPath();
           if (pathString.isEmpty()) {
@@ -225,15 +224,14 @@ public class ViewFs extends AbstractFileSystem {
       }
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(
+      protected AbstractFileSystem getTargetFileSystem(
           final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
         return new InternalDirOfViewFs(dir, creationTime, ugi, getUri());
       }
 
       @Override
-      protected
-      AbstractFileSystem getTargetFileSystem(URI[] mergeFsURIList)
+      protected AbstractFileSystem getTargetFileSystem(final String settings,
+          final URI[] mergeFsURIList)
           throws URISyntaxException, UnsupportedFileSystemException {
         throw new UnsupportedFileSystemException("mergefs not implemented yet");
         // return MergeFs.createMergeFs(mergeFsURIList, config);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
index 4943792..808d8b0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
@@ -18,13 +18,25 @@
 package org.apache.hadoop.fs.viewfs;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
 
 import org.junit.After;
 import org.junit.Before;
-
+import org.junit.Test;
 
 
 /**
@@ -37,6 +49,8 @@ import org.junit.Before;
  */
 
 public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
+  private static final Log LOG =
+      LogFactory.getLog(TestViewFileSystemLocalFileSystem.class);
 
   @Override
   @Before
@@ -47,6 +61,65 @@ public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
     
   }
 
+  @Test
+  public void testNflyWriteSimple() throws IOException {
+    LOG.info("Starting testNflyWriteSimple");
+    final URI[] testUris = new URI[] {
+        URI.create(targetTestRoot + "/nfwd1"),
+        URI.create(targetTestRoot + "/nfwd2")
+    };
+    final String testFileName = "test.txt";
+    final Configuration testConf = new Configuration(conf);
+    final String testString = "Hello Nfly!";
+    final Path nflyRoot = new Path("/nflyroot");
+    ConfigUtil.addLinkNfly(testConf, nflyRoot.toString(), testUris);
+    final FileSystem nfly = FileSystem.get(URI.create("viewfs:///"), testConf);
+
+    final FSDataOutputStream fsDos = nfly.create(
+        new Path(nflyRoot, "test.txt"));
+    try {
+      fsDos.writeUTF(testString);
+    } finally {
+      fsDos.close();
+    }
+
+    FileStatus[] statuses = nfly.listStatus(nflyRoot);
+
+    FileSystem lfs = FileSystem.getLocal(testConf);
+    for (final URI testUri : testUris) {
+      final Path testFile = new Path(new Path(testUri), testFileName);
+      assertTrue(testFile + " should exist!",  lfs.exists(testFile));
+      final FSDataInputStream fsdis = lfs.open(testFile);
+      try {
+        assertEquals("Wrong file content", testString, fsdis.readUTF());
+      } finally {
+        fsdis.close();
+      }
+    }
+  }
+
+
+  @Test
+  public void testNflyInvalidMinReplication() throws Exception {
+    LOG.info("Starting testNflyInvalidMinReplication");
+    final URI[] testUris = new URI[] {
+        URI.create(targetTestRoot + "/nfwd1"),
+        URI.create(targetTestRoot + "/nfwd2")
+    };
+
+    final Configuration conf = new Configuration();
+    ConfigUtil.addLinkNfly(conf, "mt", "/nflyroot", "minReplication=4",
+        testUris);
+    try {
+      FileSystem.get(URI.create("viewfs://mt/"), conf);
+      fail("Expected bad minReplication exception.");
+    } catch (IOException ioe) {
+      assertTrue("No minReplication message",
+          ioe.getMessage().contains("Minimum replication"));
+    }
+  }
+
+
   @Override
   @After
   public void tearDown() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
index 895ae0c..136837f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
@@ -24,7 +24,6 @@ import java.net.URISyntaxException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.junit.Test;
 
 public class TestViewFsConfig {
@@ -43,23 +42,21 @@ public class TestViewFsConfig {
     new InodeTree<Foo>(conf, null) {
 
       @Override
-      protected Foo getTargetFileSystem(final URI uri)
-          throws URISyntaxException, UnsupportedFileSystemException {
+      protected Foo getTargetFileSystem(final URI uri) {
         return null;
       }
 
       @Override
-      protected Foo getTargetFileSystem(
-          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo> dir)
-          throws URISyntaxException {
+      protected Foo getTargetFileSystem(final INodeDir<Foo> dir) {
         return null;
       }
 
       @Override
-      protected Foo getTargetFileSystem(URI[] mergeFsURIList)
-          throws URISyntaxException, UnsupportedFileSystemException {
+      protected Foo getTargetFileSystem(final String settings,
+          final URI[] mergeFsURIList) {
         return null;
       }
+
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f3bc63e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
index b8f5379..b8bed1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
@@ -17,11 +17,9 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.EnumSet;
 
@@ -31,6 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,17 +45,26 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
+
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestViewFileSystemHdfs.class);
+
 
   private static MiniDFSCluster cluster;
   private static Path defaultWorkingDirectory;
@@ -190,12 +199,12 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
 
     //Verify file deletion within EZ
     DFSTestUtil.verifyDelete(shell, fsTarget, encFile, true);
-    Assert.assertTrue("ViewFileSystem trash roots should include EZ file trash",
+    assertTrue("ViewFileSystem trash roots should include EZ file trash",
         (fsView.getTrashRoots(true).size() == 1));
 
     //Verify deletion of EZ
     DFSTestUtil.verifyDelete(shell, fsTarget, zone, true);
-    Assert.assertTrue("ViewFileSystem trash roots should include EZ zone trash",
+    assertTrue("ViewFileSystem trash roots should include EZ zone trash",
         (fsView.getTrashRoots(true).size() == 2));
   }
 
@@ -239,14 +248,14 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
         viewFs.getFileChecksum(mountDataFilePath);
     FileChecksum fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath);
-    Assert.assertTrue("File checksum not matching!",
+    assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
 
     fileChecksumViaViewFs =
         viewFs.getFileChecksum(mountDataFilePath, fileLength / 2);
     fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath, fileLength / 2);
-    Assert.assertTrue("File checksum not matching!",
+    assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
   }
 
@@ -269,4 +278,130 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
               e);
     }
   }
+
+  @Test
+  public void testNflyClosestRepair() throws Exception {
+    testNflyRepair(NflyFSystem.NflyKey.repairOnRead);
+  }
+
+  @Test
+  public void testNflyMostRecentRepair() throws Exception {
+    testNflyRepair(NflyFSystem.NflyKey.readMostRecent);
+  }
+
+  private void testNflyRepair(NflyFSystem.NflyKey repairKey)
+      throws Exception {
+    LOG.info("Starting testNflyWriteSimpleFailover");
+    final URI uri1 = targetTestRoot.toUri();
+    final URI uri2 = targetTestRoot2.toUri();
+    final URI[] testUris = new URI[] {
+        new URI(uri1.getScheme(), uri1.getAuthority(), "/", null, null),
+        new URI(uri2.getScheme(), uri2.getAuthority(), "/", null, null)
+    };
+
+    final Configuration testConf = new Configuration(conf);
+    testConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
+
+    final String testString = "Hello Nfly!";
+    final Path nflyRoot = new Path("/nflyroot");
+
+    ConfigUtil.addLinkNfly(testConf,
+        Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE,
+        nflyRoot.toString(),
+        "minReplication=2," + repairKey + "=true", testUris);
+
+    final FileSystem nfly = FileSystem.get(URI.create("viewfs:///"), testConf);
+    // wd = /nflyroot/user/<user>
+    nfly.setWorkingDirectory(new Path(nflyRoot
+        + nfly.getWorkingDirectory().toUri().getPath()));
+
+    // 1. test mkdirs
+    final Path testDir = new Path("testdir1/sub1/sub3");
+    final Path testDir_tmp = new Path("testdir1/sub1/sub3_temp");
+    assertTrue(testDir + ": Failed to create!", nfly.mkdirs(testDir));
+
+    // Test renames
+    assertTrue(nfly.rename(testDir, testDir_tmp));
+    assertTrue(nfly.rename(testDir_tmp, testDir));
+
+    for (final URI testUri : testUris) {
+      final FileSystem fs = FileSystem.get(testUri, testConf);
+      assertTrue(testDir + " should exist!", fs.exists(testDir));
+    }
+
+    // 2. test write
+    final Path testFile = new Path("test.txt");
+    final FSDataOutputStream fsDos = nfly.create(testFile);
+    try {
+      fsDos.writeUTF(testString);
+    } finally {
+      fsDos.close();
+    }
+
+    for (final URI testUri : testUris) {
+      final FileSystem fs = FileSystem.get(testUri, testConf);
+      final FSDataInputStream fsdis = fs.open(testFile);
+      try {
+        assertEquals("Wrong file content", testString, fsdis.readUTF());
+      } finally {
+        fsdis.close();
+      }
+    }
+
+    // 3. test reads when one unavailable
+    //
+    // bring one NN down and read through nfly should still work
+    //
+    for (int i = 0; i < cluster.getNumNameNodes(); i++) {
+      cluster.shutdownNameNode(i);
+      FSDataInputStream fsDis = null;
+      try {
+        fsDis = nfly.open(testFile);
+        assertEquals("Wrong file content", testString, fsDis.readUTF());
+      } finally {
+        IOUtils.cleanupWithLogger(LOG, fsDis);
+        cluster.restartNameNode(i);
+      }
+    }
+
+    // both nodes are up again, test repair
+    final FileSystem fs1 = FileSystem.get(testUris[0], conf);
+    assertTrue(fs1.delete(testFile, false));
+    assertFalse(fs1.exists(testFile));
+    FSDataInputStream fsDis = null;
+    try {
+      fsDis = nfly.open(testFile);
+      assertEquals("Wrong file content", testString, fsDis.readUTF());
+      assertTrue(fs1.exists(testFile));
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, fsDis);
+    }
+
+    // test most recent repair
+    if (repairKey == NflyFSystem.NflyKey.readMostRecent) {
+      final FileSystem fs2 = FileSystem.get(testUris[0], conf);
+      final long expectedMtime = fs2.getFileStatus(testFile)
+          .getModificationTime();
+
+      for (final URI testUri : testUris) {
+        final FileSystem fs = FileSystem.get(testUri, conf);
+        fs.setTimes(testFile, 1L, 1L);
+        assertEquals(testUri + "Set mtime failed!", 1L,
+            fs.getFileStatus(testFile).getModificationTime());
+        assertEquals("nfly file status wrong", expectedMtime,
+            nfly.getFileStatus(testFile).getModificationTime());
+        FSDataInputStream fsDis2 = null;
+        try {
+          fsDis2 = nfly.open(testFile);
+          assertEquals("Wrong file content", testString, fsDis2.readUTF());
+          // repair is done, now trying via normal fs
+          //
+          assertEquals("Repair most recent failed!", expectedMtime,
+              fs.getFileStatus(testFile).getModificationTime());
+        } finally {
+          IOUtils.cleanupWithLogger(LOG, fsDis2);
+        }
+      }
+    }
+  }
 }


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


[25/29] hadoop git commit: HDFS-10882. Federation State Store Interface API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10882. Federation State Store Interface API. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 98efaf668ab45b1bbbd1370f64c92ec0784dfb5e
Parents: d893c33
Author: Inigo <in...@apache.org>
Authored: Thu Apr 6 19:18:52 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  11 ++
 .../server/federation/store/RecordStore.java    | 100 ++++++++++++++++
 .../store/driver/StateStoreSerializer.java      | 119 +++++++++++++++++++
 .../driver/impl/StateStoreSerializerPBImpl.java | 115 ++++++++++++++++++
 .../store/records/impl/pb/PBRecord.java         |  47 ++++++++
 .../store/records/impl/pb/package-info.java     |  29 +++++
 .../src/main/resources/hdfs-default.xml         |   8 ++
 7 files changed, 429 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/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 ce0a17a..7623839 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
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl;
 import org.apache.hadoop.http.HttpConfig;
 
 /** 
@@ -1123,6 +1124,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
       "org.apache.hadoop.hdfs.server.federation.MockResolver";
 
+  // HDFS Router-based federation State Store
+  public static final String FEDERATION_STORE_PREFIX =
+      FEDERATION_ROUTER_PREFIX + "store.";
+
+  public static final String FEDERATION_STORE_SERIALIZER_CLASS =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "serializer";
+  public static final Class<StateStoreSerializerPBImpl>
+      FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT =
+          StateStoreSerializerPBImpl.class;
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
new file mode 100644
index 0000000..524f432
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
@@ -0,0 +1,100 @@
+/**
+ * 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.store;
+
+import java.lang.reflect.Constructor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * Store records in the State Store. Subclasses provide interfaces to operate on
+ * those records.
+ *
+ * @param <R> Record to store by this interface.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class RecordStore<R extends BaseRecord> {
+
+  private static final Log LOG = LogFactory.getLog(RecordStore.class);
+
+
+  /** Class of the record stored in this State Store. */
+  private final Class<R> recordClass;
+
+  /** State store driver backed by persistent storage. */
+  private final StateStoreDriver driver;
+
+
+  /**
+   * Create a new store for records.
+   *
+   * @param clazz Class of the record to store.
+   * @param stateStoreDriver Driver for the State Store.
+   */
+  protected RecordStore(Class<R> clazz, StateStoreDriver stateStoreDriver) {
+    this.recordClass = clazz;
+    this.driver = stateStoreDriver;
+  }
+
+  /**
+   * Report a required record to the data store. The data store uses this to
+   * create/maintain storage for the record.
+   *
+   * @return The class of the required record or null if no record is required
+   *         for this interface.
+   */
+  public Class<R> getRecordClass() {
+    return this.recordClass;
+  }
+
+  /**
+   * Get the State Store driver.
+   *
+   * @return State Store driver.
+   */
+  protected StateStoreDriver getDriver() {
+    return this.driver;
+  }
+
+  /**
+   * Build a state store API implementation interface.
+   *
+   * @param interfaceClass The specific interface implementation to create
+   * @param driver The {@link StateStoreDriver} implementation in use.
+   * @return An initialized instance of the specified state store API
+   *         implementation.
+   */
+  public static <T extends RecordStore<?>> T newInstance(
+      final Class<T> clazz, final StateStoreDriver driver) {
+
+    try {
+      Constructor<T> constructor = clazz.getConstructor(StateStoreDriver.class);
+      T recordStore = constructor.newInstance(driver);
+      return recordStore;
+    } catch (Exception e) {
+      LOG.error("Cannot create new instance for " + clazz, e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
new file mode 100644
index 0000000..8540405
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
@@ -0,0 +1,119 @@
+/**
+ * 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.store.driver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Serializer to store and retrieve data in the State Store.
+ */
+public abstract class StateStoreSerializer {
+
+  /** Singleton for the serializer instance. */
+  private static StateStoreSerializer defaultSerializer;
+
+  /**
+   * Get the default serializer based.
+   * @return Singleton serializer.
+   */
+  public static StateStoreSerializer getSerializer() {
+    return getSerializer(null);
+  }
+
+  /**
+   * Get a serializer based on the provided configuration.
+   * @param conf Configuration. Default if null.
+   * @return Singleton serializer.
+   */
+  public static StateStoreSerializer getSerializer(Configuration conf) {
+    if (conf == null) {
+      synchronized (StateStoreSerializer.class) {
+        if (defaultSerializer == null) {
+          conf = new Configuration();
+          defaultSerializer = newSerializer(conf);
+        }
+      }
+      return defaultSerializer;
+    } else {
+      return newSerializer(conf);
+    }
+  }
+
+  private static StateStoreSerializer newSerializer(final Configuration conf) {
+    Class<? extends StateStoreSerializer> serializerName = conf.getClass(
+        DFSConfigKeys.FEDERATION_STORE_SERIALIZER_CLASS,
+        DFSConfigKeys.FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT,
+        StateStoreSerializer.class);
+    return ReflectionUtils.newInstance(serializerName, conf);
+  }
+
+  /**
+   * Create a new record.
+   * @param clazz Class of the new record.
+   * @return New record.
+   */
+  public static <T> T newRecord(Class<T> clazz) {
+    return getSerializer(null).newRecordInstance(clazz);
+  }
+
+  /**
+   * Create a new record.
+   * @param clazz Class of the new record.
+   * @return New record.
+   */
+  public abstract <T> T newRecordInstance(Class<T> clazz);
+
+  /**
+   * Serialize a record into a byte array.
+   * @param record Record to serialize.
+   * @return Byte array with the serialized record.
+   */
+  public abstract byte[] serialize(BaseRecord record);
+
+  /**
+   * Serialize a record into a string.
+   * @param record Record to serialize.
+   * @return String with the serialized record.
+   */
+  public abstract String serializeString(BaseRecord record);
+
+  /**
+   * Deserialize a bytes array into a record.
+   * @param byteArray Byte array to deserialize.
+   * @param clazz Class of the record.
+   * @return New record.
+   * @throws IOException If it cannot deserialize the record.
+   */
+  public abstract <T extends BaseRecord> T deserialize(
+      byte[] byteArray, Class<T> clazz) throws IOException;
+
+  /**
+   * Deserialize a string into a record.
+   * @param data String with the data to deserialize.
+   * @param clazz Class of the record.
+   * @return New record.
+   * @throws IOException If it cannot deserialize the record.
+   */
+  public abstract <T extends BaseRecord> T deserialize(
+      String data, Class<T> clazz) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java
new file mode 100644
index 0000000..45c5dd6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.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.federation.store.driver.impl;
+
+import java.io.IOException;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the State Store serializer.
+ */
+public final class StateStoreSerializerPBImpl extends StateStoreSerializer {
+
+  private static final String PB_IMPL_PACKAGE_SUFFIX = "impl.pb";
+  private static final String PB_IMPL_CLASS_SUFFIX = "PBImpl";
+
+  private Configuration localConf = new Configuration();
+
+
+  private StateStoreSerializerPBImpl() {
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> T newRecordInstance(Class<T> clazz) {
+    try {
+      String clazzPBImpl = getPBImplClassName(clazz);
+      Class<?> pbClazz = localConf.getClassByName(clazzPBImpl);
+      Object retObject = ReflectionUtils.newInstance(pbClazz, localConf);
+      return (T)retObject;
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private String getPBImplClassName(Class<?> clazz) {
+    String srcPackagePart = getPackageName(clazz);
+    String srcClassName = getClassName(clazz);
+    String destPackagePart = srcPackagePart + "." + PB_IMPL_PACKAGE_SUFFIX;
+    String destClassPart = srcClassName + PB_IMPL_CLASS_SUFFIX;
+    return destPackagePart + "." + destClassPart;
+  }
+
+  private String getClassName(Class<?> clazz) {
+    String fqName = clazz.getName();
+    return (fqName.substring(fqName.lastIndexOf(".") + 1, fqName.length()));
+  }
+
+  private String getPackageName(Class<?> clazz) {
+    return clazz.getPackage().getName();
+  }
+
+  @Override
+  public byte[] serialize(BaseRecord record) {
+    byte[] byteArray64 = null;
+    if (record instanceof PBRecord) {
+      PBRecord recordPB = (PBRecord) record;
+      Message msg = recordPB.getProto();
+      byte[] byteArray = msg.toByteArray();
+      byteArray64 = Base64.encodeBase64(byteArray, false);
+    }
+    return byteArray64;
+  }
+
+  @Override
+  public String serializeString(BaseRecord record) {
+    byte[] byteArray64 = serialize(record);
+    String base64Encoded = StringUtils.newStringUtf8(byteArray64);
+    return base64Encoded;
+  }
+
+  @Override
+  public <T extends BaseRecord> T deserialize(
+      byte[] byteArray, Class<T> clazz) throws IOException {
+
+    T record = newRecord(clazz);
+    if (record instanceof PBRecord) {
+      PBRecord pbRecord = (PBRecord) record;
+      byte[] byteArray64 = Base64.encodeBase64(byteArray, false);
+      String base64Encoded = StringUtils.newStringUtf8(byteArray64);
+      pbRecord.readInstance(base64Encoded);
+    }
+    return record;
+  }
+
+  @Override
+  public <T extends BaseRecord> T deserialize(String data, Class<T> clazz)
+      throws IOException {
+    byte[] byteArray64 = Base64.decodeBase64(data);
+    return deserialize(byteArray64, clazz);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java
new file mode 100644
index 0000000..c369275
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java
@@ -0,0 +1,47 @@
+/**
+ * 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.store.records.impl.pb;
+
+import java.io.IOException;
+
+import com.google.protobuf.Message;
+
+/**
+ * A record implementation using Protobuf.
+ */
+public interface PBRecord {
+
+  /**
+   * Get the protocol for the record.
+   * @return The protocol for this record.
+   */
+  Message getProto();
+
+  /**
+   * Set the protocol for the record.
+   * @param proto Protocol for this record.
+   */
+  void setProto(Message proto);
+
+  /**
+   * Populate this record with serialized data.
+   * @param base64String Serialized data in base64.
+   * @throws IOException If it cannot read the data.
+   */
+  void readInstance(String base64String) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java
new file mode 100644
index 0000000..b329732
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.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.
+ */
+
+/**
+ * The protobuf implementations of state store data records defined in the
+ * org.apache.hadoop.hdfs.server.federation.store.records package. Each
+ * implementation wraps an associated protobuf proto definition.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98efaf66/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 6e31388..f1dce6b 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
@@ -4635,4 +4635,12 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.store.serializer</name>
+    <value>org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl</value>
+    <description>
+      Class to serialize State Store records.
+    </description>
+  </property>
+
 </configuration>


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


[16/29] hadoop git commit: HDFS-12223. Rebasing HDFS-10467. Contributed by Inigo Goiri.

Posted by in...@apache.org.
HDFS-12223. Rebasing HDFS-10467. Contributed by Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 4d1180e51e185e033fae07803698cbf04f8a262e
Parents: 0b381a9
Author: Inigo Goiri <in...@apache.org>
Authored: Fri Jul 28 15:55:10 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../federation/router/RouterRpcServer.java      | 59 +++++++++++++++++---
 1 file changed, 51 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d1180e5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 4bae71e..eaaab39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -64,8 +64,9 @@ import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AddingECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlocksStats;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -75,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -85,6 +87,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -1736,13 +1739,6 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   }
 
   @Override // ClientProtocol
-  public AddingECPolicyResponse[] addErasureCodingPolicies(
-      ErasureCodingPolicy[] policies) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
   public void unsetErasureCodingPolicy(String src) throws IOException {
     checkOperation(OperationCategory.WRITE, false);
   }
@@ -1808,6 +1804,53 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
   }
 
+  @Override
+  public AddECPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] arg0) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
+  }
+
+  @Override
+  public void removeErasureCodingPolicy(String arg0) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void disableErasureCodingPolicy(String arg0) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void enableErasureCodingPolicy(String arg0) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public ECBlockGroupsStats getECBlockGroupsStats() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public BlocksStats getBlocksStats() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public BatchedEntries<OpenFileEntry> listOpenFiles(long arg0)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
   /**
    * Locate the location with the matching block pool id.
    *


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


[08/29] hadoop git commit: HDFS-10881. Federation State Store Driver API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10881. Federation State Store Driver API. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: d893c335cae363cee5229e893a72af2f6bd93924
Parents: f7ffbf3
Author: Inigo <in...@apache.org>
Authored: Wed Mar 29 19:35:06 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:43 2017 -0700

----------------------------------------------------------------------
 .../store/StateStoreUnavailableException.java   |  33 ++++
 .../federation/store/StateStoreUtils.java       |  72 +++++++
 .../store/driver/StateStoreDriver.java          | 172 +++++++++++++++++
 .../driver/StateStoreRecordOperations.java      | 164 ++++++++++++++++
 .../store/driver/impl/StateStoreBaseImpl.java   |  69 +++++++
 .../store/driver/impl/package-info.java         |  39 ++++
 .../federation/store/driver/package-info.java   |  37 ++++
 .../federation/store/protocol/package-info.java |  31 +++
 .../federation/store/records/BaseRecord.java    | 189 +++++++++++++++++++
 .../federation/store/records/QueryResult.java   |  56 ++++++
 .../federation/store/records/package-info.java  |  36 ++++
 11 files changed, 898 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java
new file mode 100644
index 0000000..4e6f8c8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.store;
+
+import java.io.IOException;
+
+/**
+ * Thrown when the state store is not reachable or available. Cached APIs and
+ * queries may succeed. Client should retry again later.
+ */
+public class StateStoreUnavailableException extends IOException {
+
+  private static final long serialVersionUID = 1L;
+
+  public StateStoreUnavailableException(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
new file mode 100644
index 0000000..8c681df
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
@@ -0,0 +1,72 @@
+/**
+ * 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.store;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * Set of utility functions used to query, create, update and delete data
+ * records in the state store.
+ */
+public final class StateStoreUtils {
+
+  private static final Log LOG = LogFactory.getLog(StateStoreUtils.class);
+
+  private StateStoreUtils() {
+    // Utility class
+  }
+
+  /**
+   * Get the base class for a record class. If we get an implementation of a
+   * record we will return the real parent record class.
+   *
+   * @param clazz Class of the data record to check.
+   * @return Base class for the record.
+   */
+  @SuppressWarnings("unchecked")
+  public static <T extends BaseRecord>
+      Class<? extends BaseRecord> getRecordClass(final Class<T> clazz) {
+
+    // We ignore the Impl classes and go to the super class
+    Class<? extends BaseRecord> actualClazz = clazz;
+    while (actualClazz.getSimpleName().endsWith("Impl")) {
+      actualClazz = (Class<? extends BaseRecord>) actualClazz.getSuperclass();
+    }
+
+    // Check if we went too far
+    if (actualClazz.equals(BaseRecord.class)) {
+      LOG.error("We went too far (" + actualClazz + ") with " + clazz);
+      actualClazz = clazz;
+    }
+    return actualClazz;
+  }
+
+  /**
+   * Get the base class for a record. If we get an implementation of a record we
+   * will return the real parent record class.
+   *
+   * @param record Record to check its main class.
+   * @return Base class for the record.
+   */
+  public static <T extends BaseRecord>
+      Class<? extends BaseRecord> getRecordClass(final T record) {
+    return getRecordClass(record.getClass());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
new file mode 100644
index 0000000..a1527df
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
@@ -0,0 +1,172 @@
+/**
+ * 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.store.driver;
+
+import java.net.InetAddress;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Driver class for an implementation of a {@link StateStoreService}
+ * provider. Driver implementations will extend this class and implement some of
+ * the default methods.
+ */
+public abstract class StateStoreDriver implements StateStoreRecordOperations {
+
+  private static final Log LOG = LogFactory.getLog(StateStoreDriver.class);
+
+
+  /** State Store configuration. */
+  private Configuration conf;
+
+  /** Identifier for the driver. */
+  private String identifier;
+
+
+  /**
+   * Initialize the state store connection.
+   * @param config Configuration for the driver.
+   * @param id Identifier for the driver.
+   * @param records Records that are supported.
+   * @return If initialized and ready, false if failed to initialize driver.
+   */
+  public boolean init(final Configuration config, final String id,
+      final List<Class<? extends BaseRecord>> records) {
+
+    this.conf = config;
+    this.identifier = id;
+
+    if (this.identifier == null) {
+      LOG.warn("The identifier for the State Store connection is not set");
+    }
+
+    // TODO stub
+    return false;
+  }
+
+  /**
+   * Get the State Store configuration.
+   *
+   * @return Configuration for the State Store.
+   */
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
+   * Gets a unique identifier for the running task/process. Typically the
+   * router address.
+   *
+   * @return Unique identifier for the running task.
+   */
+  public String getIdentifier() {
+    return this.identifier;
+  }
+
+  /**
+   * Prepare the driver to access data storage.
+   *
+   * @return True if the driver was successfully initialized. If false is
+   *         returned, the state store will periodically attempt to
+   *         re-initialize the driver and the router will remain in safe mode
+   *         until the driver is initialized.
+   */
+  public abstract boolean initDriver();
+
+  /**
+   * Initialize storage for a single record class.
+   *
+   * @param name String reference of the record class to initialize, used to
+   *             construct paths and file names for the record. Determined by
+   *             configuration settings for the specific driver.
+   * @param clazz Record type corresponding to the provided name.
+   * @return True if successful, false otherwise.
+   */
+  public abstract <T extends BaseRecord> boolean initRecordStorage(
+      String className, Class<T> clazz);
+
+  /**
+   * Check if the driver is currently running and the data store connection is
+   * valid.
+   *
+   * @return True if the driver is initialized and the data store is ready.
+   */
+  public abstract boolean isDriverReady();
+
+  /**
+   * Check if the driver is ready to be used and throw an exception otherwise.
+   *
+   * @throws StateStoreUnavailableException If the driver is not ready.
+   */
+  public void verifyDriverReady() throws StateStoreUnavailableException {
+    if (!isDriverReady()) {
+      String driverName = getDriverName();
+      String hostname = getHostname();
+      throw new StateStoreUnavailableException("State Store driver " +
+          driverName + " in " + hostname + " is not ready.");
+    }
+  }
+
+  /**
+   * Close the State Store driver connection.
+   */
+  public abstract void close() throws Exception;
+
+  /**
+   * Returns the current time synchronization from the underlying store.
+   * Override for stores that supply a current date. The data store driver is
+   * responsible for maintaining the official synchronization time/date for all
+   * distributed components.
+   *
+   * @return Current time stamp, used for all synchronization dates.
+   */
+  public long getTime() {
+    return Time.now();
+  }
+
+  /**
+   * Get the name of the driver implementation for debugging.
+   *
+   * @return Name of the driver implementation.
+   */
+  private String getDriverName() {
+    return this.getClass().getSimpleName();
+  }
+
+  /**
+   * Get the host name of the machine running the driver for debugging.
+   *
+   * @return Host name of the machine running the driver.
+   */
+  private String getHostname() {
+    String hostname = "Unknown";
+    try {
+      hostname = InetAddress.getLocalHost().getHostName();
+    } catch (Exception e) {
+      LOG.error("Cannot get local address", e);
+    }
+    return hostname;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
new file mode 100644
index 0000000..739eeba
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
@@ -0,0 +1,164 @@
+/**
+ * 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.store.driver;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.apache.hadoop.io.retry.AtMostOnce;
+import org.apache.hadoop.io.retry.Idempotent;
+
+/**
+ * Operations for a driver to manage records in the State Store.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface StateStoreRecordOperations {
+
+  /**
+   * Get all records of the requested record class from the data store. To use
+   * the default implementations in this class, getAll must return new instances
+   * of the records on each call. It is recommended to override the default
+   * implementations for better performance.
+   *
+   * @param clazz Class of record to fetch.
+   * @return List of all records that match the clazz.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @Idempotent
+  <T extends BaseRecord> QueryResult<T> get(Class<T> clazz) throws IOException;
+
+  /**
+   * Get all records of the requested record class from the data store. To use
+   * the default implementations in this class, getAll must return new instances
+   * of the records on each call. It is recommended to override the default
+   * implementations for better performance.
+   *
+   * @param clazz Class of record to fetch.
+   * @param sub Sub path.
+   * @return List of all records that match the clazz and the sub path.
+   * @throws IOException
+   */
+  @Idempotent
+  <T extends BaseRecord> QueryResult<T> get(Class<T> clazz, String sub)
+      throws IOException;
+
+  /**
+   * Get a single record from the store that matches the query.
+   *
+   * @param clazz Class of record to fetch.
+   * @param query Map of field names and objects to filter results.
+   * @return A single record matching the query. Null if there are no matching
+   *         records or more than one matching record in the store.
+   * @throws IOException If multiple records match or if the data store cannot
+   *           be queried.
+   */
+  @Idempotent
+  <T extends BaseRecord> T get(Class<T> clazz, Map<String, String> query)
+      throws IOException;
+
+  /**
+   * Get multiple records from the store that match a query. This method
+   * assumes the underlying driver does not support filtering. If the driver
+   * supports filtering it should overwrite this method.
+   *
+   * @param clazz Class of record to fetch.
+   * @param query Map of field names and objects to filter results.
+   * @return Records of type clazz that match the query or empty list if none
+   *         are found.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @Idempotent
+  <T extends BaseRecord> List<T> getMultiple(
+      Class<T> clazz, Map<String, String> query) throws IOException;
+
+  /**
+   * Creates a single record. Optionally updates an existing record with same
+   * primary key.
+   *
+   * @param record The record to insert or update.
+   * @param allowUpdate True if update of exiting record is allowed.
+   * @param errorIfExists True if an error should be returned when inserting
+   *          an existing record. Only used if allowUpdate = false.
+   * @return True if the operation was successful.
+   *
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean put(
+       T record, boolean allowUpdate, boolean errorIfExists) throws IOException;
+
+  /**
+   * Creates multiple records. Optionally updates existing records that have
+   * the same primary key.
+   *
+   * @param records List of data records to update or create. All records must
+   *                be of class clazz.
+   * @param clazz Record class of records.
+   * @param allowUpdate True if update of exiting record is allowed.
+   * @param errorIfExists True if an error should be returned when inserting
+   *          an existing record. Only used if allowUpdate = false.
+   * @return true if all operations were successful.
+   *
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean putAll(
+      List<T> records, boolean allowUpdate, boolean errorIfExists)
+          throws IOException;
+
+  /**
+   * Remove a single record.
+   *
+   * @param record Record to be removed.
+   * @return true If the record was successfully removed. False if the record
+   *              could not be removed or not stored.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean remove(T record) throws IOException;
+
+  /**
+   * Remove all records of this class from the store.
+   *
+   * @param clazz Class of records to remove.
+   * @return True if successful.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean removeAll(Class<T> clazz) throws IOException;
+
+  /**
+   * Remove multiple records of a specific class that match a query. Requires
+   * the getAll implementation to fetch fresh records on each call.
+   *
+   * @param clazz Class of record to remove.
+   * @param filter matching filter to remove.
+   * @return The number of records removed.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> int remove(Class<T> clazz, Map<String, String> filter)
+      throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
new file mode 100644
index 0000000..b711fa9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
@@ -0,0 +1,69 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * Base implementation of a State Store driver. It contains default
+ * implementations for the optional functions. These implementations use an
+ * uncached read/write all algorithm for all changes. In most cases it is
+ * recommended to override the optional functions.
+ * <p>
+ * Drivers may optionally override additional routines for performance
+ * optimization, such as custom get/put/remove queries, depending on the
+ * capabilities of the data store.
+ * <p>
+ */
+public abstract class StateStoreBaseImpl extends StateStoreDriver {
+
+  @Override
+  public <T extends BaseRecord> T get(
+      Class<T> clazz, Map<String, String> query) throws IOException {
+    List<T> records = getMultiple(clazz, query);
+    if (records.size() > 1) {
+      throw new IOException("Found more than one object in collection");
+    } else if (records.size() == 1) {
+      return records.get(0);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean put(
+      T record, boolean allowUpdate, boolean errorIfExists) throws IOException {
+    List<T> singletonList = new ArrayList<T>();
+    singletonList.add(record);
+    return putAll(singletonList, allowUpdate, errorIfExists);
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean remove(T record) throws IOException {
+    Map<String, String> primaryKeys = record.getPrimaryKeys();
+    Class<? extends BaseRecord> clazz = StateStoreUtils.getRecordClass(record);
+    return remove(clazz, primaryKeys) == 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java
new file mode 100644
index 0000000..a18433e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java
@@ -0,0 +1,39 @@
+/**
+ * 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.
+ */
+
+/**
+ * Implementations of state store data providers/drivers. Each driver is
+ * responsible for maintaining, querying, updating and deleting persistent data
+ * records. Data records are defined as subclasses of
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord}.
+ * Each driver implements the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} interface.
+ * <p>
+ * Currently supported drivers:
+ * <ul>
+ * <li>{@link StateStoreFileImpl} A file-based data storage backend.
+ * <li>{@link StateStoreZooKeeperImpl} A zookeeper based data storage backend.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.store.driver.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java
new file mode 100644
index 0000000..da998b5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.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.
+ */
+
+/**
+ * The state store uses modular data storage classes derived from
+ * StateStoreDriver to handle querying, updating and deleting data records. The
+ * data storage driver is initialized and maintained by the StateStoreService.
+ * The state store supports fetching all records of a type, filtering by column
+ * values or fetching a single record by its primary key.
+ * <p>
+ * Each data storage backend is required to implement the methods contained in
+ * the StateStoreDriver interface. These methods allow the querying, updating,
+ * inserting and deleting of data records into the state store.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store.driver;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java
new file mode 100644
index 0000000..0249d2c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains the abstract definitions of the API request and response objects for
+ * the various state store APIs. The state store supports multiple interface
+ * APIs and multiple data records. Each protocol object requires a serialization
+ * implementation, the default is protobuf.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
new file mode 100644
index 0000000..4192a3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
@@ -0,0 +1,189 @@
+/**
+ * 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.store.records;
+
+import java.util.Map;
+
+import org.apache.hadoop.util.Time;
+
+/**
+ * Abstract base of a data record in the StateStore. All StateStore records are
+ * derived from this class. Data records are persisted in the data store and
+ * are identified by their primary key. Each data record contains:
+ * <ul>
+ * <li>A primary key consisting of a combination of record data fields.
+ * <li>A modification date.
+ * <li>A creation date.
+ * </ul>
+ */
+public abstract class BaseRecord implements Comparable<BaseRecord> {
+
+  /**
+   * Set the modification time for the record.
+   *
+   * @param time Modification time of the record.
+   */
+  public abstract void setDateModified(long time);
+
+  /**
+   * Get the modification time for the record.
+   *
+   * @return Modification time of the record.
+   */
+  public abstract long getDateModified();
+
+  /**
+   * Set the creation time for the record.
+   *
+   * @param time Creation time of the record.
+   */
+  public abstract void setDateCreated(long time);
+
+  /**
+   * Get the creation time for the record.
+   *
+   * @return Creation time of the record
+   */
+  public abstract long getDateCreated();
+
+  /**
+   * Get the expiration time for the record.
+   *
+   * @return Expiration time for the record.
+   */
+  public abstract long getExpirationMs();
+
+  /**
+   * Map of primary key names->values for the record. The primary key can be a
+   * combination of 1-n different State Store serialized values.
+   *
+   * @return Map of key/value pairs that constitute this object's primary key.
+   */
+  public abstract Map<String, String> getPrimaryKeys();
+
+  /**
+   * Initialize the object.
+   */
+  public void init() {
+    // Call this after the object has been constructed
+    initDefaultTimes();
+  }
+
+  /**
+   * Initialize default times. The driver may update these timestamps on insert
+   * and/or update. This should only be called when initializing an object that
+   * is not backed by a data store.
+   */
+  private void initDefaultTimes() {
+    long now = Time.now();
+    this.setDateCreated(now);
+    this.setDateModified(now);
+  }
+
+  /**
+   * Join the primary keys into one single primary key.
+   *
+   * @return A string that is guaranteed to be unique amongst all records of
+   *         this type.
+   */
+  public String getPrimaryKey() {
+    return generateMashupKey(getPrimaryKeys());
+  }
+
+  /**
+   * Generates a cache key from a map of values.
+   *
+   * @param keys Map of values.
+   * @return String mashup of key values.
+   */
+  protected static String generateMashupKey(final Map<String, String> keys) {
+    StringBuilder builder = new StringBuilder();
+    for (Object value : keys.values()) {
+      if (builder.length() > 0) {
+        builder.append("-");
+      }
+      builder.append(value);
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Override equals check to use primary key(s) for comparison.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof BaseRecord)) {
+      return false;
+    }
+
+    BaseRecord baseObject = (BaseRecord) obj;
+    Map<String, String> keyset1 = this.getPrimaryKeys();
+    Map<String, String> keyset2 = baseObject.getPrimaryKeys();
+    return keyset1.equals(keyset2);
+  }
+
+  /**
+   * Override hash code to use primary key(s) for comparison.
+   */
+  @Override
+  public int hashCode() {
+    Map<String, String> keyset = this.getPrimaryKeys();
+    return keyset.hashCode();
+  }
+
+  @Override
+  public int compareTo(BaseRecord record) {
+    if (record == null) {
+      return -1;
+    }
+    // Descending date order
+    return (int) (record.getDateModified() - this.getDateModified());
+  }
+
+  /**
+   * Called when the modification time and current time is available, checks for
+   * expirations.
+   *
+   * @param currentTime The current timestamp in ms from the data store, to be
+   *          compared against the modification and creation dates of the
+   *          object.
+   * @return boolean True if the record has been updated and should be
+   *         committed to the data store. Override for customized behavior.
+   */
+  public boolean checkExpired(long currentTime) {
+    long expiration = getExpirationMs();
+    if (getDateModified() > 0 && expiration > 0) {
+      return (getDateModified() + expiration) < currentTime;
+    }
+    return false;
+  }
+
+  /**
+   * Validates the record. Called when the record is created, populated from the
+   * state store, and before committing to the state store.
+   * @return If the record is valid.
+   */
+  public boolean validate() {
+    return getDateCreated() > 0 && getDateModified() > 0;
+  }
+
+  @Override
+  public String toString() {
+    return getPrimaryKey();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java
new file mode 100644
index 0000000..64c2c71
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java
@@ -0,0 +1,56 @@
+/**
+ * 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.store.records;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Encapsulates a state store query result that includes a set of records and a
+ * time stamp for the result.
+ */
+public class QueryResult<T extends BaseRecord> {
+
+  /** Data result. */
+  private final List<T> records;
+  /** Time stamp of the data results. */
+  private final long timestamp;
+
+  public QueryResult(final List<T> recs, final long time) {
+    this.records = recs;
+    this.timestamp = time;
+  }
+
+  /**
+   * Get the result of the query.
+   *
+   * @return List of records.
+   */
+  public List<T> getRecords() {
+    return Collections.unmodifiableList(this.records);
+  }
+
+  /**
+   * The timetamp in driver time of this query.
+   *
+   * @return Timestamp in driver time.
+   */
+  public long getTimestamp() {
+    return this.timestamp;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d893c335/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java
new file mode 100644
index 0000000..63b13af
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java
@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+
+/**
+ * Contains the abstract definitions of the state store data records. The state
+ * store supports multiple multiple data records.
+ * <p>
+ * Data records inherit from a common class
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord
+ * BaseRecord}. Data records are serialized when written to the data store using
+ * a modular serialization implementation. The default is profobuf
+ * serialization. Data is stored as rows of records of the same type with each
+ * data member in a record representing a column.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;


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


[15/29] hadoop git commit: HDFS-11826. Federation Namenode Heartbeat. Contributed by Inigo Goiri.

Posted by in...@apache.org.
HDFS-11826. Federation Namenode Heartbeat. Contributed by Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 740ff17415c558f8f3eb5407964a81f543accbd6
Parents: 79fce5e
Author: Inigo Goiri <in...@apache.org>
Authored: Tue Aug 1 14:40:27 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  14 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  38 ++
 .../resolver/NamenodeStatusReport.java          | 193 ++++++++++
 .../federation/router/FederationUtil.java       |  66 ++++
 .../router/NamenodeHeartbeatService.java        | 350 +++++++++++++++++++
 .../hdfs/server/federation/router/Router.java   | 112 ++++++
 .../src/main/resources/hdfs-default.xml         |  32 ++
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   8 +
 .../hdfs/server/federation/MockResolver.java    |   9 +-
 .../server/federation/RouterConfigBuilder.java  |  22 ++
 .../server/federation/RouterDFSCluster.java     |  43 +++
 .../router/TestNamenodeHeartbeat.java           | 168 +++++++++
 .../server/federation/router/TestRouter.java    |   3 +
 13 files changed, 1057 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/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 afb5bbf..d1c2b41 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
@@ -1144,6 +1144,20 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       FEDERATION_ROUTER_PREFIX + "rpc.enable";
   public static final boolean DFS_ROUTER_RPC_ENABLE_DEFAULT = true;
 
+  // HDFS Router heartbeat
+  public static final String DFS_ROUTER_HEARTBEAT_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "heartbeat.enable";
+  public static final boolean DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT = true;
+  public static final String DFS_ROUTER_HEARTBEAT_INTERVAL_MS =
+      FEDERATION_ROUTER_PREFIX + "heartbeat.interval";
+  public static final long DFS_ROUTER_HEARTBEAT_INTERVAL_MS_DEFAULT =
+      TimeUnit.SECONDS.toMillis(5);
+  public static final String DFS_ROUTER_MONITOR_NAMENODE =
+      FEDERATION_ROUTER_PREFIX + "monitor.namenode";
+  public static final String DFS_ROUTER_MONITOR_LOCAL_NAMENODE =
+      FEDERATION_ROUTER_PREFIX + "monitor.localnamenode.enable";
+  public static final boolean DFS_ROUTER_MONITOR_LOCAL_NAMENODE_DEFAULT = true;
+
   // HDFS Router NN client
   public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE =
       FEDERATION_ROUTER_PREFIX + "connection.pool-size";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7776dc2..29936f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1241,6 +1241,44 @@ public class DFSUtil {
   }
 
   /**
+   * Map a logical namenode ID to its web address. Use the given nameservice if
+   * specified, or the configured one if none is given.
+   *
+   * @param conf Configuration
+   * @param nsId which nameservice nnId is a part of, optional
+   * @param nnId the namenode ID to get the service addr for
+   * @return the service addr, null if it could not be determined
+   */
+  public static String getNamenodeWebAddr(final Configuration conf, String nsId,
+      String nnId) {
+
+    if (nsId == null) {
+      nsId = getOnlyNameServiceIdOrNull(conf);
+    }
+
+    String webAddrKey = DFSUtilClient.concatSuffixes(
+        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nsId, nnId);
+
+    String webAddr =
+        conf.get(webAddrKey, DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_DEFAULT);
+    return webAddr;
+  }
+
+  /**
+   * Get all of the Web addresses of the individual NNs in a given nameservice.
+   *
+   * @param conf Configuration
+   * @param nsId the nameservice whose NNs addresses we want.
+   * @param defaultValue default address to return in case key is not found.
+   * @return A map from nnId -> Web address of each NN in the nameservice.
+   */
+  public static Map<String, InetSocketAddress> getWebAddressesForNameserviceId(
+      Configuration conf, String nsId, String defaultValue) {
+    return DFSUtilClient.getAddressesForNameserviceId(conf, nsId, defaultValue,
+        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+  }
+
+  /**
    * If the configuration refers to only a single nameservice, return the
    * name of that nameservice. If it refers to 0 or more than 1, return null.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/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 9259048..f8759e8 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
@@ -39,8 +39,29 @@ public class NamenodeStatusReport {
   private HAServiceState status = HAServiceState.STANDBY;
   private boolean safeMode = false;
 
+  /** Datanodes stats. */
+  private int liveDatanodes = -1;
+  private int deadDatanodes = -1;
+  /** Decommissioning datanodes. */
+  private int decomDatanodes = -1;
+  /** Live decommissioned datanodes. */
+  private int liveDecomDatanodes = -1;
+  /** Dead decommissioned datanodes. */
+  private int deadDecomDatanodes = -1;
+
+  /** Space stats. */
+  private long availableSpace = -1;
+  private long numOfFiles = -1;
+  private long numOfBlocks = -1;
+  private long numOfBlocksMissing = -1;
+  private long numOfBlocksPendingReplication = -1;
+  private long numOfBlocksUnderReplicated = -1;
+  private long numOfBlocksPendingDeletion = -1;
+  private long totalSpace = -1;
+
   /** If the fields are valid. */
   private boolean registrationValid = false;
+  private boolean statsValid = false;
   private boolean haStateValid = false;
 
   public NamenodeStatusReport(String ns, String nn, String rpc, String service,
@@ -54,6 +75,15 @@ public class NamenodeStatusReport {
   }
 
   /**
+   * If the statistics are valid.
+   *
+   * @return If the statistics are valid.
+   */
+  public boolean statsValid() {
+    return this.statsValid;
+  }
+
+  /**
    * If the registration is valid.
    *
    * @return If the registration is valid.
@@ -187,6 +217,169 @@ public class NamenodeStatusReport {
     return this.safeMode;
   }
 
+  /**
+   * Set the datanode information.
+   *
+   * @param numLive Number of live nodes.
+   * @param numDead Number of dead nodes.
+   * @param numDecom Number of decommissioning nodes.
+   * @param numLiveDecom Number of decommissioned live nodes.
+   * @param numDeadDecom Number of decommissioned dead nodes.
+   */
+  public void setDatanodeInfo(int numLive, int numDead, int numDecom,
+      int numLiveDecom, int numDeadDecom) {
+    this.liveDatanodes = numLive;
+    this.deadDatanodes = numDead;
+    this.decomDatanodes = numDecom;
+    this.liveDecomDatanodes = numLiveDecom;
+    this.deadDecomDatanodes = numDeadDecom;
+    this.statsValid = true;
+  }
+
+  /**
+   * Get the number of live blocks.
+   *
+   * @return The number of dead nodes.
+   */
+  public int getNumLiveDatanodes() {
+    return this.liveDatanodes;
+  }
+
+  /**
+   * Get the number of dead blocks.
+   *
+   * @return The number of dead nodes.
+   */
+  public int getNumDeadDatanodes() {
+    return this.deadDatanodes;
+  }
+
+  /**
+   * Get the number of decommissionining nodes.
+   *
+   * @return The number of decommissionining nodes.
+   */
+  public int getNumDecommissioningDatanodes() {
+    return this.decomDatanodes;
+  }
+
+  /**
+   * Get the number of live decommissioned nodes.
+   *
+   * @return The number of live decommissioned nodes.
+   */
+  public int getNumDecomLiveDatanodes() {
+    return this.liveDecomDatanodes;
+  }
+
+  /**
+   * Get the number of dead decommissioned nodes.
+   *
+   * @return The number of dead decommissioned nodes.
+   */
+  public int getNumDecomDeadDatanodes() {
+    return this.deadDecomDatanodes;
+  }
+
+  /**
+   * Set the filesystem information.
+   *
+   * @param available Available capacity.
+   * @param total Total capacity.
+   * @param numFiles Number of files.
+   * @param numBlocks Total number of blocks.
+   * @param numBlocksMissing Number of missing blocks.
+   * @param numOfBlocksPendingReplication Number of blocks pending replication.
+   * @param numOfBlocksUnderReplicated Number of blocks under replication.
+   * @param numOfBlocksPendingDeletion Number of blocks pending deletion.
+   */
+  public void setNamesystemInfo(long available, long total,
+      long numFiles, long numBlocks, long numBlocksMissing,
+      long numBlocksPendingReplication, long numBlocksUnderReplicated,
+      long numBlocksPendingDeletion) {
+    this.totalSpace = total;
+    this.availableSpace = available;
+    this.numOfBlocks = numBlocks;
+    this.numOfBlocksMissing = numBlocksMissing;
+    this.numOfBlocksPendingReplication = numBlocksPendingReplication;
+    this.numOfBlocksUnderReplicated = numBlocksUnderReplicated;
+    this.numOfBlocksPendingDeletion = numBlocksPendingDeletion;
+    this.numOfFiles = numFiles;
+    this.statsValid = true;
+  }
+
+  /**
+   * Get the number of blocks.
+   *
+   * @return The number of blocks.
+   */
+  public long getNumBlocks() {
+    return this.numOfBlocks;
+  }
+
+  /**
+   * Get the number of files.
+   *
+   * @return The number of files.
+   */
+  public long getNumFiles() {
+    return this.numOfFiles;
+  }
+
+  /**
+   * Get the total space.
+   *
+   * @return The total space.
+   */
+  public long getTotalSpace() {
+    return this.totalSpace;
+  }
+
+  /**
+   * Get the available space.
+   *
+   * @return The available space.
+   */
+  public long getAvailableSpace() {
+    return this.availableSpace;
+  }
+
+  /**
+   * Get the number of missing blocks.
+   *
+   * @return Number of missing blocks.
+   */
+  public long getNumBlocksMissing() {
+    return this.numOfBlocksMissing;
+  }
+
+  /**
+   * Get the number of pending replication blocks.
+   *
+   * @return Number of pending replication blocks.
+   */
+  public long getNumOfBlocksPendingReplication() {
+    return this.numOfBlocksPendingReplication;
+  }
+
+  /**
+   * Get the number of under replicated blocks.
+   *
+   * @return Number of under replicated blocks.
+   */
+  public long getNumOfBlocksUnderReplicated() {
+    return this.numOfBlocksUnderReplicated;
+  }
+
+  /**
+   * Get the number of pending deletion blocks.
+   *
+   * @return Number of pending deletion blocks.
+   */
+  public long getNumOfBlocksPendingDeletion() {
+    return this.numOfBlocksPendingDeletion;
+  }
+
   @Override
   public String toString() {
     return String.format("%s-%s:%s",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
index 0129a37..78c473a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
@@ -17,13 +17,22 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.lang.reflect.Constructor;
+import java.net.URL;
+import java.net.URLConnection;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,6 +49,63 @@ public final class FederationUtil {
   }
 
   /**
+   * Get a JMX data from a web endpoint.
+   *
+   * @param beanQuery JMX bean.
+   * @param webAddress Web address of the JMX endpoint.
+   * @return JSON with the JMX data
+   */
+  public static JSONArray getJmx(String beanQuery, String webAddress) {
+    JSONArray ret = null;
+    BufferedReader reader = null;
+    try {
+      String host = webAddress;
+      int port = -1;
+      if (webAddress.indexOf(":") > 0) {
+        String[] webAddressSplit = webAddress.split(":");
+        host = webAddressSplit[0];
+        port = Integer.parseInt(webAddressSplit[1]);
+      }
+      URL jmxURL = new URL("http", host, port, "/jmx?qry=" + beanQuery);
+      URLConnection conn = jmxURL.openConnection();
+      conn.setConnectTimeout(5 * 1000);
+      conn.setReadTimeout(5 * 1000);
+      InputStream in = conn.getInputStream();
+      InputStreamReader isr = new InputStreamReader(in, "UTF-8");
+      reader = new BufferedReader(isr);
+
+      StringBuilder sb = new StringBuilder();
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        sb.append(line);
+      }
+      String jmxOutput = sb.toString();
+
+      // Parse JSON
+      JSONObject json = new JSONObject(jmxOutput);
+      ret = json.getJSONArray("beans");
+    } catch (IOException e) {
+      LOG.error("Cannot read JMX bean {} from server {}: {}",
+          beanQuery, webAddress, e.getMessage());
+    } catch (JSONException e) {
+      LOG.error("Cannot parse JMX output for {} from server {}: {}",
+          beanQuery, webAddress, e.getMessage());
+    } catch (Exception e) {
+      LOG.error("Cannot parse JMX output for {} from server {}: {}",
+          beanQuery, webAddress, e);
+    } finally {
+      if (reader != null) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Problem closing {}", webAddress, e);
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
    * Create an instance of an interface with a constructor using a context.
    *
    * @param conf Configuration for the class names.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/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
new file mode 100644
index 0000000..fe4f939
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
@@ -0,0 +1,350 @@
+/**
+ * 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.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HEARTBEAT_INTERVAL_MS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HEARTBEAT_INTERVAL_MS_DEFAULT;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The {@link Router} periodically checks the state of a Namenode (usually on
+ * the same server) and reports their high availability (HA) state and
+ * load/space status to the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.StateStoreService}
+ * . Note that this is an optional role as a Router can be independent of any
+ * subcluster.
+ * <p>
+ * For performance with Namenode HA, the Router uses the high availability state
+ * information in the State Store to forward the request to the Namenode that is
+ * most likely to be active.
+ * <p>
+ * Note that this service can be embedded into the Namenode itself to simplify
+ * the operation.
+ */
+public class NamenodeHeartbeatService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NamenodeHeartbeatService.class);
+
+
+  /** Configuration for the heartbeat. */
+  private Configuration conf;
+
+  /** Router performing the heartbeating. */
+  private final ActiveNamenodeResolver resolver;
+
+  /** Interface to the tracked NN. */
+  private final String nameserviceId;
+  private final String namenodeId;
+
+  /** Namenode HA target. */
+  private NNHAServiceTarget localTarget;
+  /** RPC address for the namenode. */
+  private String rpcAddress;
+  /** Service RPC address for the namenode. */
+  private String serviceAddress;
+  /** Service RPC address for the namenode. */
+  private String lifelineAddress;
+  /** HTTP address for the namenode. */
+  private String webAddress;
+
+  /**
+   * Create a new Namenode status updater.
+   * @param resolver Namenode resolver service to handle NN registration.
+   * @param nameserviceId Identifier of the nameservice.
+   * @param namenodeId Identifier of the namenode in HA.
+   */
+  public NamenodeHeartbeatService(
+      ActiveNamenodeResolver resolver, String nsId, String nnId) {
+    super(NamenodeHeartbeatService.class.getSimpleName() + " " + nsId + " " +
+        nnId);
+
+    this.resolver = resolver;
+
+    this.nameserviceId = nsId;
+    this.namenodeId = nnId;
+
+  }
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+
+    this.conf = configuration;
+
+    if (this.namenodeId != null && !this.namenodeId.isEmpty()) {
+      this.localTarget = new NNHAServiceTarget(
+          conf, nameserviceId, namenodeId);
+    } else {
+      this.localTarget = null;
+    }
+
+    // Get the RPC address for the clients to connect
+    this.rpcAddress = getRpcAddress(conf, nameserviceId, namenodeId);
+    LOG.info("{}-{} RPC address: {}",
+        nameserviceId, namenodeId, rpcAddress);
+
+    // Get the Service RPC address for monitoring
+    this.serviceAddress =
+        DFSUtil.getNamenodeServiceAddr(conf, nameserviceId, namenodeId);
+    if (this.serviceAddress == null) {
+      LOG.error("Cannot locate RPC service address for NN {}-{}, " +
+          "using RPC address {}", nameserviceId, namenodeId, this.rpcAddress);
+      this.serviceAddress = this.rpcAddress;
+    }
+    LOG.info("{}-{} Service RPC address: {}",
+        nameserviceId, namenodeId, serviceAddress);
+
+    // Get the Lifeline RPC address for faster monitoring
+    this.lifelineAddress =
+        DFSUtil.getNamenodeLifelineAddr(conf, nameserviceId, namenodeId);
+    if (this.lifelineAddress == null) {
+      this.lifelineAddress = this.serviceAddress;
+    }
+    LOG.info("{}-{} Lifeline RPC address: {}",
+        nameserviceId, namenodeId, lifelineAddress);
+
+    // Get the Web address for UI
+    this.webAddress =
+        DFSUtil.getNamenodeWebAddr(conf, nameserviceId, namenodeId);
+    LOG.info("{}-{} Web address: {}", nameserviceId, namenodeId, webAddress);
+
+    this.setIntervalMs(conf.getLong(
+        DFS_ROUTER_HEARTBEAT_INTERVAL_MS,
+        DFS_ROUTER_HEARTBEAT_INTERVAL_MS_DEFAULT));
+
+
+    super.serviceInit(configuration);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    updateState();
+  }
+
+  /**
+   * Get the RPC address for a Namenode.
+   * @param conf Configuration.
+   * @param nsId Name service identifier.
+   * @param nnId Name node identifier.
+   * @return RPC address in format hostname:1234.
+   */
+  private static String getRpcAddress(
+      Configuration conf, String nsId, String nnId) {
+
+    // Get it from the regular RPC setting
+    String confKey = DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+    String ret = conf.get(confKey);
+
+    if (nsId != null && nnId != null) {
+      // Get if for the proper nameservice and namenode
+      confKey = DFSUtil.addKeySuffixes(confKey, nsId, nnId);
+      ret = conf.get(confKey);
+
+      // If not available, get it from the map
+      if (ret == null) {
+        Map<String, InetSocketAddress> rpcAddresses =
+            DFSUtil.getRpcAddressesForNameserviceId(conf, nsId, null);
+        if (rpcAddresses.containsKey(nnId)) {
+          InetSocketAddress sockAddr = rpcAddresses.get(nnId);
+          InetAddress addr = sockAddr.getAddress();
+          ret = addr.getHostAddress() + ":" + sockAddr.getPort();
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Update the state of the Namenode.
+   */
+  private void updateState() {
+    NamenodeStatusReport report = getNamenodeStatusReport();
+    if (!report.registrationValid()) {
+      // Not operational
+      LOG.error("Namenode is not operational: {}", getNamenodeDesc());
+    } else if (report.haStateValid()) {
+      // block and HA status available
+      LOG.debug("Received service state: {} from HA namenode: {}",
+          report.getState(), getNamenodeDesc());
+    } else if (localTarget == null) {
+      // block info available, HA status not expected
+      LOG.debug(
+          "Reporting non-HA namenode as operational: " + getNamenodeDesc());
+    } else {
+      // block info available, HA status should be available, but was not
+      // fetched do nothing and let the current state stand
+      return;
+    }
+    try {
+      if (!resolver.registerNamenode(report)) {
+        LOG.warn("Cannot register namenode {}", report);
+      }
+    } catch (IOException e) {
+      LOG.info("Cannot register namenode in the State Store");
+    } catch (Exception ex) {
+      LOG.error("Unhandled exception updating NN registration for {}",
+          getNamenodeDesc(), ex);
+    }
+  }
+
+  /**
+   * Get the status report for the Namenode monitored by this heartbeater.
+   * @return Namenode status report.
+   */
+  protected NamenodeStatusReport getNamenodeStatusReport() {
+    NamenodeStatusReport report = new NamenodeStatusReport(nameserviceId,
+        namenodeId, rpcAddress, serviceAddress, lifelineAddress, webAddress);
+
+    try {
+      LOG.debug("Probing NN at service address: {}", serviceAddress);
+
+      URI serviceURI = new URI("hdfs://" + serviceAddress);
+      // Read the filesystem info from RPC (required)
+      NamenodeProtocol nn = NameNodeProxies
+          .createProxy(this.conf, serviceURI, NamenodeProtocol.class)
+          .getProxy();
+
+      if (nn != null) {
+        NamespaceInfo info = nn.versionRequest();
+        if (info != null) {
+          report.setNamespaceInfo(info);
+        }
+      }
+      if (!report.registrationValid()) {
+        return report;
+      }
+
+      // Check for safemode from the client protocol. Currently optional, but
+      // should be required at some point for QoS
+      try {
+        ClientProtocol client = NameNodeProxies
+            .createProxy(this.conf, serviceURI, ClientProtocol.class)
+            .getProxy();
+        if (client != null) {
+          boolean isSafeMode = client.setSafeMode(
+              SafeModeAction.SAFEMODE_GET, false);
+          report.setSafeMode(isSafeMode);
+        }
+      } catch (Exception e) {
+        LOG.error("Cannot fetch safemode state for {}", getNamenodeDesc(), e);
+      }
+
+      // Read the stats from JMX (optional)
+      updateJMXParameters(webAddress, report);
+
+      if (localTarget != null) {
+        // Try to get the HA status
+        try {
+          // Determine if NN is active
+          // TODO: dynamic timeout
+          HAServiceProtocol haProtocol = localTarget.getProxy(conf, 30*1000);
+          HAServiceStatus status = haProtocol.getServiceStatus();
+          report.setHAServiceState(status.getState());
+        } catch (Throwable e) {
+          // Failed to fetch HA status, ignoring failure
+          LOG.error("Cannot fetch HA status for {}: {}",
+              getNamenodeDesc(), e.getMessage(), e);
+        }
+      }
+    } catch(IOException e) {
+      LOG.error("Cannot communicate with {}: {}",
+          getNamenodeDesc(), e.getMessage());
+    } catch(Throwable e) {
+      // Generic error that we don't know about
+      LOG.error("Unexpected exception while communicating with {}: {}",
+          getNamenodeDesc(), e.getMessage(), e);
+    }
+    return report;
+  }
+
+  /**
+   * Get the description of the Namenode to monitor.
+   * @return Description of the Namenode to monitor.
+   */
+  public String getNamenodeDesc() {
+    if (namenodeId != null && !namenodeId.isEmpty()) {
+      return nameserviceId + "-" + namenodeId + ":" + serviceAddress;
+    } else {
+      return nameserviceId + ":" + serviceAddress;
+    }
+  }
+
+  /**
+   * Get the parameters for a Namenode from JMX and add them to the report.
+   * @param webAddress Web interface of the Namenode to monitor.
+   * @param report Namenode status report to update with JMX data.
+   */
+  private void updateJMXParameters(
+      String address, NamenodeStatusReport report) {
+    try {
+      // TODO part of this should be moved to its own utility
+      String query = "Hadoop:service=NameNode,name=FSNamesystem*";
+      JSONArray aux = FederationUtil.getJmx(query, address);
+      if (aux != null) {
+        for (int i = 0; i < aux.length(); i++) {
+          JSONObject jsonObject = aux.getJSONObject(i);
+          String name = jsonObject.getString("name");
+          if (name.equals("Hadoop:service=NameNode,name=FSNamesystemState")) {
+            report.setDatanodeInfo(
+                jsonObject.getInt("NumLiveDataNodes"),
+                jsonObject.getInt("NumDeadDataNodes"),
+                jsonObject.getInt("NumDecommissioningDataNodes"),
+                jsonObject.getInt("NumDecomLiveDataNodes"),
+                jsonObject.getInt("NumDecomDeadDataNodes"));
+          } else if (name.equals(
+              "Hadoop:service=NameNode,name=FSNamesystem")) {
+            report.setNamesystemInfo(
+                jsonObject.getLong("CapacityRemaining"),
+                jsonObject.getLong("CapacityTotal"),
+                jsonObject.getLong("FilesTotal"),
+                jsonObject.getLong("BlocksTotal"),
+                jsonObject.getLong("MissingBlocks"),
+                jsonObject.getLong("PendingReplicationBlocks"),
+                jsonObject.getLong("UnderReplicatedBlocks"),
+                jsonObject.getLong("PendingDeletionBlocks"));
+          }
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Cannot get stat from {} using JMX", getNamenodeDesc(), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index 019a5cd..cfddf20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -25,12 +25,16 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
@@ -85,6 +89,8 @@ public class Router extends CompositeService {
 
   /** Interface to identify the active NN for a nameservice or blockpool ID. */
   private ActiveNamenodeResolver namenodeResolver;
+  /** Updates the namenode status in the namenode resolver. */
+  private Collection<NamenodeHeartbeatService> namenodeHearbeatServices;
 
 
   /** Usage string for help message. */
@@ -133,6 +139,22 @@ public class Router extends CompositeService {
       this.setRpcServerAddress(rpcServer.getRpcAddress());
     }
 
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT)) {
+
+      // Create status updater for each monitored Namenode
+      this.namenodeHearbeatServices = createNamenodeHearbeatServices();
+      for (NamenodeHeartbeatService hearbeatService :
+          this.namenodeHearbeatServices) {
+        addService(hearbeatService);
+      }
+
+      if (this.namenodeHearbeatServices.isEmpty()) {
+        LOG.error("Heartbeat is enabled but there are no namenodes to monitor");
+      }
+    }
+
     super.serviceInit(conf);
   }
 
@@ -243,6 +265,96 @@ public class Router extends CompositeService {
   }
 
   /////////////////////////////////////////////////////////
+  // Namenode heartbeat monitors
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create each of the services that will monitor a Namenode.
+   *
+   * @return List of heartbeat services.
+   */
+  protected Collection<NamenodeHeartbeatService>
+      createNamenodeHearbeatServices() {
+
+    Map<String, NamenodeHeartbeatService> ret = new HashMap<>();
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE,
+        DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE_DEFAULT)) {
+      // Create a local heartbet service
+      NamenodeHeartbeatService localHeartbeatService =
+          createLocalNamenodeHearbeatService();
+      if (localHeartbeatService != null) {
+        String nnDesc = localHeartbeatService.getNamenodeDesc();
+        ret.put(nnDesc, localHeartbeatService);
+      }
+    }
+
+    // Create heartbeat services for a list specified by the admin
+    String namenodes = this.conf.get(
+        DFSConfigKeys.DFS_ROUTER_MONITOR_NAMENODE);
+    if (namenodes != null) {
+      for (String namenode : namenodes.split(",")) {
+        String[] namenodeSplit = namenode.split("\\.");
+        String nsId = null;
+        String nnId = null;
+        if (namenodeSplit.length == 2) {
+          nsId = namenodeSplit[0];
+          nnId = namenodeSplit[1];
+        } else if (namenodeSplit.length == 1) {
+          nsId = namenode;
+        } else {
+          LOG.error("Wrong Namenode to monitor: {}", namenode);
+        }
+        if (nsId != null) {
+          NamenodeHeartbeatService heartbeatService =
+              createNamenodeHearbeatService(nsId, nnId);
+          if (heartbeatService != null) {
+            ret.put(heartbeatService.getNamenodeDesc(), heartbeatService);
+          }
+        }
+      }
+    }
+
+    return ret.values();
+  }
+
+  /**
+   * Create a new status updater for the local Namenode.
+   *
+   * @return Updater of the status for the local Namenode.
+   */
+  protected NamenodeHeartbeatService createLocalNamenodeHearbeatService() {
+    // Detect NN running in this machine
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    String nnId = null;
+    if (HAUtil.isHAEnabled(conf, nsId)) {
+      nnId = HAUtil.getNameNodeId(conf, nsId);
+      if (nnId == null) {
+        LOG.error("Cannot find namenode id for local {}", nsId);
+      }
+    }
+
+    return createNamenodeHearbeatService(nsId, nnId);
+  }
+
+  /**
+   * Create a heartbeat monitor for a particular Namenode.
+   *
+   * @param nsId Identifier of the nameservice to monitor.
+   * @param nnId Identifier of the namenode (HA) to monitor.
+   * @return Updater of the status for the specified Namenode.
+   */
+  protected NamenodeHeartbeatService createNamenodeHearbeatService(
+      String nsId, String nnId) {
+
+    LOG.info("Creating heartbeat service for Namenode {} in {}", nnId, nsId);
+    NamenodeHeartbeatService ret = new NamenodeHeartbeatService(
+        namenodeResolver, nsId, nnId);
+    return ret;
+  }
+
+  /////////////////////////////////////////////////////////
   // Submodule getters
   /////////////////////////////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/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 57a0cd8..6f00261 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
@@ -4770,4 +4770,36 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.heartbeat.enable</name>
+    <value>true</value>
+    <description>
+      Enables the Router to heartbeat into the State Store.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.heartbeat.interval</name>
+    <value>5000</value>
+    <description>
+      How often the Router should heartbeat into the State Store in milliseconds.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.monitor.namenode</name>
+    <value></value>
+    <description>
+      The identifier of the namenodes to monitor and heartbeat.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.monitor.localnamenode.enable</name>
+    <value>true</value>
+    <description>
+      If the Router should monitor the namenode in the local machine.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/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 0345cf5..da91006 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
@@ -582,6 +582,14 @@ public class MiniDFSCluster implements AutoCloseable {
     public void setStartOpt(StartupOption startOpt) {
       this.startOpt = startOpt;
     }
+
+    public String getNameserviceId() {
+      return this.nameserviceId;
+    }
+
+    public String getNamenodeId() {
+      return this.nnId;
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
index 2875750..87427fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -56,9 +56,16 @@ public class MockResolver
   private Set<FederationNamespaceInfo> namespaces = new HashSet<>();
   private String defaultNamespace = null;
 
+  public MockResolver() {
+    this.cleanRegistrations();
+  }
+
+  public MockResolver(Configuration conf) {
+    this();
+  }
 
   public MockResolver(Configuration conf, StateStoreService store) {
-    this.cleanRegistrations();
+    this();
   }
 
   public void addLocation(String mount, String nsId, String location) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
index 39fcf7a..21555c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
@@ -28,6 +28,8 @@ public class RouterConfigBuilder {
   private Configuration conf;
 
   private boolean enableRpcServer = false;
+  private boolean enableHeartbeat = false;
+  private boolean enableLocalHeartbeat = false;
 
   public RouterConfigBuilder(Configuration configuration) {
     this.conf = configuration;
@@ -39,6 +41,13 @@ public class RouterConfigBuilder {
 
   public RouterConfigBuilder all() {
     this.enableRpcServer = true;
+    this.enableHeartbeat = true;
+    this.enableLocalHeartbeat = true;
+    return this;
+  }
+
+  public RouterConfigBuilder enableLocalHeartbeat(boolean enable) {
+    this.enableLocalHeartbeat = enable;
     return this;
   }
 
@@ -47,12 +56,25 @@ public class RouterConfigBuilder {
     return this;
   }
 
+  public RouterConfigBuilder heartbeat(boolean enable) {
+    this.enableHeartbeat = enable;
+    return this;
+  }
+
   public RouterConfigBuilder rpc() {
     return this.rpc(true);
   }
 
+  public RouterConfigBuilder heartbeat() {
+    return this.heartbeat(true);
+  }
+
   public Configuration build() {
     conf.setBoolean(DFSConfigKeys.DFS_ROUTER_RPC_ENABLE, this.enableRpcServer);
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE,
+        this.enableHeartbeat);
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE,
+        this.enableLocalHeartbeat);
     return conf;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
index 4031b7f..0830c19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf;
@@ -754,6 +755,48 @@ public class RouterDFSCluster {
   }
 
   /**
+   * Switch a namenode in a nameservice to be the active.
+   * @param nsId Nameservice identifier.
+   * @param nnId Namenode identifier.
+   */
+  public void switchToActive(String nsId, String nnId) {
+    try {
+      int total = cluster.getNumNameNodes();
+      NameNodeInfo[] nns = cluster.getNameNodeInfos();
+      for (int i = 0; i < total; i++) {
+        NameNodeInfo nn = nns[i];
+        if (nn.getNameserviceId().equals(nsId) &&
+            nn.getNamenodeId().equals(nnId)) {
+          cluster.transitionToActive(i);
+        }
+      }
+    } catch (Throwable e) {
+      LOG.error("Cannot transition to active", e);
+    }
+  }
+
+  /**
+   * Switch a namenode in a nameservice to be in standby.
+   * @param nsId Nameservice identifier.
+   * @param nnId Namenode identifier.
+   */
+  public void switchToStandby(String nsId, String nnId) {
+    try {
+      int total = cluster.getNumNameNodes();
+      NameNodeInfo[] nns = cluster.getNameNodeInfos();
+      for (int i = 0; i < total; i++) {
+        NameNodeInfo nn = nns[i];
+        if (nn.getNameserviceId().equals(nsId) &&
+            nn.getNamenodeId().equals(nnId)) {
+          cluster.transitionToStandby(i);
+        }
+      }
+    } catch (Throwable e) {
+      LOG.error("Cannot transition to standby", e);
+    }
+  }
+
+  /**
    * Stop the federated HDFS cluster.
    */
   public void shutdown() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNamenodeHeartbeat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNamenodeHeartbeat.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNamenodeHeartbeat.java
new file mode 100644
index 0000000..877fb02
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNamenodeHeartbeat.java
@@ -0,0 +1,168 @@
+/**
+ * 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.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMESERVICES;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.MockResolver;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.service.Service.STATE;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test the service that heartbeats the state of the namenodes to the State
+ * Store.
+ */
+public class TestNamenodeHeartbeat {
+
+  private static RouterDFSCluster cluster;
+  private static ActiveNamenodeResolver namenodeResolver;
+  private static List<NamenodeHeartbeatService> services;
+
+  @Rule
+  public TestName name = new TestName();
+
+  @BeforeClass
+  public static void globalSetUp() throws Exception {
+
+    cluster = new RouterDFSCluster(true, 2);
+
+    // Start NNs and DNs and wait until ready
+    cluster.startCluster();
+
+    // Mock locator that records the heartbeats
+    List<String> nss = cluster.getNameservices();
+    String ns = nss.get(0);
+    Configuration conf = cluster.generateNamenodeConfiguration(ns);
+    namenodeResolver = new MockResolver(conf);
+    namenodeResolver.setRouterId("testrouter");
+
+    // Create one heartbeat service per NN
+    services = new ArrayList<>();
+    for (NamenodeContext nn : cluster.getNamenodes()) {
+      String nsId = nn.getNameserviceId();
+      String nnId = nn.getNamenodeId();
+      NamenodeHeartbeatService service = new NamenodeHeartbeatService(
+          namenodeResolver, nsId, nnId);
+      service.init(conf);
+      service.start();
+      services.add(service);
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    cluster.shutdown();
+    for (NamenodeHeartbeatService service: services) {
+      service.stop();
+      service.close();
+    }
+  }
+
+  @Test
+  public void testNamenodeHeartbeatService() throws IOException {
+
+    RouterDFSCluster testCluster = new RouterDFSCluster(true, 1);
+    Configuration heartbeatConfig = testCluster.generateNamenodeConfiguration(
+        NAMESERVICES[0]);
+    NamenodeHeartbeatService server = new NamenodeHeartbeatService(
+        namenodeResolver, NAMESERVICES[0], NAMENODES[0]);
+    server.init(heartbeatConfig);
+    assertEquals(STATE.INITED, server.getServiceState());
+    server.start();
+    assertEquals(STATE.STARTED, server.getServiceState());
+    server.stop();
+    assertEquals(STATE.STOPPED, server.getServiceState());
+    server.close();
+  }
+
+  @Test
+  public void testHearbeat() throws InterruptedException, IOException {
+
+    // Set NAMENODE1 to active for all nameservices
+    if (cluster.isHighAvailability()) {
+      for (String ns : cluster.getNameservices()) {
+        cluster.switchToActive(ns, NAMENODES[0]);
+        cluster.switchToStandby(ns, NAMENODES[1]);
+      }
+    }
+
+    // Wait for heartbeats to record
+    Thread.sleep(5000);
+
+    // Verify the locator has matching NN entries for each NS
+    for (String ns : cluster.getNameservices()) {
+      List<? extends FederationNamenodeContext> nns =
+          namenodeResolver.getNamenodesForNameserviceId(ns);
+
+      // Active
+      FederationNamenodeContext active = nns.get(0);
+      assertEquals(NAMENODES[0], active.getNamenodeId());
+
+      // Standby
+      FederationNamenodeContext standby = nns.get(1);
+      assertEquals(NAMENODES[1], standby.getNamenodeId());
+    }
+
+    // Switch active NNs in 1/2 nameservices
+    List<String> nss = cluster.getNameservices();
+    String failoverNS = nss.get(0);
+    String normalNs = nss.get(1);
+
+    cluster.switchToStandby(failoverNS, NAMENODES[0]);
+    cluster.switchToActive(failoverNS, NAMENODES[1]);
+
+    // Wait for heartbeats to record
+    Thread.sleep(5000);
+
+    // Verify the locator has recorded the failover for the failover NS
+    List<? extends FederationNamenodeContext> failoverNSs =
+        namenodeResolver.getNamenodesForNameserviceId(failoverNS);
+    // Active
+    FederationNamenodeContext active = failoverNSs.get(0);
+    assertEquals(NAMENODES[1], active.getNamenodeId());
+
+    // Standby
+    FederationNamenodeContext standby = failoverNSs.get(1);
+    assertEquals(NAMENODES[0], standby.getNamenodeId());
+
+    // Verify the locator has the same records for the other ns
+    List<? extends FederationNamenodeContext> normalNss =
+        namenodeResolver.getNamenodesForNameserviceId(normalNs);
+    // Active
+    active = normalNss.get(0);
+    assertEquals(NAMENODES[0], active.getNamenodeId());
+    // Standby
+    standby = normalNss.get(1);
+    assertEquals(NAMENODES[1], standby.getNamenodeId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/740ff174/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
index d8afb39..2074d3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
@@ -98,6 +98,9 @@ public class TestRouter {
     // Rpc only
     testRouterStartup(new RouterConfigBuilder(conf).rpc().build());
 
+    // Heartbeat only
+    testRouterStartup(new RouterConfigBuilder(conf).heartbeat().build());
+
     // Run with all services
     testRouterStartup(new RouterConfigBuilder(conf).all().build());
   }


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


[22/29] hadoop git commit: HDFS-12312. Rebasing HDFS-10467 (2). Contributed by Inigo Goiri.

Posted by in...@apache.org.
HDFS-12312. Rebasing HDFS-10467 (2). Contributed by Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 4007ebb472ab0501a3463516737e2a35515d4385
Parents: 20203eb
Author: Inigo Goiri <in...@apache.org>
Authored: Wed Aug 16 17:31:37 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs                   | 1 -
 .../hadoop/hdfs/server/federation/router/RouterRpcServer.java       | 1 +
 2 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4007ebb4/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index d51a8e2..d122ff7 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -31,7 +31,6 @@ function hadoop_usage
   hadoop_add_option "--hosts filename" "list of hosts to use in worker mode"
   hadoop_add_option "--workers" "turn on worker mode"
 
-<<<<<<< HEAD
   hadoop_add_subcommand "balancer" daemon "run a cluster balancing utility"
   hadoop_add_subcommand "cacheadmin" admin "configure the HDFS cache"
   hadoop_add_subcommand "classpath" client "prints the class path needed to get the hadoop jar and the required libraries"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4007ebb4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index eaaab39..c77d255 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -1946,6 +1946,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     }
     long inodeId = 0;
     return new HdfsFileStatus(0, true, 0, 0, modTime, accessTime, permission,
+        EnumSet.noneOf(HdfsFileStatus.Flags.class),
         owner, group, new byte[0], DFSUtil.string2Bytes(name), inodeId,
         childrenNum, null, (byte) 0, null);
   }


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


[27/29] hadoop git commit: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 24792bb..4bae71e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -17,16 +17,109 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_KEY;
+
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.QuotaUsage;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.AddBlockFlag;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.AddingECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
 
 /**
  * This class is responsible for handling all of the RPC calls to the It is
@@ -36,10 +129,42 @@ import com.google.common.annotations.VisibleForTesting;
  * the requests to the active
  * {@link org.apache.hadoop.hdfs.server.namenode.NameNode NameNode}.
  */
-public class RouterRpcServer extends AbstractService {
+public class RouterRpcServer extends AbstractService implements ClientProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterRpcServer.class);
+
+
+  /** Configuration for the RPC server. */
+  private Configuration conf;
+
+  /** Identifier for the super user. */
+  private final String superUser;
+  /** Identifier for the super group. */
+  private final String superGroup;
+
+  /** Router using this RPC server. */
+  private final Router router;
 
   /** The RPC server that listens to requests from clients. */
   private final Server rpcServer;
+  /** The address for this RPC server. */
+  private final InetSocketAddress rpcAddress;
+
+  /** RPC clients to connect to the Namenodes. */
+  private final RouterRpcClient rpcClient;
+
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+  /** Interface to map global name space to HDFS subcluster name spaces. */
+  private final FileSubclusterResolver subclusterResolver;
+
+
+  /** Category of the operation that a thread is executing. */
+  private final ThreadLocal<OperationCategory> opCategory = new ThreadLocal<>();
+
 
   /**
    * Construct a router RPC server.
@@ -54,31 +179,94 @@ public class RouterRpcServer extends AbstractService {
           throws IOException {
     super(RouterRpcServer.class.getName());
 
-    this.rpcServer = null;
-  }
+    this.conf = configuration;
+    this.router = router;
+    this.namenodeResolver = nnResolver;
+    this.subclusterResolver = fileResolver;
 
-  /**
-   * Allow access to the client RPC server for testing.
-   *
-   * @return The RPC server.
-   */
-  @VisibleForTesting
-  public Server getServer() {
-    return this.rpcServer;
+    // User and group for reporting
+    this.superUser = System.getProperty("user.name");
+    this.superGroup = this.conf.get(
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
+
+    // RPC server settings
+    int handlerCount = this.conf.getInt(DFS_ROUTER_HANDLER_COUNT_KEY,
+        DFS_ROUTER_HANDLER_COUNT_DEFAULT);
+
+    int readerCount = this.conf.getInt(DFS_ROUTER_READER_COUNT_KEY,
+        DFS_ROUTER_READER_COUNT_DEFAULT);
+
+    int handlerQueueSize = this.conf.getInt(DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY,
+        DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT);
+
+    // Override Hadoop Common IPC setting
+    int readerQueueSize = this.conf.getInt(DFS_ROUTER_READER_QUEUE_SIZE_KEY,
+        DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY,
+        readerQueueSize);
+
+    RPC.setProtocolEngine(this.conf, ClientNamenodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+
+    ClientNamenodeProtocolServerSideTranslatorPB
+        clientProtocolServerTranslator =
+            new ClientNamenodeProtocolServerSideTranslatorPB(this);
+    BlockingService clientNNPbService = ClientNamenodeProtocol
+        .newReflectiveBlockingService(clientProtocolServerTranslator);
+
+    InetSocketAddress confRpcAddress = conf.getSocketAddr(
+        DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY,
+        DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY,
+        DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_DEFAULT,
+        DFSConfigKeys.DFS_ROUTER_RPC_PORT_DEFAULT);
+    LOG.info("RPC server binding to {} with {} handlers for Router {}",
+        confRpcAddress, handlerCount, this.router.getRouterId());
+
+    this.rpcServer = new RPC.Builder(this.conf)
+        .setProtocol(ClientNamenodeProtocolPB.class)
+        .setInstance(clientNNPbService)
+        .setBindAddress(confRpcAddress.getHostName())
+        .setPort(confRpcAddress.getPort())
+        .setNumHandlers(handlerCount)
+        .setnumReaders(readerCount)
+        .setQueueSizePerHandler(handlerQueueSize)
+        .setVerbose(false)
+        .build();
+    // We don't want the server to log the full stack trace for some exceptions
+    this.rpcServer.addTerseExceptions(
+        RemoteException.class,
+        StandbyException.class,
+        SafeModeException.class,
+        FileNotFoundException.class,
+        FileAlreadyExistsException.class,
+        AccessControlException.class,
+        LeaseExpiredException.class,
+        NotReplicatedYetException.class,
+        IOException.class);
+
+    // The RPC-server port can be ephemeral... ensure we have the correct info
+    InetSocketAddress listenAddress = this.rpcServer.getListenerAddress();
+    this.rpcAddress = new InetSocketAddress(
+        confRpcAddress.getHostName(), listenAddress.getPort());
+
+    // Create the client
+    this.rpcClient = new RouterRpcClient(this.conf, this.router.getRouterId(),
+        this.namenodeResolver);
   }
 
   @Override
   protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
     super.serviceInit(configuration);
   }
 
-  /**
-   * Start client and service RPC servers.
-   */
   @Override
   protected void serviceStart() throws Exception {
     if (this.rpcServer != null) {
       this.rpcServer.start();
+      LOG.info("Router RPC up at: {}", this.getRpcAddress());
     }
     super.serviceStart();
   }
@@ -92,11 +280,1652 @@ public class RouterRpcServer extends AbstractService {
   }
 
   /**
-   * Wait until the RPC servers have shutdown.
+   * Get the RPC client to the Namenode.
+   *
+   * @return RPC clients to the Namenodes.
    */
-  void join() throws InterruptedException {
-    if (this.rpcServer != null) {
-      this.rpcServer.join();
+  public RouterRpcClient getRPCClient() {
+    return rpcClient;
+  }
+
+  /**
+   * Allow access to the client RPC server for testing.
+   *
+   * @return The RPC server.
+   */
+  @VisibleForTesting
+  public Server getServer() {
+    return rpcServer;
+  }
+
+  /**
+   * Get the RPC address of the service.
+   *
+   * @return RPC service address.
+   */
+  public InetSocketAddress getRpcAddress() {
+    return rpcAddress;
+  }
+
+  /**
+   * Check if the Router is in safe mode. We should only see READ, WRITE, and
+   * UNCHECKED. It includes a default handler when we haven't implemented an
+   * operation. If not supported, it always throws an exception reporting the
+   * operation.
+   *
+   * @param op Category of the operation to check.
+   * @param supported If the operation is supported or not. If not, it will
+   *                  throw an UnsupportedOperationException.
+   * @throws StandbyException If the Router is in safe mode and cannot serve
+   *                          client requests.
+   * @throws UnsupportedOperationException If the operation is not supported.
+   */
+  private void checkOperation(OperationCategory op, boolean supported)
+      throws StandbyException, UnsupportedOperationException {
+    checkOperation(op);
+
+    if (!supported) {
+      String methodName = getMethodName();
+      throw new UnsupportedOperationException(
+          "Operation \"" + methodName + "\" is not supported");
+    }
+  }
+
+  /**
+   * Check if the Router is in safe mode. We should only see READ, WRITE, and
+   * UNCHECKED. This function should be called by all ClientProtocol functions.
+   *
+   * @param op Category of the operation to check.
+   * @throws StandbyException If the Router is in safe mode and cannot serve
+   *                          client requests.
+   */
+  private void checkOperation(OperationCategory op) throws StandbyException {
+    // Log the function we are currently calling.
+    if (LOG.isDebugEnabled()) {
+      String methodName = getMethodName();
+      LOG.debug("Proxying operation: {}", methodName);
+    }
+
+    // Store the category of the operation category for this thread
+    opCategory.set(op);
+
+    // We allow unchecked and read operations
+    if (op == OperationCategory.UNCHECKED || op == OperationCategory.READ) {
+      return;
+    }
+
+    // TODO check Router safe mode and return Standby exception
+  }
+
+  @Override // ClientProtocol
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
+  }
+
+  /**
+   * The the delegation token from each name service.
+   * @param renewer
+   * @return Name service -> Token.
+   * @throws IOException
+   */
+  public Map<FederationNamespaceInfo, Token<DelegationTokenIdentifier>>
+      getDelegationTokens(Text renewer) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return 0;
+  }
+
+  @Override // ClientProtocol
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public LocatedBlocks getBlockLocations(String src, final long offset,
+      final long length) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations",
+        new Class<?>[] {String.class, long.class, long.class},
+        new RemoteParam(), offset, length);
+    return (LocatedBlocks) rpcClient.invokeSequential(locations, remoteMethod,
+        LocatedBlocks.class, null);
+  }
+
+  @Override // ClientProtocol
+  public FsServerDefaults getServerDefaults() throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getServerDefaults");
+    String ns = subclusterResolver.getDefaultNamespace();
+    return (FsServerDefaults) rpcClient.invokeSingle(ns, method);
+  }
+
+  @Override // ClientProtocol
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize,
+      CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteLocation createLocation = getCreateLocation(src);
+    RemoteMethod method = new RemoteMethod("create",
+        new Class<?>[] {String.class, FsPermission.class, String.class,
+                        EnumSetWritable.class, boolean.class, short.class,
+                        long.class, CryptoProtocolVersion[].class,
+                        String.class},
+        createLocation.getDest(), masked, clientName, flag, createParent,
+        replication, blockSize, supportedVersions, ecPolicyName);
+    return (HdfsFileStatus) rpcClient.invokeSingle(createLocation, method);
+  }
+
+  /**
+   * Get the location to create a file. It checks if the file already existed
+   * in one of the locations.
+   *
+   * @param src Path of the file to check.
+   * @return The remote location for this file.
+   * @throws IOException If the file has no creation location.
+   */
+  private RemoteLocation getCreateLocation(final String src)
+      throws IOException {
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    if (locations == null || locations.isEmpty()) {
+      throw new IOException("Cannot get locations to create " + src);
+    }
+
+    RemoteLocation createLocation = locations.get(0);
+    if (locations.size() > 1) {
+      try {
+        // Check if this file already exists in other subclusters
+        LocatedBlocks existingLocation = getBlockLocations(src, 0, 1);
+        if (existingLocation != null) {
+          // Forward to the existing location and let the NN handle the error
+          LocatedBlock existingLocationLastLocatedBlock =
+              existingLocation.getLastLocatedBlock();
+          if (existingLocationLastLocatedBlock == null) {
+            // The block has no blocks yet, check for the meta data
+            for (RemoteLocation location : locations) {
+              RemoteMethod method = new RemoteMethod("getFileInfo",
+                  new Class<?>[] {String.class}, new RemoteParam());
+              if (rpcClient.invokeSingle(location, method) != null) {
+                createLocation = location;
+                break;
+              }
+            }
+          } else {
+            ExtendedBlock existingLocationLastBlock =
+                existingLocationLastLocatedBlock.getBlock();
+            String blockPoolId = existingLocationLastBlock.getBlockPoolId();
+            createLocation = getLocationForPath(src, true, blockPoolId);
+          }
+        }
+      } catch (FileNotFoundException fne) {
+        // Ignore if the file is not found
+      }
+    }
+    return createLocation;
+  }
+
+  // Medium
+  @Override // ClientProtocol
+  public LastBlockWithStatus append(String src, final String clientName,
+      final EnumSetWritable<CreateFlag> flag) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("append",
+        new Class<?>[] {String.class, String.class, EnumSetWritable.class},
+        new RemoteParam(), clientName, flag);
+    return (LastBlockWithStatus) rpcClient.invokeSequential(
+        locations, method, LastBlockWithStatus.class, null);
+  }
+
+  // Low
+  @Override // ClientProtocol
+  public boolean recoverLease(String src, String clientName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("recoverLease",
+        new Class<?>[] {String.class, String.class}, new RemoteParam(),
+        clientName);
+    Object result = rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE);
+    return (boolean) result;
+  }
+
+  @Override // ClientProtocol
+  public boolean setReplication(String src, short replication)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setReplication",
+        new Class<?>[] {String.class, short.class}, new RemoteParam(),
+        replication);
+    Object result = rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE);
+    return (boolean) result;
+  }
+
+  @Override
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setStoragePolicy",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), policyName);
+    rpcClient.invokeSequential(locations, method, null, null);
+  }
+
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getStoragePolicies");
+    String ns = subclusterResolver.getDefaultNamespace();
+    return (BlockStoragePolicy[]) rpcClient.invokeSingle(ns, method);
+  }
+
+  @Override // ClientProtocol
+  public void setPermission(String src, FsPermission permissions)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setPermission",
+        new Class<?>[] {String.class, FsPermission.class},
+        new RemoteParam(), permissions);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public void setOwner(String src, String username, String groupname)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setOwner",
+        new Class<?>[] {String.class, String.class, String.class},
+        new RemoteParam(), username, groupname);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  /**
+   * Excluded and favored nodes are not verified and will be ignored by
+   * placement policy if they are not in the same nameservice as the file.
+   */
+  @Override // ClientProtocol
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
+      String[] favoredNodes, EnumSet<AddBlockFlag> addBlockFlags)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("addBlock",
+        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+                        DatanodeInfo[].class, long.class, String[].class,
+                        EnumSet.class},
+        new RemoteParam(), clientName, previous, excludedNodes, fileId,
+        favoredNodes, addBlockFlags);
+    // TODO verify the excludedNodes and favoredNodes are acceptable to this NN
+    return (LocatedBlock) rpcClient.invokeSequential(
+        locations, method, LocatedBlock.class, null);
+  }
+
+  /**
+   * Excluded nodes are not verified and will be ignored by placement if they
+   * are not in the same nameservice as the file.
+   */
+  @Override // ClientProtocol
+  public LocatedBlock getAdditionalDatanode(final String src, final long fileId,
+      final ExtendedBlock blk, final DatanodeInfo[] existings,
+      final String[] existingStorageIDs, final DatanodeInfo[] excludes,
+      final int numAdditionalNodes, final String clientName)
+          throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getAdditionalDatanode",
+        new Class<?>[] {String.class, long.class, ExtendedBlock.class,
+                        DatanodeInfo[].class, String[].class,
+                        DatanodeInfo[].class, int.class, String.class},
+        new RemoteParam(), fileId, blk, existings, existingStorageIDs, excludes,
+        numAdditionalNodes, clientName);
+    return (LocatedBlock) rpcClient.invokeSequential(
+        locations, method, LocatedBlock.class, null);
+  }
+
+  @Override // ClientProtocol
+  public void abandonBlock(ExtendedBlock b, long fileId, String src,
+      String holder) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("abandonBlock",
+        new Class<?>[] {ExtendedBlock.class, long.class, String.class,
+                        String.class},
+        b, fileId, new RemoteParam(), holder);
+    rpcClient.invokeSingle(b, method);
+  }
+
+  @Override // ClientProtocol
+  public boolean complete(String src, String clientName, ExtendedBlock last,
+      long fileId) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("complete",
+        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+                        long.class},
+        new RemoteParam(), clientName, last, fileId);
+    // Complete can return true/false, so don't expect a result
+    return ((Boolean) rpcClient.invokeSequential(
+        locations, method, Boolean.class, null)).booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public LocatedBlock updateBlockForPipeline(
+      ExtendedBlock block, String clientName) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("updateBlockForPipeline",
+        new Class<?>[] {ExtendedBlock.class, String.class},
+        block, clientName);
+    return (LocatedBlock) rpcClient.invokeSingle(block, method);
+  }
+
+  /**
+   * Datanode are not verified to be in the same nameservice as the old block.
+   * TODO This may require validation.
+   */
+  @Override // ClientProtocol
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
+          throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("updatePipeline",
+        new Class<?>[] {String.class, ExtendedBlock.class, ExtendedBlock.class,
+                        DatanodeID[].class, String[].class},
+        clientName, oldBlock, newBlock, newNodes, newStorageIDs);
+    rpcClient.invokeSingle(oldBlock, method);
+  }
+
+  @Override // ClientProtocol
+  public long getPreferredBlockSize(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("getPreferredBlockSize",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return ((Long) rpcClient.invokeSequential(
+        locations, method, Long.class, null)).longValue();
+  }
+
+  /**
+   * Determines combinations of eligible src/dst locations for a rename. A
+   * rename cannot change the namespace. Renames are only allowed if there is an
+   * eligible dst location in the same namespace as the source.
+   *
+   * @param srcLocations List of all potential source destinations where the
+   *          path may be located. On return this list is trimmed to include
+   *          only the paths that have corresponding destinations in the same
+   *          namespace.
+   * @param dst The destination path
+   * @return A map of all eligible source namespaces and their corresponding
+   *         replacement value.
+   * @throws IOException If the dst paths could not be determined.
+   */
+  private RemoteParam getRenameDestinations(
+      final List<RemoteLocation> srcLocations, final String dst)
+          throws IOException {
+
+    final List<RemoteLocation> dstLocations = getLocationsForPath(dst, true);
+    final Map<RemoteLocation, String> dstMap = new HashMap<>();
+
+    Iterator<RemoteLocation> iterator = srcLocations.iterator();
+    while (iterator.hasNext()) {
+      RemoteLocation srcLocation = iterator.next();
+      RemoteLocation eligibleDst =
+          getFirstMatchingLocation(srcLocation, dstLocations);
+      if (eligibleDst != null) {
+        // Use this dst for this source location
+        dstMap.put(srcLocation, eligibleDst.getDest());
+      } else {
+        // This src destination is not valid, remove from the source list
+        iterator.remove();
+      }
+    }
+    return new RemoteParam(dstMap);
+  }
+
+  /**
+   * Get first matching location.
+   *
+   * @param location Location we are looking for.
+   * @param locations List of locations.
+   * @return The first matchin location in the list.
+   */
+  private RemoteLocation getFirstMatchingLocation(RemoteLocation location,
+      List<RemoteLocation> locations) {
+    for (RemoteLocation loc : locations) {
+      if (loc.getNameserviceId().equals(location.getNameserviceId())) {
+        // Return first matching location
+        return loc;
+      }
+    }
+    return null;
+  }
+
+  @Deprecated
+  @Override // ClientProtocol
+  public boolean rename(final String src, final String dst)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations = getLocationsForPath(src, true);
+    // srcLocations may be trimmed by getRenameDestinations()
+    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+    RemoteParam dstParam = getRenameDestinations(locs, dst);
+    if (locs.isEmpty()) {
+      throw new IOException(
+          "Rename of " + src + " to " + dst + " is not allowed," +
+          " no eligible destination in the same namespace was found.");
+    }
+    RemoteMethod method = new RemoteMethod("rename",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), dstParam);
+    return ((Boolean) rpcClient.invokeSequential(
+        locs, method, Boolean.class, Boolean.TRUE)).booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public void rename2(final String src, final String dst,
+      final Options.Rename... options) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations = getLocationsForPath(src, true);
+    // srcLocations may be trimmed by getRenameDestinations()
+    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+    RemoteParam dstParam = getRenameDestinations(locs, dst);
+    if (locs.isEmpty()) {
+      throw new IOException(
+          "Rename of " + src + " to " + dst + " is not allowed," +
+          " no eligible destination in the same namespace was found.");
+    }
+    RemoteMethod method = new RemoteMethod("rename2",
+        new Class<?>[] {String.class, String.class, options.getClass()},
+        new RemoteParam(), dstParam, options);
+    rpcClient.invokeSequential(locs, method, null, null);
+  }
+
+  @Override // ClientProtocol
+  public void concat(String trg, String[] src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // See if the src and target files are all in the same namespace
+    LocatedBlocks targetBlocks = getBlockLocations(trg, 0, 1);
+    if (targetBlocks == null) {
+      throw new IOException("Cannot locate blocks for target file - " + trg);
+    }
+    LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock();
+    String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId();
+    for (String source : src) {
+      LocatedBlocks sourceBlocks = getBlockLocations(source, 0, 1);
+      if (sourceBlocks == null) {
+        throw new IOException(
+            "Cannot located blocks for source file " + source);
+      }
+      String sourceBlockPoolId =
+          sourceBlocks.getLastLocatedBlock().getBlock().getBlockPoolId();
+      if (!sourceBlockPoolId.equals(targetBlockPoolId)) {
+        throw new IOException("Cannot concatenate source file " + source
+            + " because it is located in a different namespace"
+            + " with block pool id " + sourceBlockPoolId
+            + " from the target file with block pool id "
+            + targetBlockPoolId);
+      }
+    }
+
+    // Find locations in the matching namespace.
+    final RemoteLocation targetDestination =
+        getLocationForPath(trg, true, targetBlockPoolId);
+    String[] sourceDestinations = new String[src.length];
+    for (int i = 0; i < src.length; i++) {
+      String sourceFile = src[i];
+      RemoteLocation location =
+          getLocationForPath(sourceFile, true, targetBlockPoolId);
+      sourceDestinations[i] = location.getDest();
+    }
+    // Invoke
+    RemoteMethod method = new RemoteMethod("concat",
+        new Class<?>[] {String.class, String[].class},
+        targetDestination.getDest(), sourceDestinations);
+    rpcClient.invokeSingle(targetDestination, method);
+  }
+
+  @Override // ClientProtocol
+  public boolean truncate(String src, long newLength, String clientName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("truncate",
+        new Class<?>[] {String.class, long.class, String.class},
+        new RemoteParam(), newLength, clientName);
+    return ((Boolean) rpcClient.invokeSequential(locations, method,
+        Boolean.class, Boolean.TRUE)).booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public boolean delete(String src, boolean recursive) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("delete",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        recursive);
+    return ((Boolean) rpcClient.invokeSequential(locations, method,
+        Boolean.class, Boolean.TRUE)).booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    if (locations.size() > 1) {
+      // Check if this directory already exists
+      try {
+        HdfsFileStatus fileStatus = getFileInfo(src);
+        if (fileStatus != null) {
+          // When existing, the NN doesn't return an exception; return true
+          return true;
+        }
+      } catch (IOException ioe) {
+        // Can't query if this file exists or not.
+        LOG.error("Error requesting file info for path {} while proxing mkdirs",
+            src, ioe);
+      }
+    }
+
+    RemoteLocation firstLocation = locations.get(0);
+    RemoteMethod method = new RemoteMethod("mkdirs",
+        new Class<?>[] {String.class, FsPermission.class, boolean.class},
+        new RemoteParam(), masked, createParent);
+    return ((Boolean) rpcClient.invokeSingle(firstLocation, method))
+        .booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public void renewLease(String clientName) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("renewLease",
+        new Class<?>[] {String.class}, clientName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, false, false);
+  }
+
+  @Override // ClientProtocol
+  public DirectoryListing getListing(String src, byte[] startAfter,
+      boolean needLocation) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // Locate the dir and fetch the listing
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("getListing",
+        new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
+        new RemoteParam(), startAfter, needLocation);
+    Map<RemoteLocation, Object> listings =
+        rpcClient.invokeConcurrent(locations, method, false, false);
+
+    Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
+    int totalRemainingEntries = 0;
+    int remainingEntries = 0;
+    boolean namenodeListingExists = false;
+    if (listings != null) {
+      // Check the subcluster listing with the smallest name
+      String lastName = null;
+      for (Entry<RemoteLocation, Object> entry : listings.entrySet()) {
+        RemoteLocation location = entry.getKey();
+        DirectoryListing listing = (DirectoryListing) entry.getValue();
+        if (listing == null) {
+          LOG.debug("Cannot get listing from {}", location);
+        } else {
+          totalRemainingEntries += listing.getRemainingEntries();
+          HdfsFileStatus[] partialListing = listing.getPartialListing();
+          int length = partialListing.length;
+          if (length > 0) {
+            HdfsFileStatus lastLocalEntry = partialListing[length-1];
+            String lastLocalName = lastLocalEntry.getLocalName();
+            if (lastName == null || lastName.compareTo(lastLocalName) > 0) {
+              lastName = lastLocalName;
+            }
+          }
+        }
+      }
+
+      // Add existing entries
+      for (Object value : listings.values()) {
+        DirectoryListing listing = (DirectoryListing) value;
+        if (listing != null) {
+          namenodeListingExists = true;
+          for (HdfsFileStatus file : listing.getPartialListing()) {
+            String filename = file.getLocalName();
+            if (totalRemainingEntries > 0 && filename.compareTo(lastName) > 0) {
+              // Discarding entries further than the lastName
+              remainingEntries++;
+            } else {
+              nnListing.put(filename, file);
+            }
+          }
+          remainingEntries += listing.getRemainingEntries();
+        }
+      }
+    }
+
+    // Add mount points at this level in the tree
+    final List<String> children = subclusterResolver.getMountPoints(src);
+    if (children != null) {
+      // Get the dates for each mount point
+      Map<String, Long> dates = getMountPointDates(src);
+
+      // Create virtual folder with the mount name
+      for (String child : children) {
+        long date = 0;
+        if (dates != null && dates.containsKey(child)) {
+          date = dates.get(child);
+        }
+        // TODO add number of children
+        HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
+
+        // This may overwrite existing listing entries with the mount point
+        // TODO don't add if already there?
+        nnListing.put(child, dirStatus);
+      }
+    }
+
+    if (!namenodeListingExists && nnListing.size() == 0) {
+      // NN returns a null object if the directory cannot be found and has no
+      // listing. If we didn't retrieve any NN listing data, and there are no
+      // mount points here, return null.
+      return null;
+    }
+
+    // Generate combined listing
+    HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()];
+    combinedData = nnListing.values().toArray(combinedData);
+    return new DirectoryListing(combinedData, remainingEntries);
+  }
+
+  @Override // ClientProtocol
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+    HdfsFileStatus ret = (HdfsFileStatus) rpcClient.invokeSequential(
+        locations, method, HdfsFileStatus.class, null);
+
+    // If there is no real path, check mount points
+    if (ret == null) {
+      List<String> children = subclusterResolver.getMountPoints(src);
+      if (children != null && !children.isEmpty()) {
+        Map<String, Long> dates = getMountPointDates(src);
+        long date = 0;
+        if (dates != null && dates.containsKey(src)) {
+          date = dates.get(src);
+        }
+        ret = getMountPointStatus(src, children.size(), date);
+      }
+    }
+
+    return ret;
+  }
+
+  @Override // ClientProtocol
+  public boolean isFileClosed(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("isFileClosed",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return ((Boolean) rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE)).booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileLinkInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (HdfsFileStatus) rpcClient.invokeSequential(
+        locations, method, HdfsFileStatus.class, null);
+  }
+
+  @Override // ClientProtocol
+  public long[] getStats() throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("getStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+    long[] combinedData = new long[STATS_ARRAY_LENGTH];
+    for (Object o : results.values()) {
+      long[] data = (long[]) o;
+      for (int i = 0; i < combinedData.length && i < data.length; i++) {
+        if (data[i] >= 0) {
+          combinedData[i] += data[i];
+        }
+      }
+    }
+    return combinedData;
+  }
+
+  @Override // ClientProtocol
+  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+      throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    Map<String, DatanodeInfo> datanodesMap = new LinkedHashMap<>();
+    RemoteMethod method = new RemoteMethod("getDatanodeReport",
+        new Class<?>[] {DatanodeReportType.class}, type);
+
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+    for (Entry<FederationNamespaceInfo, Object> entry : results.entrySet()) {
+      FederationNamespaceInfo ns = entry.getKey();
+      DatanodeInfo[] result = (DatanodeInfo[]) entry.getValue();
+      for (DatanodeInfo node : result) {
+        String nodeId = node.getXferAddr();
+        if (!datanodesMap.containsKey(nodeId)) {
+          // Add the subcluster as a suffix to the network location
+          node.setNetworkLocation(
+              NodeBase.PATH_SEPARATOR_STR + ns.getNameserviceId() +
+              node.getNetworkLocation());
+          datanodesMap.put(nodeId, node);
+        } else {
+          LOG.debug("{} is in multiple subclusters", nodeId);
+        }
+      }
+    }
+    // Map -> Array
+    Collection<DatanodeInfo> datanodes = datanodesMap.values();
+    DatanodeInfo[] combinedData = new DatanodeInfo[datanodes.size()];
+    combinedData = datanodes.toArray(combinedData);
+    return combinedData;
+  }
+
+  @Override // ClientProtocol
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      DatanodeReportType type) throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    Map<String, DatanodeStorageReport> datanodesMap = new HashMap<>();
+    RemoteMethod method = new RemoteMethod("getDatanodeStorageReport",
+        new Class<?>[] {DatanodeReportType.class}, type);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+    for (Object r : results.values()) {
+      DatanodeStorageReport[] result = (DatanodeStorageReport[]) r;
+      for (DatanodeStorageReport node : result) {
+        String nodeId = node.getDatanodeInfo().getXferAddr();
+        if (!datanodesMap.containsKey(nodeId)) {
+          datanodesMap.put(nodeId, node);
+        }
+        // TODO merge somehow, right now it just takes the first one
+      }
+    }
+
+    Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
+    // TODO sort somehow
+    DatanodeStorageReport[] combinedData =
+        new DatanodeStorageReport[datanodes.size()];
+    combinedData = datanodes.toArray(combinedData);
+    return combinedData;
+  }
+
+  @Override // ClientProtocol
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // Set safe mode in all the name spaces
+    RemoteMethod method = new RemoteMethod("setSafeMode",
+        new Class<?>[] {SafeModeAction.class, boolean.class},
+        action, isChecked);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, true);
+
+    // We only report true if all the name space are in safe mode
+    int numSafemode = 0;
+    for (Object result : results.values()) {
+      if (result instanceof Boolean) {
+        boolean safemode = (boolean) result;
+        if (safemode) {
+          numSafemode++;
+        }
+      }
     }
+    return numSafemode == results.size();
+  }
+
+  @Override // ClientProtocol
+  public boolean restoreFailedStorage(String arg) throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("restoreFailedStorage",
+        new Class<?>[] {String.class}, arg);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    boolean success = true;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Boolean> results =
+        (Map<FederationNamespaceInfo, Boolean>)obj;
+    Collection<Boolean> sucesses = results.values();
+    for (boolean s : sucesses) {
+      if (!s) {
+        success = false;
+      }
+    }
+    return success;
+  }
+
+  @Override // ClientProtocol
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("saveNamespace",
+        new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    boolean success = true;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Boolean> results =
+        (Map<FederationNamespaceInfo, Boolean>)obj;
+    Collection<Boolean> sucesses = results.values();
+    for (boolean s : sucesses) {
+      if (!s) {
+        success = false;
+      }
+    }
+    return success;
+  }
+
+  @Override // ClientProtocol
+  public long rollEdits() throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    // Return the maximum txid
+    long txid = 0;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Long> results =
+        (Map<FederationNamespaceInfo, Long>)obj;
+    Collection<Long> txids = results.values();
+    for (long t : txids) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
+  }
+
+  @Override // ClientProtocol
+  public void refreshNodes() throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("refreshNodes", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, true);
+  }
+
+  @Override // ClientProtocol
+  public void finalizeUpgrade() throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("finalizeUpgrade",
+        new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  @Override // ClientProtocol
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+      throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("rollingUpgrade",
+        new Class<?>[] {RollingUpgradeAction.class}, action);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    // Return the first rolling upgrade info
+    RollingUpgradeInfo info = null;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> results =
+        (Map<FederationNamespaceInfo, RollingUpgradeInfo>)obj;
+    Collection<RollingUpgradeInfo> infos = results.values();
+    for (RollingUpgradeInfo infoNs : infos) {
+      if (info == null && infoNs != null) {
+        info = infoNs;
+      }
+    }
+    return info;
+  }
+
+  @Override // ClientProtocol
+  public void metaSave(String filename) throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("metaSave",
+        new Class<?>[] {String.class}, filename);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  @Override // ClientProtocol
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(path, false);
+    RemoteMethod method = new RemoteMethod("listCorruptFileBlocks",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), cookie);
+    return (CorruptFileBlocks) rpcClient.invokeSequential(
+        locations, method, CorruptFileBlocks.class, null);
+  }
+
+  @Override // ClientProtocol
+  public void setBalancerBandwidth(long bandwidth) throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("setBalancerBandwidth",
+        new Class<?>[] {Long.class}, bandwidth);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  @Override // ClientProtocol
+  public ContentSummary getContentSummary(String path) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // Get the summaries from regular files
+    Collection<ContentSummary> summaries = new LinkedList<>();
+    FileNotFoundException notFoundException = null;
+    try {
+      final List<RemoteLocation> locations = getLocationsForPath(path, false);
+      RemoteMethod method = new RemoteMethod("getContentSummary",
+          new Class<?>[] {String.class}, new RemoteParam());
+      @SuppressWarnings("unchecked")
+      Map<String, ContentSummary> results =
+          (Map<String, ContentSummary>) ((Object)rpcClient.invokeConcurrent(
+              locations, method, false, false));
+      summaries.addAll(results.values());
+    } catch (FileNotFoundException e) {
+      notFoundException = e;
+    }
+
+    // Add mount points at this level in the tree
+    final List<String> children = subclusterResolver.getMountPoints(path);
+    if (children != null) {
+      for (String child : children) {
+        Path childPath = new Path(path, child);
+        try {
+          ContentSummary mountSummary = getContentSummary(childPath.toString());
+          if (mountSummary != null) {
+            summaries.add(mountSummary);
+          }
+        } catch (Exception e) {
+          LOG.error("Cannot get content summary for mount {}: {}",
+              childPath, e.getMessage());
+        }
+      }
+    }
+
+    // Throw original exception if no original nor mount points
+    if (summaries.isEmpty() && notFoundException != null) {
+      throw notFoundException;
+    }
+
+    return aggregateContentSummary(summaries);
+  }
+
+  /**
+   * Aggregate content summaries for each subcluster.
+   *
+   * @param results Collection of individual summaries.
+   * @return Aggregated content summary.
+   */
+  private ContentSummary aggregateContentSummary(
+      Collection<ContentSummary> summaries) {
+    if (summaries.size() == 1) {
+      return summaries.iterator().next();
+    }
+
+    long length = 0;
+    long fileCount = 0;
+    long directoryCount = 0;
+    long quota = 0;
+    long spaceConsumed = 0;
+    long spaceQuota = 0;
+
+    for (ContentSummary summary : summaries) {
+      length += summary.getLength();
+      fileCount += summary.getFileCount();
+      directoryCount += summary.getDirectoryCount();
+      quota += summary.getQuota();
+      spaceConsumed += summary.getSpaceConsumed();
+      spaceQuota += summary.getSpaceQuota();
+    }
+
+    ContentSummary ret = new ContentSummary.Builder()
+        .length(length)
+        .fileCount(fileCount)
+        .directoryCount(directoryCount)
+        .quota(quota)
+        .spaceConsumed(spaceConsumed)
+        .spaceQuota(spaceQuota)
+        .build();
+    return ret;
+  }
+
+  @Override // ClientProtocol
+  public void fsync(String src, long fileId, String clientName,
+      long lastBlockLength) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("fsync",
+        new Class<?>[] {String.class, long.class, String.class, long.class },
+        new RemoteParam(), fileId, clientName, lastBlockLength);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public void setTimes(String src, long mtime, long atime) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setTimes",
+        new Class<?>[] {String.class, long.class, long.class},
+        new RemoteParam(), mtime, atime);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public void createSymlink(String target, String link, FsPermission dirPerms,
+      boolean createParent) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO Verify that the link location is in the same NS as the targets
+    final List<RemoteLocation> targetLocations =
+        getLocationsForPath(target, true);
+    final List<RemoteLocation> linkLocations =
+        getLocationsForPath(link, true);
+    RemoteLocation linkLocation = linkLocations.get(0);
+    RemoteMethod method = new RemoteMethod("createSymlink",
+        new Class<?>[] {String.class, String.class, FsPermission.class,
+                        boolean.class},
+        new RemoteParam(), linkLocation.getDest(), dirPerms, createParent);
+    rpcClient.invokeSequential(targetLocations, method);
+  }
+
+  @Override // ClientProtocol
+  public String getLinkTarget(String path) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("getLinkTarget",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (String) rpcClient.invokeSequential(
+        locations, method, String.class, null);
+  }
+
+  @Override // Client Protocol
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // Client Protocol
+  public void disallowSnapshot(String snapshot) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // Client Protocol
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String earlierSnapshotName, String laterSnapshotName) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public long addCacheDirective(CacheDirectiveInfo path,
+      EnumSet<CacheFlag> flags) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return 0;
+  }
+
+  @Override // ClientProtocol
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void removeCacheDirective(long id) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void removeCachePool(String cachePoolName) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("modifyAclEntries",
+        new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method, null, null);
+  }
+
+  @Override // ClienProtocol
+  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeAclEntries",
+        new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method, null, null);
+  }
+
+  @Override // ClientProtocol
+  public void removeDefaultAcl(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeDefaultAcl",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public void removeAcl(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeAcl",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod(
+        "setAcl", new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public AclStatus getAclStatus(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getAclStatus",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (AclStatus) rpcClient.invokeSequential(
+        locations, method, AclStatus.class, null);
+  }
+
+  @Override // ClientProtocol
+  public void createEncryptionZone(String src, String keyName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("createEncryptionZone",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), keyName);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public EncryptionZone getEZForPath(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getEZForPath",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (EncryptionZone) rpcClient.invokeSequential(
+        locations, method, EncryptionZone.class, null);
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setXAttr",
+        new Class<?>[] {String.class, XAttr.class, EnumSet.class},
+        new RemoteParam(), xAttr, flag);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override // ClientProtocol
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+      throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getXAttrs",
+        new Class<?>[] {String.class, List.class}, new RemoteParam(), xAttrs);
+    return (List<XAttr>) rpcClient.invokeSequential(
+        locations, method, List.class, null);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override // ClientProtocol
+  public List<XAttr> listXAttrs(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("listXAttrs",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (List<XAttr>) rpcClient.invokeSequential(
+        locations, method, List.class, null);
+  }
+
+  @Override // ClientProtocol
+  public void removeXAttr(String src, XAttr xAttr) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeXAttr",
+        new Class<?>[] {String.class, XAttr.class}, new RemoteParam(), xAttr);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public void checkAccess(String path, FsAction mode) throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("checkAccess",
+        new Class<?>[] {String.class, FsAction.class},
+        new RemoteParam(), mode);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override // ClientProtocol
+  public long getCurrentEditLogTxid() throws IOException {
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod(
+        "getCurrentEditLogTxid", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    // Return the maximum txid
+    long txid = 0;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Long> results =
+        (Map<FederationNamespaceInfo, Long>)obj;
+    Collection<Long> txids = results.values();
+    for (long t : txids) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
+  }
+
+  @Override // ClientProtocol
+  public EventBatchList getEditsFromTxid(long txid) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public DataEncryptionKey getDataEncryptionKey() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    return null;
+  }
+
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public AddingECPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO assign global replicas instead of applying them to each folder
+    final List<RemoteLocation> locations = getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("setQuota",
+        new Class<?>[] {String.class, Long.class, Long.class,
+            StorageType.class},
+        new RemoteParam(), namespaceQuota, storagespaceQuota, type);
+    rpcClient.invokeConcurrent(locations, method, false, false);
+  }
+
+  @Override // ClientProtocol
+  public QuotaUsage getQuotaUsage(String path) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // Block pool id -> blocks
+    Map<String, List<LocatedBlock>> blockLocations = new HashMap<>();
+    for (LocatedBlock block : blocks) {
+      String bpId = block.getBlock().getBlockPoolId();
+      List<LocatedBlock> bpBlocks = blockLocations.get(bpId);
+      if (bpBlocks == null) {
+        bpBlocks = new LinkedList<>();
+        blockLocations.put(bpId, bpBlocks);
+      }
+      bpBlocks.add(block);
+    }
+
+    // Invoke each block pool
+    for (Entry<String, List<LocatedBlock>> entry : blockLocations.entrySet()) {
+      String bpId = entry.getKey();
+      List<LocatedBlock> bpBlocks = entry.getValue();
+
+      LocatedBlock[] bpBlocksArray =
+          bpBlocks.toArray(new LocatedBlock[bpBlocks.size()]);
+      RemoteMethod method = new RemoteMethod("reportBadBlocks",
+          new Class<?>[] {LocatedBlock[].class},
+          new Object[] {bpBlocksArray});
+      rpcClient.invokeSingleBlockPool(bpId, method);
+    }
+  }
+
+  @Override
+  public void unsetStoragePolicy(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  /**
+   * Locate the location with the matching block pool id.
+   *
+   * @param path Path to check.
+   * @param failIfLocked Fail the request if locked (top mount point).
+   * @param blockPoolId The block pool ID of the namespace to search for.
+   * @return Prioritized list of locations in the federated cluster.
+   * @throws IOException if the location for this path cannot be determined.
+   */
+  private RemoteLocation getLocationForPath(
+      String path, boolean failIfLocked, String blockPoolId)
+          throws IOException {
+
+    final List<RemoteLocation> locations =
+        getLocationsForPath(path, failIfLocked);
+
+    String nameserviceId = null;
+    Set<FederationNamespaceInfo> namespaces =
+        this.namenodeResolver.getNamespaces();
+    for (FederationNamespaceInfo namespace : namespaces) {
+      if (namespace.getBlockPoolId().equals(blockPoolId)) {
+        nameserviceId = namespace.getNameserviceId();
+        break;
+      }
+    }
+    if (nameserviceId != null) {
+      for (RemoteLocation location : locations) {
+        if (location.getNameserviceId().equals(nameserviceId)) {
+          return location;
+        }
+      }
+    }
+    throw new IOException(
+        "Cannot locate a nameservice for block pool " + blockPoolId);
+  }
+
+  /**
+   * Get the possible locations of a path in the federated cluster.
+   *
+   * @param path Path to check.
+   * @param failIfLocked Fail the request if locked (top mount point).
+   * @return Prioritized list of locations in the federated cluster.
+   * @throws IOException If the location for this path cannot be determined.
+   */
+  private List<RemoteLocation> getLocationsForPath(
+      String path, boolean failIfLocked) throws IOException {
+    // Check the location for this path
+    final PathLocation location =
+        this.subclusterResolver.getDestinationForPath(path);
+    if (location == null) {
+      throw new IOException("Cannot find locations for " + path + " in " +
+          this.subclusterResolver);
+    }
+
+    // Log the access to a path
+    return location.getDestinations();
+  }
+
+  /**
+   * Get the modification dates for mount points.
+   *
+   * @param path Name of the path to start checking dates from.
+   * @return Map with the modification dates for all sub-entries.
+   */
+  private Map<String, Long> getMountPointDates(String path) {
+    Map<String, Long> ret = new TreeMap<>();
+    // TODO add when we have a Mount Table
+    return ret;
+  }
+
+  /**
+   * Create a new file status for a mount point.
+   *
+   * @param name Name of the mount point.
+   * @param childrenNum Number of children.
+   * @param dates Map with the dates.
+   * @return New HDFS file status representing a mount point.
+   */
+  private HdfsFileStatus getMountPointStatus(
+      String name, int childrenNum, long date) {
+    long modTime = date;
+    long accessTime = date;
+    FsPermission permission = FsPermission.getDirDefault();
+    String owner = this.superUser;
+    String group = this.superGroup;
+    try {
+      // TODO support users, it should be the user for the pointed folder
+      UserGroupInformation ugi = getRemoteUser();
+      owner = ugi.getUserName();
+      group = ugi.getPrimaryGroupName();
+    } catch (IOException e) {
+      LOG.error("Cannot get the remote user: {}", e.getMessage());
+    }
+    long inodeId = 0;
+    return new HdfsFileStatus(0, true, 0, 0, modTime, accessTime, permission,
+        owner, group, new byte[0], DFSUtil.string2Bytes(name), inodeId,
+        childrenNum, null, (byte) 0, null);
+  }
+
+  /**
+   * Get the name of the method that is calling this function.
+   *
+   * @return Name of the method calling this function.
+   */
+  private static String getMethodName() {
+    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
+    String methodName = stack[3].getMethodName();
+    return methodName;
+  }
+
+  /**
+   * Get the user that is invoking this operation.
+   *
+   * @return Remote user group information.
+   * @throws IOException If we cannot get the user information.
+   */
+  static UserGroupInformation getRemoteUser() throws IOException {
+    UserGroupInformation ugi = Server.getRemoteUser();
+    return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/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 a8410b2..33fff0f 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
@@ -4620,6 +4620,101 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.default.nameserviceId</name>
+    <value></value>
+    <description>
+      Nameservice identifier of the default subcluster to monitor.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.rpc.enable</name>
+    <value>true</value>
+    <description>
+      If the RPC service to handle client requests in the router is enabled.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.rpc-address</name>
+    <value>0.0.0.0:8888</value>
+    <description>
+      RPC address that handles all clients requests.
+      The value of this property will take the form of router-host1:rpc-port.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.rpc-bind-host</name>
+    <value></value>
+    <description>
+      The actual address the RPC server will bind to. If this optional address is
+      set, it overrides only the hostname portion of
+      dfs.federation.router.rpc-address. This is useful for making the name node
+      listen on all interfaces by setting it to 0.0.0.0.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.handler.count</name>
+    <value>10</value>
+    <description>
+      The number of server threads for the router to handle RPC requests from
+      clients.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.handler.queue.size</name>
+    <value>100</value>
+    <description>
+      The size of the queue for the number of handlers to handle RPC client requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.reader.count</name>
+    <value>1</value>
+    <description>
+      The number of readers for the router to handle RPC client requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.reader.queue.size</name>
+    <value>100</value>
+    <description>
+      The size of the queue for the number of readers for the router to handle RPC client requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.connection.pool-size</name>
+    <value>1</value>
+    <description>
+      Size of the pool of connections from the router to namenodes.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.connection.clean.ms</name>
+    <value>10000</value>
+    <description>
+      Time interval, in milliseconds, to check if the connection pool should
+      remove unused connections.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.connection.pool.clean.ms</name>
+    <value>60000</value>
+    <description>
+      Time interval, in milliseconds, to check if the connection manager should
+      remove unused connection pools.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.file.resolver.client.class</name>
     <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
index 8674682..bbb548ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.federation;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.FileNotFoundException;
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
@@ -51,17 +52,11 @@ import org.apache.hadoop.security.AccessControlException;
  */
 public final class FederationTestUtils {
 
-  public final static String NAMESERVICE1 = "ns0";
-  public final static String NAMESERVICE2 = "ns1";
-  public final static String NAMENODE1 = "nn0";
-  public final static String NAMENODE2 = "nn1";
-  public final static String NAMENODE3 = "nn2";
-  public final static String NAMENODE4 = "nn3";
-  public final static String ROUTER1 = "router0";
-  public final static String ROUTER2 = "router1";
-  public final static String ROUTER3 = "router2";
-  public final static String ROUTER4 = "router3";
-  public final static long BLOCK_SIZE_BYTES = 134217728;
+  public final static String[] NAMESERVICES = {"ns0", "ns1"};
+  public final static String[] NAMENODES = {"nn0", "nn1", "nn2", "nn3"};
+  public final static String[] ROUTERS =
+      {"router0", "router1", "router2", "router3"};
+
 
   private FederationTestUtils() {
     // Utility class
@@ -81,7 +76,7 @@ public final class FederationTestUtils {
       triggeredException = e;
     }
     if (exceptionClass != null) {
-      assertNotNull("No exception was triggered, expected exception - "
+      assertNotNull("No exception was triggered, expected exception"
           + exceptionClass.getName(), triggeredException);
       assertEquals(exceptionClass, triggeredException.getClass());
     } else {
@@ -101,48 +96,43 @@ public final class FederationTestUtils {
       return report;
     }
     report.setHAServiceState(state);
-    report.setNamespaceInfo(new NamespaceInfo(1, "tesclusterid", ns, 0,
-            "testbuildvesion", "testsoftwareversion"));
+    NamespaceInfo nsInfo = new NamespaceInfo(
+        1, "tesclusterid", ns, 0, "testbuildvesion", "testsoftwareversion");
+    report.setNamespaceInfo(nsInfo);
     return report;
   }
 
   public static void waitNamenodeRegistered(ActiveNamenodeResolver resolver,
-      String nameserviceId, String namenodeId,
-      FederationNamenodeServiceState finalState)
+      String nsId, String nnId, FederationNamenodeServiceState finalState)
           throws InterruptedException, IllegalStateException, IOException {
 
     for (int loopCount = 0; loopCount < 20; loopCount++) {
-
       if (loopCount > 0) {
         Thread.sleep(1000);
       }
 
-      List<? extends FederationNamenodeContext> namenodes;
-      namenodes =
-          resolver.getNamenodesForNameserviceId(nameserviceId);
+      List<? extends FederationNamenodeContext> namenodes =
+          resolver.getNamenodesForNameserviceId(nsId);
       for (FederationNamenodeContext namenode : namenodes) {
-        if (namenodeId != null
-            && !namenode.getNamenodeId().equals(namenodeId)) {
-          // Keep looking
-          continue;
+        // Check if this is the Namenode we are checking
+        if (namenode.getNamenodeId() == nnId  ||
+            namenode.getNamenodeId().equals(nnId)) {
+          if (finalState != null && !namenode.getState().equals(finalState)) {
+            // Wrong state, wait a bit more
+            break;
+          } else {
+            // Found and verified
+            return;
+          }
         }
-        if (finalState != null && !namenode.getState().equals(finalState)) {
-          // Wrong state, wait a bit more
-          break;
-        }
-        // Found
-        return;
       }
     }
-    assertTrue("Failed to verify state store registration for state - "
-        + finalState + " - " + " - " + nameserviceId + " - ", false);
+    fail("Failed to verify State Store registration of " + nsId + " " + nnId +
+        " for state " + finalState);
   }
 
   public static boolean verifyDate(Date d1, Date d2, long precision) {
-    if (Math.abs(d1.getTime() - d2.getTime()) < precision) {
-      return true;
-    }
-    return false;
+    return Math.abs(d1.getTime() - d2.getTime()) < precision;
   }
 
   public static boolean addDirectory(FileSystem context, String path)
@@ -165,15 +155,14 @@ public final class FederationTestUtils {
     } catch (Exception e) {
       return false;
     }
-
     return false;
   }
 
-  public static boolean checkForFileInDirectory(FileSystem context,
-      String testPath, String targetFile) throws AccessControlException,
-          FileNotFoundException,
-          UnsupportedFileSystemException, IllegalArgumentException,
-          IOException {
+  public static boolean checkForFileInDirectory(
+      FileSystem context, String testPath, String targetFile)
+          throws IOException, AccessControlException, FileNotFoundException,
+          UnsupportedFileSystemException, IllegalArgumentException {
+
     FileStatus[] fileStatus = context.listStatus(new Path(testPath));
     String file = null;
     String verifyPath = testPath + "/" + targetFile;
@@ -194,7 +183,8 @@ public final class FederationTestUtils {
 
   public static int countContents(FileSystem context, String testPath)
       throws IOException {
-    FileStatus[] fileStatus = context.listStatus(new Path(testPath));
+    Path path = new Path(testPath);
+    FileStatus[] fileStatus = context.listStatus(path);
     return fileStatus.length;
   }
 
@@ -202,7 +192,7 @@ public final class FederationTestUtils {
       throws IOException {
     FsPermission permissions = new FsPermission("700");
     FSDataOutputStream writeStream = fs.create(new Path(path), permissions,
-        true, 1000, (short) 1, BLOCK_SIZE_BYTES, null);
+        true, 1000, (short) 1, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null);
     for (int i = 0; i < length; i++) {
       writeStream.write(i);
     }


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


[17/29] hadoop git commit: HDFS-10880. Federation Mount Table State Store internal API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryRequestPBImpl.java
new file mode 100644
index 0000000..7f7c998
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryRequestPBImpl.java
@@ -0,0 +1,76 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * RemoveMountTableEntryRequest.
+ */
+public class RemoveMountTableEntryRequestPBImpl
+    extends RemoveMountTableEntryRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<RemoveMountTableEntryRequestProto,
+      RemoveMountTableEntryRequestProto.Builder,
+      RemoveMountTableEntryRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<RemoveMountTableEntryRequestProto,
+              RemoveMountTableEntryRequestProto.Builder,
+              RemoveMountTableEntryRequestProtoOrBuilder>(
+                  RemoveMountTableEntryRequestProto.class);
+
+  public RemoveMountTableEntryRequestPBImpl() {
+  }
+
+  public RemoveMountTableEntryRequestPBImpl(
+      RemoveMountTableEntryRequestProto proto) {
+    this.setProto(proto);
+  }
+
+  @Override
+  public RemoveMountTableEntryRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public String getSrcPath() {
+    return this.translator.getProtoOrBuilder().getSrcPath();
+  }
+
+  @Override
+  public void setSrcPath(String path) {
+    this.translator.getBuilder().setSrcPath(path);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryResponsePBImpl.java
new file mode 100644
index 0000000..0c943ac
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RemoveMountTableEntryResponsePBImpl.java
@@ -0,0 +1,76 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * RemoveMountTableEntryResponse.
+ */
+public class RemoveMountTableEntryResponsePBImpl
+    extends RemoveMountTableEntryResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<RemoveMountTableEntryResponseProto,
+      Builder, RemoveMountTableEntryResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<RemoveMountTableEntryResponseProto,
+              RemoveMountTableEntryResponseProto.Builder,
+              RemoveMountTableEntryResponseProtoOrBuilder>(
+                  RemoveMountTableEntryResponseProto.class);
+
+  public RemoveMountTableEntryResponsePBImpl() {
+  }
+
+  public RemoveMountTableEntryResponsePBImpl(
+      RemoveMountTableEntryResponseProto proto) {
+    this.setProto(proto);
+  }
+
+  @Override
+  public RemoveMountTableEntryResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getStatus() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setStatus(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryRequestPBImpl.java
new file mode 100644
index 0000000..621bb3a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryRequestPBImpl.java
@@ -0,0 +1,96 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.MountTablePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * UpdateMountTableEntryRequest.
+ */
+public class UpdateMountTableEntryRequestPBImpl
+    extends UpdateMountTableEntryRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<UpdateMountTableEntryRequestProto,
+      UpdateMountTableEntryRequestProto.Builder,
+      UpdateMountTableEntryRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<UpdateMountTableEntryRequestProto,
+              UpdateMountTableEntryRequestProto.Builder,
+              UpdateMountTableEntryRequestProtoOrBuilder>(
+                  UpdateMountTableEntryRequestProto.class);
+
+  public UpdateMountTableEntryRequestPBImpl() {
+  }
+
+  public UpdateMountTableEntryRequestPBImpl(
+      UpdateMountTableEntryRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public UpdateMountTableEntryRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public MountTable getEntry() throws IOException {
+    MountTableRecordProto statsProto =
+        this.translator.getProtoOrBuilder().getEntry();
+    MountTable stats = StateStoreSerializer.newRecord(MountTable.class);
+    if (stats instanceof MountTablePBImpl) {
+      MountTablePBImpl entryPB = (MountTablePBImpl)stats;
+      entryPB.setProto(statsProto);
+      return entryPB;
+    } else {
+      throw new IOException("Cannot get stats for the membership");
+    }
+  }
+
+  @Override
+  public void setEntry(MountTable mount) throws IOException {
+    if (mount instanceof MountTablePBImpl) {
+      MountTablePBImpl mountPB = (MountTablePBImpl)mount;
+      MountTableRecordProto mountProto =
+          (MountTableRecordProto)mountPB.getProto();
+      this.translator.getBuilder().setEntry(mountProto);
+    } else {
+      throw new IOException("Cannot set mount table entry");
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryResponsePBImpl.java
new file mode 100644
index 0000000..5d566d6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/UpdateMountTableEntryResponsePBImpl.java
@@ -0,0 +1,76 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * UpdateMountTableEntryResponse.
+ */
+public class UpdateMountTableEntryResponsePBImpl
+    extends UpdateMountTableEntryResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<UpdateMountTableEntryResponseProto,
+      UpdateMountTableEntryResponseProto.Builder,
+      UpdateMountTableEntryResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<UpdateMountTableEntryResponseProto,
+          UpdateMountTableEntryResponseProto.Builder,
+          UpdateMountTableEntryResponseProtoOrBuilder>(
+              UpdateMountTableEntryResponseProto.class);
+
+  public UpdateMountTableEntryResponsePBImpl() {
+  }
+
+  public UpdateMountTableEntryResponsePBImpl(
+      UpdateMountTableEntryResponseProto proto) {
+    this.setProto(proto);
+  }
+
+  @Override
+  public UpdateMountTableEntryResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getStatus() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setStatus(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/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
new file mode 100644
index 0000000..16f2b8b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
@@ -0,0 +1,301 @@
+/**
+ * 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.store.records;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.fs.Path;
+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.store.driver.StateStoreSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Data schema for
+ * {@link org.apache.hadoop.hdfs.server.federation.store.
+ * MountTableStore FederationMountTableStore} data stored in the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.
+ * StateStoreService FederationStateStoreService}. Supports string
+ * serialization.
+ */
+public abstract class MountTable extends BaseRecord {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MountTable.class);
+
+
+  /**
+   * Default constructor for a mount table entry.
+   */
+  public MountTable() {
+    super();
+  }
+
+  public static MountTable newInstance() {
+    MountTable record = StateStoreSerializer.newRecord(MountTable.class);
+    record.init();
+    return record;
+  }
+
+  /**
+   * Constructor for a mount table entry with a single destinations.
+   *
+   * @param src Source path in the mount entry.
+   * @param destinations Nameservice destination of the mount point.
+   * @param dateCreated Created date.
+   * @param dateModified Modified date.
+   * @throws IOException
+   */
+  public static MountTable newInstance(final String src,
+      final Map<String, String> destinations,
+      long dateCreated, long dateModified) throws IOException {
+
+    MountTable record = newInstance(src, destinations);
+    record.setDateCreated(dateCreated);
+    record.setDateModified(dateModified);
+    return record;
+  }
+
+  /**
+   * Constructor for a mount table entry with multiple destinations.
+   *
+   * @param src Source path in the mount entry.
+   * @param destinations Nameservice destinations of the mount point.
+   * @throws IOException
+   */
+  public static MountTable newInstance(final String src,
+      final Map<String, String> destinations) throws IOException {
+    MountTable record = newInstance();
+
+    // Normalize the mount path
+    record.setSourcePath(normalizeFileSystemPath(src));
+
+    // Build a list of remote locations
+    final List<RemoteLocation> locations = new LinkedList<>();
+    for (Entry<String, String> entry : destinations.entrySet()) {
+      String nsId = entry.getKey();
+      String path = normalizeFileSystemPath(entry.getValue());
+      RemoteLocation location = new RemoteLocation(nsId, path);
+      locations.add(location);
+    }
+
+    // Set the serialized dest string
+    record.setDestinations(locations);
+
+    // Validate
+    record.validate();
+    return record;
+  }
+
+  /**
+   * Get source path in the federated namespace.
+   *
+   * @return Source path in the federated namespace.
+   */
+  public abstract String getSourcePath();
+
+  /**
+   * Set source path in the federated namespace.
+   *
+   * @param path Source path in the federated namespace.
+   */
+  public abstract void setSourcePath(String path);
+
+  /**
+   * Get a list of destinations (namespace + path) present for this entry.
+   *
+   * @return List of RemoteLocation destinations. Null if no destinations.
+   */
+  public abstract List<RemoteLocation> getDestinations();
+
+  /**
+   * Set the destination paths.
+   *
+   * @param paths Destination paths.
+   */
+  public abstract void setDestinations(List<RemoteLocation> dests);
+
+  /**
+   * Add a new destination to this mount table entry.
+   */
+  public abstract boolean addDestination(String nsId, String path);
+
+  /**
+   * Check if the entry is read only.
+   *
+   * @return If the entry is read only.
+   */
+  public abstract boolean isReadOnly();
+
+  /**
+   * Set an entry to be read only.
+   *
+   * @param ro If the entry is read only.
+   */
+  public abstract void setReadOnly(boolean ro);
+
+  /**
+   * Get the order of the destinations for this mount table entry.
+   *
+   * @return Order of the destinations.
+   */
+  public abstract DestinationOrder getDestOrder();
+
+  /**
+   * Set the order of the destinations for this mount table entry.
+   *
+   * @param order Order of the destinations.
+   */
+  public abstract void setDestOrder(DestinationOrder order);
+
+  /**
+   * Get the default location.
+   * @return The default location.
+   */
+  public RemoteLocation getDefaultLocation() {
+    List<RemoteLocation> dests = this.getDestinations();
+    if (dests == null || dests.isEmpty()) {
+      return null;
+    }
+    return dests.get(0);
+  }
+
+  @Override
+  public boolean like(final BaseRecord o) {
+    if (o instanceof MountTable) {
+      MountTable other = (MountTable)o;
+      if (getSourcePath() != null &&
+          !getSourcePath().equals(other.getSourcePath())) {
+        return false;
+      }
+      if (getDestinations() != null &&
+          !getDestinations().equals(other.getDestinations())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(this.getSourcePath());
+    sb.append("->");
+    List<RemoteLocation> destinations = this.getDestinations();
+    sb.append(destinations);
+    if (destinations != null && destinations.size() > 1) {
+      sb.append("[" + this.getDestOrder() + "]");
+    }
+    if (this.isReadOnly()) {
+      sb.append("[RO]");
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public SortedMap<String, String> getPrimaryKeys() {
+    SortedMap<String, String> map = new TreeMap<>();
+    map.put("sourcePath", this.getSourcePath());
+    return map;
+  }
+
+  @Override
+  public boolean validate() {
+    boolean ret = super.validate();
+    if (this.getSourcePath() == null || this.getSourcePath().length() == 0) {
+      LOG.error("Invalid entry, no source path specified ", this);
+      ret = false;
+    }
+    if (!this.getSourcePath().startsWith("/")) {
+      LOG.error("Invalid entry, all mount points must start with / ", this);
+      ret = false;
+    }
+    if (this.getDestinations() == null || this.getDestinations().size() == 0) {
+      LOG.error("Invalid entry, no destination paths specified ", this);
+      ret = false;
+    }
+    for (RemoteLocation loc : getDestinations()) {
+      String nsId = loc.getNameserviceId();
+      if (nsId == null || nsId.length() == 0) {
+        LOG.error("Invalid entry, invalid destination nameservice ", this);
+        ret = false;
+      }
+      if (loc.getDest() == null || loc.getDest().length() == 0) {
+        LOG.error("Invalid entry, invalid destination path ", this);
+        ret = false;
+      }
+      if (!loc.getDest().startsWith("/")) {
+        LOG.error("Invalid entry, all destination must start with / ", this);
+        ret = false;
+      }
+    }
+    return ret;
+  }
+
+  @Override
+  public long getExpirationMs() {
+    return 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 31)
+        .append(this.getSourcePath())
+        .append(this.getDestinations())
+        .append(this.isReadOnly())
+        .append(this.getDestOrder())
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof MountTable) {
+      MountTable other = (MountTable)obj;
+      if (!this.getSourcePath().equals(other.getSourcePath())) {
+        return false;
+      } else if (!this.getDestinations().equals(other.getDestinations())) {
+        return false;
+      } else if (this.isReadOnly() != other.isReadOnly()) {
+        return false;
+      } else if (!this.getDestOrder().equals(other.getDestOrder())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Normalize a path for that filesystem.
+   *
+   * @param path Path to normalize.
+   * @return Normalized path.
+   */
+  private static String normalizeFileSystemPath(final String path) {
+    Path normalizedPath = new Path(path);
+    return normalizedPath.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/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
new file mode 100644
index 0000000..d2870bd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.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.federation.store.records.impl.pb;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+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;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProtoOrBuilder;
+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.store.protocol.impl.pb.FederationProtocolPBTranslator;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the MountTable record.
+ */
+public class MountTablePBImpl extends MountTable implements PBRecord {
+
+  private FederationProtocolPBTranslator<MountTableRecordProto, Builder,
+      MountTableRecordProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<MountTableRecordProto, Builder,
+              MountTableRecordProtoOrBuilder>(MountTableRecordProto.class);
+
+  public MountTablePBImpl() {
+  }
+
+  public MountTablePBImpl(MountTableRecordProto proto) {
+    this.setProto(proto);
+  }
+
+  @Override
+  public MountTableRecordProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public String getSourcePath() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasSrcPath()) {
+      return null;
+    }
+    return proto.getSrcPath();
+  }
+
+  @Override
+  public void setSourcePath(String path) {
+    Builder builder = this.translator.getBuilder();
+    if (path == null) {
+      builder.clearSrcPath();
+    } else {
+      builder.setSrcPath(path);
+    }
+  }
+
+  @Override
+  public List<RemoteLocation> getDestinations() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (proto.getDestinationsCount() == 0) {
+      return null;
+    }
+
+    final List<RemoteLocation> ret = new LinkedList<>();
+    final List<RemoteLocationProto> destList = proto.getDestinationsList();
+    for (RemoteLocationProto dest : destList) {
+      String nsId = dest.getNameserviceId();
+      String path = dest.getPath();
+      RemoteLocation loc = new RemoteLocation(nsId, path);
+      ret.add(loc);
+    }
+    return ret;
+  }
+
+  @Override
+  public void setDestinations(final List<RemoteLocation> dests) {
+    Builder builder = this.translator.getBuilder();
+    builder.clearDestinations();
+    for (RemoteLocation dest : dests) {
+      RemoteLocationProto.Builder itemBuilder =
+          RemoteLocationProto.newBuilder();
+      String nsId = dest.getNameserviceId();
+      String path = dest.getDest();
+      itemBuilder.setNameserviceId(nsId);
+      itemBuilder.setPath(path);
+      RemoteLocationProto item = itemBuilder.build();
+      builder.addDestinations(item);
+    }
+  }
+
+  @Override
+  public boolean addDestination(String nsId, String path) {
+    // Check if the location is already there
+    List<RemoteLocation> dests = getDestinations();
+    for (RemoteLocation dest : dests) {
+      if (dest.getNameserviceId().equals(nsId) && dest.getDest().equals(path)) {
+        return false;
+      }
+    }
+
+    // Add it to the existing list
+    Builder builder = this.translator.getBuilder();
+    RemoteLocationProto.Builder itemBuilder =
+        RemoteLocationProto.newBuilder();
+    itemBuilder.setNameserviceId(nsId);
+    itemBuilder.setPath(path);
+    RemoteLocationProto item = itemBuilder.build();
+    builder.addDestinations(item);
+    return true;
+  }
+
+  @Override
+  public void setDateModified(long time) {
+    this.translator.getBuilder().setDateModified(time);
+  }
+
+  @Override
+  public long getDateModified() {
+    return this.translator.getProtoOrBuilder().getDateModified();
+  }
+
+  @Override
+  public void setDateCreated(long time) {
+    this.translator.getBuilder().setDateCreated(time);
+  }
+
+  @Override
+  public long getDateCreated() {
+    return this.translator.getProtoOrBuilder().getDateCreated();
+  }
+
+  @Override
+  public boolean isReadOnly() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasReadOnly()) {
+      return false;
+    }
+    return proto.getReadOnly();
+  }
+
+  @Override
+  public void setReadOnly(boolean ro) {
+    this.translator.getBuilder().setReadOnly(ro);
+  }
+
+  @Override
+  public DestinationOrder getDestOrder() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    return convert(proto.getDestOrder());
+  }
+
+  @Override
+  public void setDestOrder(DestinationOrder order) {
+    Builder builder = this.translator.getBuilder();
+    if (order == null) {
+      builder.clearDestOrder();
+    } else {
+      builder.setDestOrder(convert(order));
+    }
+  }
+
+  private DestinationOrder convert(DestOrder order) {
+    switch (order) {
+    case LOCAL:
+      return DestinationOrder.LOCAL;
+    case RANDOM:
+      return DestinationOrder.RANDOM;
+    default:
+      return DestinationOrder.HASH;
+    }
+  }
+
+  private DestOrder convert(DestinationOrder order) {
+    switch (order) {
+    case LOCAL:
+      return DestOrder.LOCAL;
+    case RANDOM:
+      return DestOrder.RANDOM;
+    default:
+      return DestOrder.HASH;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/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 487fe46..32a6250 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
@@ -104,4 +104,63 @@ message NamenodeHeartbeatRequestProto {
 
 message NamenodeHeartbeatResponseProto {
   optional bool status = 1;
-}
\ No newline at end of file
+}
+
+
+/////////////////////////////////////////////////
+// Mount table
+/////////////////////////////////////////////////
+
+message RemoteLocationProto {
+  optional string nameserviceId = 1;
+  optional string path = 2;
+}
+
+message MountTableRecordProto {
+  optional string srcPath = 1;
+  repeated RemoteLocationProto destinations = 2;
+  optional uint64 dateCreated = 3;
+  optional uint64 dateModified = 4;
+  optional bool readOnly = 5 [default = false];
+
+  enum DestOrder {
+    HASH = 0;
+    LOCAL = 1;
+    RANDOM = 2;
+  }
+  optional DestOrder destOrder = 6 [default = HASH];
+}
+
+message AddMountTableEntryRequestProto {
+  optional MountTableRecordProto entry = 1;
+}
+
+message AddMountTableEntryResponseProto {
+  optional bool status = 1;
+}
+
+message UpdateMountTableEntryRequestProto {
+  optional MountTableRecordProto entry = 1;
+}
+
+message UpdateMountTableEntryResponseProto {
+  optional bool status = 1;
+}
+
+message RemoveMountTableEntryRequestProto {
+  optional string srcPath = 1;
+}
+
+message RemoveMountTableEntryResponseProto{
+  optional bool status = 1;
+}
+
+message GetMountTableEntriesRequestProto {
+  optional string srcPath = 1;
+}
+
+message GetMountTableEntriesResponseProto {
+  repeated MountTableRecordProto entries = 1;
+  optional uint64 timestamp = 2;
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
index 87427fd..a481553 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.NamenodePriorityCompara
 import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
 import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.util.Time;
 
@@ -68,6 +69,10 @@ public class MockResolver
     this();
   }
 
+  public MockResolver(Configuration conf, Router router) {
+    this();
+  }
+
   public void addLocation(String mount, String nsId, String location) {
     List<RemoteLocation> locationsList = this.locations.get(mount);
     if (locationsList == null) {
@@ -258,7 +263,6 @@ public class MockResolver
 
   @Override
   public PathLocation getDestinationForPath(String path) throws IOException {
-    Set<String> namespaceSet = new HashSet<>();
     List<RemoteLocation> remoteLocations = new LinkedList<>();
     for (String key : this.locations.keySet()) {
       if (path.startsWith(key)) {
@@ -268,7 +272,6 @@ public class MockResolver
           RemoteLocation remoteLocation =
               new RemoteLocation(nameservice, finalPath);
           remoteLocations.add(remoteLocation);
-          namespaceSet.add(nameservice);
         }
         break;
       }
@@ -277,7 +280,7 @@ public class MockResolver
       // Path isn't supported, mimic resolver behavior.
       return null;
     }
-    return new PathLocation(path, remoteLocations, namespaceSet);
+    return new PathLocation(path, remoteLocations);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
new file mode 100644
index 0000000..682d569
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
@@ -0,0 +1,396 @@
+/**
+ * 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.resolver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test the {@link MountTableStore} from the {@link Router}.
+ */
+public class TestMountTableResolver {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMountTableResolver.class);
+
+  private MountTableResolver mountTable;
+
+  private Map<String, String> getMountTableEntry(
+      String subcluster, String path) {
+    Map<String, String> ret = new HashMap<>();
+    ret.put(subcluster, path);
+    return ret;
+  }
+
+  /**
+   * Setup the mount table.
+   * / -> 1:/
+   * __tmp -> 2:/tmp
+   * __user -> 3:/user
+   * ____a -> 2:/user/test
+   * ______demo
+   * ________test
+   * __________a -> 1:/user/test
+   * __________b -> 3:/user/test
+   * ____b
+   * ______file1.txt -> 4:/user/file1.txt
+   * __usr
+   * ____bin -> 2:/bin
+   *
+   * @throws IOException If it cannot set the mount table.
+   */
+  private void setupMountTable() throws IOException {
+    Configuration conf = new Configuration();
+    mountTable = new MountTableResolver(conf);
+
+    // Root mount point
+    Map<String, String> map = getMountTableEntry("1", "/");
+    mountTable.addEntry(MountTable.newInstance("/", map));
+
+    // /tmp
+    map = getMountTableEntry("2", "/");
+    mountTable.addEntry(MountTable.newInstance("/tmp", map));
+
+    // /user
+    map = getMountTableEntry("3", "/user");
+    mountTable.addEntry(MountTable.newInstance("/user", map));
+
+    // /usr/bin
+    map = getMountTableEntry("2", "/bin");
+    mountTable.addEntry(MountTable.newInstance("/usr/bin", map));
+
+    // /user/a
+    map = getMountTableEntry("2", "/user/test");
+    mountTable.addEntry(MountTable.newInstance("/user/a", map));
+
+    // /user/b/file1.txt
+    map = getMountTableEntry("4", "/user/file1.txt");
+    mountTable.addEntry(MountTable.newInstance("/user/b/file1.txt", map));
+
+    // /user/a/demo/test/a
+    map = getMountTableEntry("1", "/user/test");
+    mountTable.addEntry(MountTable.newInstance("/user/a/demo/test/a", map));
+
+    // /user/a/demo/test/b
+    map = getMountTableEntry("3", "/user/test");
+    mountTable.addEntry(MountTable.newInstance("/user/a/demo/test/b", map));
+  }
+
+  @Before
+  public void setup() throws IOException {
+    setupMountTable();
+  }
+
+  @Test
+  public void testDestination() throws IOException {
+
+    // Check files
+    assertEquals("1->/tesfile1.txt",
+        mountTable.getDestinationForPath("/tesfile1.txt").toString());
+
+    assertEquals("3->/user/testfile2.txt",
+        mountTable.getDestinationForPath("/user/testfile2.txt").toString());
+
+    assertEquals("2->/user/test/testfile3.txt",
+        mountTable.getDestinationForPath("/user/a/testfile3.txt").toString());
+
+    assertEquals("3->/user/b/testfile4.txt",
+        mountTable.getDestinationForPath("/user/b/testfile4.txt").toString());
+
+    assertEquals("1->/share/file5.txt",
+        mountTable.getDestinationForPath("/share/file5.txt").toString());
+
+    assertEquals("2->/bin/file7.txt",
+        mountTable.getDestinationForPath("/usr/bin/file7.txt").toString());
+
+    assertEquals("1->/usr/file8.txt",
+        mountTable.getDestinationForPath("/usr/file8.txt").toString());
+
+    assertEquals("2->/user/test/demo/file9.txt",
+        mountTable.getDestinationForPath("/user/a/demo/file9.txt").toString());
+
+    // Check folders
+    assertEquals("3->/user/testfolder",
+        mountTable.getDestinationForPath("/user/testfolder").toString());
+
+    assertEquals("2->/user/test/b",
+        mountTable.getDestinationForPath("/user/a/b").toString());
+
+    assertEquals("3->/user/test/a",
+        mountTable.getDestinationForPath("/user/test/a").toString());
+
+  }
+
+  private void compareLists(List<String> list1, String[] list2) {
+    assertEquals(list1.size(), list2.length);
+    for (String item : list2) {
+      assertTrue(list1.contains(item));
+    }
+  }
+
+  @Test
+  public void testGetMountPoints() throws IOException {
+
+    // Check getting all mount points (virtual and real) beneath a path
+    List<String> mounts = mountTable.getMountPoints("/");
+    assertEquals(3, mounts.size());
+    compareLists(mounts, new String[] {"tmp", "user", "usr"});
+
+    mounts = mountTable.getMountPoints("/user");
+    assertEquals(2, mounts.size());
+    compareLists(mounts, new String[] {"a", "b"});
+
+    mounts = mountTable.getMountPoints("/user/a");
+    assertEquals(1, mounts.size());
+    compareLists(mounts, new String[] {"demo"});
+
+    mounts = mountTable.getMountPoints("/user/a/demo");
+    assertEquals(1, mounts.size());
+    compareLists(mounts, new String[] {"test"});
+
+    mounts = mountTable.getMountPoints("/user/a/demo/test");
+    assertEquals(2, mounts.size());
+    compareLists(mounts, new String[] {"a", "b"});
+
+    mounts = mountTable.getMountPoints("/tmp");
+    assertEquals(0, mounts.size());
+
+    mounts = mountTable.getMountPoints("/t");
+    assertNull(mounts);
+
+    mounts = mountTable.getMountPoints("/unknownpath");
+    assertNull(mounts);
+  }
+
+  private void compareRecords(List<MountTable> list1, String[] list2) {
+    assertEquals(list1.size(), list2.length);
+    for (String item : list2) {
+      for (MountTable record : list1) {
+        if (record.getSourcePath().equals(item)) {
+          return;
+        }
+      }
+    }
+    fail();
+  }
+
+  @Test
+  public void testGetMounts() throws IOException {
+
+    // Check listing the mount table records at or beneath a path
+    List<MountTable> records = mountTable.getMounts("/");
+    assertEquals(8, records.size());
+    compareRecords(records, new String[] {"/", "/tmp", "/user", "/usr/bin",
+        "user/a", "/user/a/demo/a", "/user/a/demo/b", "/user/b/file1.txt"});
+
+    records = mountTable.getMounts("/user");
+    assertEquals(5, records.size());
+    compareRecords(records, new String[] {"/user", "/user/a/demo/a",
+        "/user/a/demo/b", "user/a", "/user/b/file1.txt"});
+
+    records = mountTable.getMounts("/user/a");
+    assertEquals(3, records.size());
+    compareRecords(records,
+        new String[] {"/user/a/demo/a", "/user/a/demo/b", "/user/a"});
+
+    records = mountTable.getMounts("/tmp");
+    assertEquals(1, records.size());
+    compareRecords(records, new String[] {"/tmp"});
+  }
+
+  @Test
+  public void testRemoveSubTree()
+      throws UnsupportedOperationException, IOException {
+
+    // 3 mount points are present /tmp, /user, /usr
+    compareLists(mountTable.getMountPoints("/"),
+        new String[] {"user", "usr", "tmp"});
+
+    // /tmp currently points to namespace 2
+    assertEquals("2", mountTable.getDestinationForPath("/tmp/testfile.txt")
+        .getDefaultLocation().getNameserviceId());
+
+    // Remove tmp
+    mountTable.removeEntry("/tmp");
+
+    // Now 2 mount points are present /user, /usr
+    compareLists(mountTable.getMountPoints("/"),
+        new String[] {"user", "usr"});
+
+    // /tmp no longer exists, uses default namespace for mapping /
+    assertEquals("1", mountTable.getDestinationForPath("/tmp/testfile.txt")
+        .getDefaultLocation().getNameserviceId());
+  }
+
+  @Test
+  public void testRemoveVirtualNode()
+      throws UnsupportedOperationException, IOException {
+
+    // 3 mount points are present /tmp, /user, /usr
+    compareLists(mountTable.getMountPoints("/"),
+        new String[] {"user", "usr", "tmp"});
+
+    // /usr is virtual, uses namespace 1->/
+    assertEquals("1", mountTable.getDestinationForPath("/usr/testfile.txt")
+        .getDefaultLocation().getNameserviceId());
+
+    // Attempt to remove /usr
+    mountTable.removeEntry("/usr");
+
+    // Verify the remove failed
+    compareLists(mountTable.getMountPoints("/"),
+        new String[] {"user", "usr", "tmp"});
+  }
+
+  @Test
+  public void testRemoveLeafNode()
+      throws UnsupportedOperationException, IOException {
+
+    // /user/a/demo/test/a currently points to namespace 1
+    assertEquals("1", mountTable.getDestinationForPath("/user/a/demo/test/a")
+        .getDefaultLocation().getNameserviceId());
+
+    // Remove /user/a/demo/test/a
+    mountTable.removeEntry("/user/a/demo/test/a");
+
+    // Now /user/a/demo/test/a points to namespace 2 using the entry for /user/a
+    assertEquals("2", mountTable.getDestinationForPath("/user/a/demo/test/a")
+        .getDefaultLocation().getNameserviceId());
+
+    // Verify the virtual node at /user/a/demo still exists and was not deleted
+    compareLists(mountTable.getMountPoints("/user/a"), new String[] {"demo"});
+
+    // Verify the sibling node was unaffected and still points to ns 3
+    assertEquals("3", mountTable.getDestinationForPath("/user/a/demo/test/b")
+        .getDefaultLocation().getNameserviceId());
+  }
+
+  @Test
+  public void testRefreshEntries()
+      throws UnsupportedOperationException, IOException {
+
+    // Initial table loaded
+    testDestination();
+    assertEquals(8, mountTable.getMounts("/").size());
+
+    // Replace table with /1 and /2
+    List<MountTable> records = new ArrayList<>();
+    Map<String, String> map1 = getMountTableEntry("1", "/");
+    records.add(MountTable.newInstance("/1", map1));
+    Map<String, String> map2 = getMountTableEntry("2", "/");
+    records.add(MountTable.newInstance("/2", map2));
+    mountTable.refreshEntries(records);
+
+    // Verify addition
+    PathLocation destination1 = mountTable.getDestinationForPath("/1");
+    RemoteLocation defaultLoc1 = destination1.getDefaultLocation();
+    assertEquals("1", defaultLoc1.getNameserviceId());
+
+    PathLocation destination2 = mountTable.getDestinationForPath("/2");
+    RemoteLocation defaultLoc2 = destination2.getDefaultLocation();
+    assertEquals("2", defaultLoc2.getNameserviceId());
+
+    // Verify existing entries were removed
+    assertEquals(2, mountTable.getMounts("/").size());
+    boolean assertionThrown = false;
+    try {
+      testDestination();
+      fail();
+    } catch (AssertionError e) {
+      // The / entry was removed, so it triggers an exception
+      assertionThrown = true;
+    }
+    assertTrue(assertionThrown);
+  }
+
+  @Test
+  public void testMountTableScalability() throws IOException {
+
+    List<MountTable> emptyList = new ArrayList<>();
+    mountTable.refreshEntries(emptyList);
+
+    // Add 100,000 entries in flat list
+    for (int i = 0; i < 100000; i++) {
+      Map<String, String> map = getMountTableEntry("1", "/" + i);
+      MountTable record = MountTable.newInstance("/" + i, map);
+      mountTable.addEntry(record);
+      if (i % 10000 == 0) {
+        LOG.info("Adding flat mount record {}: {}", i, record);
+      }
+    }
+
+    assertEquals(100000, mountTable.getMountPoints("/").size());
+    assertEquals(100000, mountTable.getMounts("/").size());
+
+    // Add 1000 entries in deep list
+    mountTable.refreshEntries(emptyList);
+    String parent = "/";
+    for (int i = 0; i < 1000; i++) {
+      final int index = i;
+      Map<String, String> map = getMountTableEntry("1", "/" + index);
+      if (i > 0) {
+        parent = parent + "/";
+      }
+      parent = parent + i;
+      MountTable record = MountTable.newInstance(parent, map);
+      mountTable.addEntry(record);
+    }
+
+    assertEquals(1, mountTable.getMountPoints("/").size());
+    assertEquals(1000, mountTable.getMounts("/").size());
+
+    // Add 100,000 entries in deep and wide tree
+    mountTable.refreshEntries(emptyList);
+    Random rand = new Random();
+    parent = "/" + Integer.toString(rand.nextInt());
+    int numRootTrees = 1;
+    for (int i = 0; i < 100000; i++) {
+      final int index = i;
+      Map<String, String> map = getMountTableEntry("1", "/" + index);
+      parent = parent + "/" + i;
+      if (parent.length() > 2000) {
+        // Start new tree
+        parent = "/" + Integer.toString(rand.nextInt());
+        numRootTrees++;
+      }
+      MountTable record = MountTable.newInstance(parent, map);
+      mountTable.addEntry(record);
+    }
+
+    assertEquals(numRootTrees, mountTable.getMountPoints("/").size());
+    assertEquals(100000, mountTable.getMounts("/").size());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
index 598b9cf..dbb8f3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
@@ -25,7 +25,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
@@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFile
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.util.Time;
 
 /**
@@ -234,6 +237,19 @@ public final class FederationStateStoreTestUtils {
     return false;
   }
 
+  public static List<MountTable> createMockMountTable(
+      List<String> nameservices) throws IOException {
+    // create table entries
+    List<MountTable> entries = new ArrayList<>();
+    for (String ns : nameservices) {
+      Map<String, String> destMap = new HashMap<>();
+      destMap.put(ns, "/target-" + ns);
+      MountTable entry = MountTable.newInstance("/" + ns, destMap);
+      entries.add(entry);
+    }
+    return entries;
+  }
+
   public static MembershipState createMockRegistrationForNamenode(
       String nameserviceId, String namenodeId,
       FederationNamenodeServiceState state) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMountTable.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMountTable.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMountTable.java
new file mode 100644
index 0000000..d30d6ba
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMountTable.java
@@ -0,0 +1,250 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMESERVICES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyException;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.clearRecords;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.createMockMountTable;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.synchronizeRecords;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the basic {@link StateStoreService}
+ * {@link MountTableStore} functionality.
+ */
+public class TestStateStoreMountTable extends TestStateStoreBase {
+
+  private static List<String> nameservices;
+  private static MountTableStore mountStore;
+
+  @BeforeClass
+  public static void create() throws IOException {
+    nameservices = new ArrayList<>();
+    nameservices.add(NAMESERVICES[0]);
+    nameservices.add(NAMESERVICES[1]);
+  }
+
+  @Before
+  public void setup() throws IOException, InterruptedException {
+    mountStore =
+        getStateStore().getRegisteredRecordStore(MountTableStore.class);
+    // Clear Mount table registrations
+    assertTrue(clearRecords(getStateStore(), MountTable.class));
+  }
+
+  @Test
+  public void testStateStoreDisconnected() throws Exception {
+
+    // Close the data store driver
+    getStateStore().closeDriver();
+    assertFalse(getStateStore().isDriverReady());
+
+    // Test APIs that access the store to check they throw the correct exception
+    AddMountTableEntryRequest addRequest =
+        AddMountTableEntryRequest.newInstance();
+    verifyException(mountStore, "addMountTableEntry",
+        StateStoreUnavailableException.class,
+        new Class[] {AddMountTableEntryRequest.class},
+        new Object[] {addRequest});
+
+    UpdateMountTableEntryRequest updateRequest =
+        UpdateMountTableEntryRequest.newInstance();
+    verifyException(mountStore, "updateMountTableEntry",
+        StateStoreUnavailableException.class,
+        new Class[] {UpdateMountTableEntryRequest.class},
+        new Object[] {updateRequest});
+
+    RemoveMountTableEntryRequest removeRequest =
+        RemoveMountTableEntryRequest.newInstance();
+    verifyException(mountStore, "removeMountTableEntry",
+        StateStoreUnavailableException.class,
+        new Class[] {RemoveMountTableEntryRequest.class},
+        new Object[] {removeRequest});
+
+    GetMountTableEntriesRequest getRequest =
+        GetMountTableEntriesRequest.newInstance();
+    mountStore.loadCache(true);
+    verifyException(mountStore, "getMountTableEntries",
+        StateStoreUnavailableException.class,
+        new Class[] {GetMountTableEntriesRequest.class},
+        new Object[] {getRequest});
+  }
+
+  @Test
+  public void testSynchronizeMountTable() throws IOException {
+    // Synchronize and get mount table entries
+    List<MountTable> entries = createMockMountTable(nameservices);
+    assertTrue(synchronizeRecords(getStateStore(), entries, MountTable.class));
+    for (MountTable e : entries) {
+      mountStore.loadCache(true);
+      MountTable entry = getMountTableEntry(e.getSourcePath());
+      assertNotNull(entry);
+      assertEquals(e.getDefaultLocation().getDest(),
+          entry.getDefaultLocation().getDest());
+    }
+  }
+
+  @Test
+  public void testAddMountTableEntry() throws IOException {
+
+    // Add 1
+    List<MountTable> entries = createMockMountTable(nameservices);
+    List<MountTable> entries1 = getMountTableEntries("/").getRecords();
+    assertEquals(0, entries1.size());
+    MountTable entry0 = entries.get(0);
+    AddMountTableEntryRequest request =
+        AddMountTableEntryRequest.newInstance(entry0);
+    AddMountTableEntryResponse response =
+        mountStore.addMountTableEntry(request);
+    assertTrue(response.getStatus());
+
+    mountStore.loadCache(true);
+    List<MountTable> entries2 = getMountTableEntries("/").getRecords();
+    assertEquals(1, entries2.size());
+  }
+
+  @Test
+  public void testRemoveMountTableEntry() throws IOException {
+
+    // Add many
+    List<MountTable> entries = createMockMountTable(nameservices);
+    synchronizeRecords(getStateStore(), entries, MountTable.class);
+    mountStore.loadCache(true);
+    List<MountTable> entries1 = getMountTableEntries("/").getRecords();
+    assertEquals(entries.size(), entries1.size());
+
+    // Remove 1
+    RemoveMountTableEntryRequest request =
+        RemoveMountTableEntryRequest.newInstance();
+    request.setSrcPath(entries.get(0).getSourcePath());
+    assertTrue(mountStore.removeMountTableEntry(request).getStatus());
+
+    // Verify remove
+    mountStore.loadCache(true);
+    List<MountTable> entries2 = getMountTableEntries("/").getRecords();
+    assertEquals(entries.size() - 1, entries2.size());
+  }
+
+  @Test
+  public void testUpdateMountTableEntry() throws IOException {
+
+    // Add 1
+    List<MountTable> entries = createMockMountTable(nameservices);
+    MountTable entry0 = entries.get(0);
+    String srcPath = entry0.getSourcePath();
+    String nsId = entry0.getDefaultLocation().getNameserviceId();
+    AddMountTableEntryRequest request =
+        AddMountTableEntryRequest.newInstance(entry0);
+    AddMountTableEntryResponse response =
+        mountStore.addMountTableEntry(request);
+    assertTrue(response.getStatus());
+
+    // Verify
+    mountStore.loadCache(true);
+    MountTable matchingEntry0 = getMountTableEntry(srcPath);
+    assertNotNull(matchingEntry0);
+    assertEquals(nsId, matchingEntry0.getDefaultLocation().getNameserviceId());
+
+    // Edit destination nameservice for source path
+    Map<String, String> destMap =
+        Collections.singletonMap("testnameservice", "/");
+    MountTable replacement =
+        MountTable.newInstance(srcPath, destMap);
+    UpdateMountTableEntryRequest updateRequest =
+        UpdateMountTableEntryRequest.newInstance(replacement);
+    UpdateMountTableEntryResponse updateResponse =
+        mountStore.updateMountTableEntry(updateRequest);
+    assertTrue(updateResponse.getStatus());
+
+    // Verify
+    mountStore.loadCache(true);
+    MountTable matchingEntry1 = getMountTableEntry(srcPath);
+    assertNotNull(matchingEntry1);
+    assertEquals("testnameservice",
+        matchingEntry1.getDefaultLocation().getNameserviceId());
+  }
+
+  /**
+   * Gets an existing mount table record in the state store.
+   *
+   * @param mount The mount point of the record to remove.
+   * @return The matching record if found, null if it is not found.
+   * @throws IOException If the state store could not be accessed.
+   */
+  private MountTable getMountTableEntry(String mount) throws IOException {
+    GetMountTableEntriesRequest request =
+        GetMountTableEntriesRequest.newInstance(mount);
+    GetMountTableEntriesResponse response =
+        mountStore.getMountTableEntries(request);
+    List<MountTable> results = response.getEntries();
+    if (results.size() > 0) {
+      // First result is sorted to have the shortest mount string length
+      return results.get(0);
+    }
+    return null;
+  }
+
+  /**
+   * Fetch all mount table records beneath a root path.
+   *
+   * @param store FederationMountTableStore instance to commit the data.
+   * @param mount The root search path, enter "/" to return all mount table
+   *          records.
+   *
+   * @return A list of all mount table records found below the root mount.
+   *
+   * @throws IOException If the state store could not be accessed.
+   */
+  private QueryResult<MountTable> getMountTableEntries(String mount)
+      throws IOException {
+    if (mount == null) {
+      throw new IOException("Please specify a root search path");
+    }
+    GetMountTableEntriesRequest request =
+        GetMountTableEntriesRequest.newInstance();
+    request.setSrcPath(mount);
+    GetMountTableEntriesResponse response =
+        mountStore.getMountTableEntries(request);
+    List<MountTable> records = response.getEntries();
+    long timestamp = response.getTimestamp();
+    return new QueryResult<MountTable>(records, timestamp);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
index dc51ee9..8239fb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUt
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.server.federation.store.records.Query;
 import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
 import org.junit.AfterClass;
@@ -109,6 +111,11 @@ public class TestStateStoreDriverBase {
           generateRandomString(), generateRandomString(),
           generateRandomString(), generateRandomString(),
           generateRandomEnum(FederationNamenodeServiceState.class), false);
+    } else if (recordClass == MountTable.class) {
+      String src = "/" + generateRandomString();
+      Map<String, String> destMap = Collections.singletonMap(
+          generateRandomString(), "/" + generateRandomString());
+      return (T) MountTable.newInstance(src, destMap);
     }
 
     return null;
@@ -155,6 +162,7 @@ public class TestStateStoreDriverBase {
 
   public static void removeAll(StateStoreDriver driver) throws IOException {
     driver.removeAll(MembershipState.class);
+    driver.removeAll(MountTable.class);
   }
 
   public <T extends BaseRecord> void testInsert(
@@ -347,22 +355,26 @@ public class TestStateStoreDriverBase {
   public void testInsert(StateStoreDriver driver)
       throws IllegalArgumentException, IllegalAccessException, IOException {
     testInsert(driver, MembershipState.class);
+    testInsert(driver, MountTable.class);
   }
 
   public void testPut(StateStoreDriver driver)
       throws IllegalArgumentException, ReflectiveOperationException,
       IOException, SecurityException {
     testPut(driver, MembershipState.class);
+    testPut(driver, MountTable.class);
   }
 
   public void testRemove(StateStoreDriver driver)
       throws IllegalArgumentException, IllegalAccessException, IOException {
     testRemove(driver, MembershipState.class);
+    testRemove(driver, MountTable.class);
   }
 
   public void testFetchErrors(StateStoreDriver driver)
       throws IllegalArgumentException, IllegalAccessException, IOException {
     testFetchErrors(driver, MembershipState.class);
+    testFetchErrors(driver, MountTable.class);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMountTable.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMountTable.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMountTable.java
new file mode 100644
index 0000000..b6f91cf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMountTable.java
@@ -0,0 +1,176 @@
+/**
+ * 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.store.records;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+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.store.driver.StateStoreSerializer;
+import org.junit.Test;
+
+/**
+ * Test the Mount Table entry in the State Store.
+ */
+public class TestMountTable {
+
+  private static final String SRC = "/test";
+  private static final String DST_NS_0 = "ns0";
+  private static final String DST_NS_1 = "ns1";
+  private static final String DST_PATH_0 = "/path1";
+  private static final String DST_PATH_1 = "/path/path2";
+  private static final List<RemoteLocation> DST = new LinkedList<>();
+  static {
+    DST.add(new RemoteLocation(DST_NS_0, DST_PATH_0));
+    DST.add(new RemoteLocation(DST_NS_1, DST_PATH_1));
+  }
+  private static final Map<String, String> DST_MAP = new LinkedHashMap<>();
+  static {
+    DST_MAP.put(DST_NS_0, DST_PATH_0);
+    DST_MAP.put(DST_NS_1, DST_PATH_1);
+  }
+
+  private static final long DATE_CREATED = 100;
+  private static final long DATE_MOD = 200;
+
+
+  @Test
+  public void testGetterSetter() throws IOException {
+
+    MountTable record = MountTable.newInstance(SRC, DST_MAP);
+
+    validateDestinations(record);
+    assertEquals(SRC, record.getSourcePath());
+    assertEquals(DST, record.getDestinations());
+    assertTrue(DATE_CREATED > 0);
+    assertTrue(DATE_MOD > 0);
+
+    MountTable record2 =
+        MountTable.newInstance(SRC, DST_MAP, DATE_CREATED, DATE_MOD);
+
+    validateDestinations(record2);
+    assertEquals(SRC, record2.getSourcePath());
+    assertEquals(DST, record2.getDestinations());
+    assertEquals(DATE_CREATED, record2.getDateCreated());
+    assertEquals(DATE_MOD, record2.getDateModified());
+    assertFalse(record.isReadOnly());
+    assertEquals(DestinationOrder.HASH, record.getDestOrder());
+  }
+
+  @Test
+  public void testSerialization() throws IOException {
+    testSerialization(DestinationOrder.RANDOM);
+    testSerialization(DestinationOrder.HASH);
+    testSerialization(DestinationOrder.LOCAL);
+  }
+
+  private void testSerialization(final DestinationOrder order)
+      throws IOException {
+
+    MountTable record = MountTable.newInstance(
+        SRC, DST_MAP, DATE_CREATED, DATE_MOD);
+    record.setReadOnly(true);
+    record.setDestOrder(order);
+
+    StateStoreSerializer serializer = StateStoreSerializer.getSerializer();
+    String serializedString = serializer.serializeString(record);
+    MountTable record2 =
+        serializer.deserialize(serializedString, MountTable.class);
+
+    validateDestinations(record2);
+    assertEquals(SRC, record2.getSourcePath());
+    assertEquals(DST, record2.getDestinations());
+    assertEquals(DATE_CREATED, record2.getDateCreated());
+    assertEquals(DATE_MOD, record2.getDateModified());
+    assertTrue(record2.isReadOnly());
+    assertEquals(order, record2.getDestOrder());
+  }
+
+  @Test
+  public void testReadOnly() throws IOException {
+
+    Map<String, String> dest = new HashMap<>();
+    dest.put(DST_NS_0, DST_PATH_0);
+    dest.put(DST_NS_1, DST_PATH_1);
+    MountTable record1 = MountTable.newInstance(SRC, dest);
+    record1.setReadOnly(true);
+
+    validateDestinations(record1);
+    assertEquals(SRC, record1.getSourcePath());
+    assertEquals(DST, record1.getDestinations());
+    assertTrue(DATE_CREATED > 0);
+    assertTrue(DATE_MOD > 0);
+    assertTrue(record1.isReadOnly());
+
+    MountTable record2 = MountTable.newInstance(
+        SRC, DST_MAP, DATE_CREATED, DATE_MOD);
+    record2.setReadOnly(true);
+
+    validateDestinations(record2);
+    assertEquals(SRC, record2.getSourcePath());
+    assertEquals(DST, record2.getDestinations());
+    assertEquals(DATE_CREATED, record2.getDateCreated());
+    assertEquals(DATE_MOD, record2.getDateModified());
+    assertTrue(record2.isReadOnly());
+  }
+
+  @Test
+  public void testOrder() throws IOException {
+    testOrder(DestinationOrder.HASH);
+    testOrder(DestinationOrder.LOCAL);
+    testOrder(DestinationOrder.RANDOM);
+  }
+
+  private void testOrder(final DestinationOrder order)
+      throws IOException {
+
+    MountTable record = MountTable.newInstance(
+        SRC, DST_MAP, DATE_CREATED, DATE_MOD);
+    record.setDestOrder(order);
+
+    validateDestinations(record);
+    assertEquals(SRC, record.getSourcePath());
+    assertEquals(DST, record.getDestinations());
+    assertEquals(DATE_CREATED, record.getDateCreated());
+    assertEquals(DATE_MOD, record.getDateModified());
+    assertEquals(order, record.getDestOrder());
+  }
+
+  private void validateDestinations(MountTable record) {
+
+    assertEquals(SRC, record.getSourcePath());
+    assertEquals(2, record.getDestinations().size());
+
+    RemoteLocation location1 = record.getDestinations().get(0);
+    assertEquals(DST_NS_0, location1.getNameserviceId());
+    assertEquals(DST_PATH_0, location1.getDest());
+
+    RemoteLocation location2 = record.getDestinations().get(1);
+    assertEquals(DST_NS_1, location2.getNameserviceId());
+    assertEquals(DST_PATH_1, location2.getDest());
+  }
+}
\ 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/29] hadoop git commit: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 0b381a9cb0142bc40517cb2ee32534554d48424c
Parents: 2096bce
Author: Inigo Goiri <in...@apache.org>
Authored: Thu May 11 09:57:03 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   38 +
 .../resolver/FederationNamespaceInfo.java       |   46 +-
 .../federation/resolver/RemoteLocation.java     |   46 +-
 .../federation/router/ConnectionContext.java    |  104 +
 .../federation/router/ConnectionManager.java    |  408 ++++
 .../federation/router/ConnectionPool.java       |  314 +++
 .../federation/router/ConnectionPoolId.java     |  117 ++
 .../router/RemoteLocationContext.java           |   38 +-
 .../server/federation/router/RemoteMethod.java  |  164 ++
 .../server/federation/router/RemoteParam.java   |   71 +
 .../hdfs/server/federation/router/Router.java   |   58 +-
 .../federation/router/RouterRpcClient.java      |  856 ++++++++
 .../federation/router/RouterRpcServer.java      | 1867 +++++++++++++++++-
 .../src/main/resources/hdfs-default.xml         |   95 +
 .../server/federation/FederationTestUtils.java  |   80 +-
 .../hdfs/server/federation/MockResolver.java    |   90 +-
 .../server/federation/RouterConfigBuilder.java  |   20 +-
 .../server/federation/RouterDFSCluster.java     |  535 +++--
 .../server/federation/router/TestRouter.java    |   31 +-
 .../server/federation/router/TestRouterRpc.java |  869 ++++++++
 .../router/TestRouterRpcMultiDestination.java   |  216 ++
 21 files changed, 5675 insertions(+), 388 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/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 8cdd450..8b39e88 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
@@ -1117,6 +1117,44 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // HDFS Router-based federation
   public static final String FEDERATION_ROUTER_PREFIX =
       "dfs.federation.router.";
+  public static final String DFS_ROUTER_DEFAULT_NAMESERVICE =
+      FEDERATION_ROUTER_PREFIX + "default.nameserviceId";
+  public static final String DFS_ROUTER_HANDLER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "handler.count";
+  public static final int DFS_ROUTER_HANDLER_COUNT_DEFAULT = 10;
+  public static final String DFS_ROUTER_READER_QUEUE_SIZE_KEY =
+      FEDERATION_ROUTER_PREFIX + "reader.queue.size";
+  public static final int DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT = 100;
+  public static final String DFS_ROUTER_READER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "reader.count";
+  public static final int DFS_ROUTER_READER_COUNT_DEFAULT = 1;
+  public static final String DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY =
+      FEDERATION_ROUTER_PREFIX + "handler.queue.size";
+  public static final int DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT = 100;
+  public static final String DFS_ROUTER_RPC_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "rpc-bind-host";
+  public static final int DFS_ROUTER_RPC_PORT_DEFAULT = 8888;
+  public static final String DFS_ROUTER_RPC_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "rpc-address";
+  public static final String DFS_ROUTER_RPC_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_RPC_PORT_DEFAULT;
+  public static final String DFS_ROUTER_RPC_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "rpc.enable";
+  public static final boolean DFS_ROUTER_RPC_ENABLE_DEFAULT = true;
+
+  // HDFS Router NN client
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE =
+      FEDERATION_ROUTER_PREFIX + "connection.pool-size";
+  public static final int DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT =
+      64;
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN =
+      FEDERATION_ROUTER_PREFIX + "connection.pool.clean.ms";
+  public static final long DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS =
+      FEDERATION_ROUTER_PREFIX + "connection.clean.ms";
+  public static final long DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT =
+      TimeUnit.SECONDS.toMillis(10);
 
   // HDFS Router State Store connection
   public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
index bbaeca3..edcd308 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
@@ -23,15 +23,14 @@ import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
  * Represents information about a single nameservice/namespace in a federated
  * HDFS cluster.
  */
-public class FederationNamespaceInfo
-    implements Comparable<FederationNamespaceInfo>, RemoteLocationContext {
+public class FederationNamespaceInfo extends RemoteLocationContext {
 
   /** Block pool identifier. */
-  private String blockPoolId;
+  private final String blockPoolId;
   /** Cluster identifier. */
-  private String clusterId;
+  private final String clusterId;
   /** Nameservice identifier. */
-  private String nameserviceId;
+  private final String nameserviceId;
 
   public FederationNamespaceInfo(String bpId, String clId, String nsId) {
     this.blockPoolId = bpId;
@@ -39,15 +38,16 @@ public class FederationNamespaceInfo
     this.nameserviceId = nsId;
   }
 
-  /**
-   * The HDFS nameservice id for this namespace.
-   *
-   * @return Nameservice identifier.
-   */
+  @Override
   public String getNameserviceId() {
     return this.nameserviceId;
   }
 
+  @Override
+  public String getDest() {
+    return this.nameserviceId;
+  }
+
   /**
    * The HDFS cluster id for this namespace.
    *
@@ -67,33 +67,7 @@ public class FederationNamespaceInfo
   }
 
   @Override
-  public int hashCode() {
-    return this.nameserviceId.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null) {
-      return false;
-    } else if (obj instanceof FederationNamespaceInfo) {
-      return this.compareTo((FederationNamespaceInfo) obj) == 0;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int compareTo(FederationNamespaceInfo info) {
-    return this.nameserviceId.compareTo(info.getNameserviceId());
-  }
-
-  @Override
   public String toString() {
     return this.nameserviceId + "->" + this.blockPoolId + ":" + this.clusterId;
   }
-
-  @Override
-  public String getDest() {
-    return this.nameserviceId;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
index eef136d..6aa12ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
@@ -17,34 +17,51 @@
  */
 package org.apache.hadoop.hdfs.server.federation.resolver;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
 
 /**
  * A single in a remote namespace consisting of a nameservice ID
  * and a HDFS path.
  */
-public class RemoteLocation implements RemoteLocationContext {
+public class RemoteLocation extends RemoteLocationContext {
 
   /** Identifier of the remote namespace for this location. */
-  private String nameserviceId;
+  private final String nameserviceId;
+  /** Identifier of the namenode in the namespace for this location. */
+  private final String namenodeId;
   /** Path in the remote location. */
-  private String path;
+  private final String path;
 
   /**
    * Create a new remote location.
    *
+   * @param nsId
+   * @param pPath
+   */
+  public RemoteLocation(String nsId, String pPath) {
+    this(nsId, null, pPath);
+  }
+
+  /**
+   * Create a new remote location pointing to a particular namenode in the
+   * namespace.
+   *
    * @param nsId Destination namespace.
    * @param pPath Path in the destination namespace.
    */
-  public RemoteLocation(String nsId, String pPath) {
+  public RemoteLocation(String nsId, String nnId, String pPath) {
     this.nameserviceId = nsId;
+    this.namenodeId = nnId;
     this.path = pPath;
   }
 
   @Override
   public String getNameserviceId() {
-    return this.nameserviceId;
+    String ret = this.nameserviceId;
+    if (this.namenodeId != null) {
+      ret += "-" + this.namenodeId;
+    }
+    return ret;
   }
 
   @Override
@@ -54,21 +71,6 @@ public class RemoteLocation implements RemoteLocationContext {
 
   @Override
   public String toString() {
-    return this.nameserviceId + "->" + this.path;
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder(17, 31)
-        .append(this.nameserviceId)
-        .append(this.path)
-        .toHashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return (obj != null &&
-        obj.getClass() == this.getClass() &&
-        obj.hashCode() == this.hashCode());
+    return getNameserviceId() + "->" + this.path;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
new file mode 100644
index 0000000..1d27b51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
@@ -0,0 +1,104 @@
+/**
+ * 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.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.ipc.RPC;
+
+/**
+ * Context to track a connection in a {@link ConnectionPool}. When a client uses
+ * a connection, it increments a counter to mark it as active. Once the client
+ * is done with the connection, it decreases the counter. It also takes care of
+ * closing the connection once is not active.
+ */
+public class ConnectionContext {
+
+  /** Client for the connection. */
+  private final ProxyAndInfo<ClientProtocol> client;
+  /** How many threads are using this connection. */
+  private int numThreads = 0;
+  /** If the connection is closed. */
+  private boolean closed = false;
+
+
+  public ConnectionContext(ProxyAndInfo<ClientProtocol> connection) {
+    this.client = connection;
+  }
+
+  /**
+   * Check if the connection is active.
+   *
+   * @return True if the connection is active.
+   */
+  public synchronized boolean isActive() {
+    return this.numThreads > 0;
+  }
+
+  /**
+   * Check if the connection is closed.
+   *
+   * @return If the connection is closed.
+   */
+  public synchronized boolean isClosed() {
+    return this.closed;
+  }
+
+  /**
+   * Check if the connection can be used. It checks if the connection is used by
+   * another thread or already closed.
+   *
+   * @return True if the connection can be used.
+   */
+  public synchronized boolean isUsable() {
+    return !isActive() && !isClosed();
+  }
+
+  /**
+   * Get the connection client.
+   *
+   * @return Connection client.
+   */
+  public synchronized ProxyAndInfo<ClientProtocol> getClient() {
+    this.numThreads++;
+    return this.client;
+  }
+
+  /**
+   * Release this connection. If the connection was closed, close the proxy.
+   * Otherwise, mark the connection as not used by us anymore.
+   */
+  public synchronized void release() {
+    if (--this.numThreads == 0 && this.closed) {
+      close();
+    }
+  }
+
+  /**
+   * We will not use this connection anymore. If it's not being used, we close
+   * it. Otherwise, we let release() do it once we are done with it.
+   */
+  public synchronized void close() {
+    this.closed = true;
+    if (this.numThreads == 0) {
+      ClientProtocol proxy = this.client.getProxy();
+      // Nobody should be using this anymore so it should close right away
+      RPC.stopProxy(proxy);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
new file mode 100644
index 0000000..d93d498
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
@@ -0,0 +1,408 @@
+/**
+ * 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 java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements a pool of connections for the {@link Router} to be able to open
+ * many connections to many Namenodes.
+ */
+public class ConnectionManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectionManager.class);
+
+  /** Number of parallel new connections to create. */
+  protected static final int MAX_NEW_CONNECTIONS = 100;
+
+  /** Minimum amount of active connections: 50%. */
+  protected static final float MIN_ACTIVE_RATIO = 0.5f;
+
+
+  /** Configuration for the connection manager, pool and sockets. */
+  private final Configuration conf;
+
+  /** Min number of connections per user + nn. */
+  private final int minSize = 1;
+  /** Max number of connections per user + nn. */
+  private final int maxSize;
+
+  /** How often we close a pool for a particular user + nn. */
+  private final long poolCleanupPeriodMs;
+  /** How often we close a connection in a pool. */
+  private final long connectionCleanupPeriodMs;
+
+  /** Map of connection pools, one pool per user + NN. */
+  private final Map<ConnectionPoolId, ConnectionPool> pools;
+  /** Lock for accessing pools. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  /** Queue for creating new connections. */
+  private final BlockingQueue<ConnectionPool> creatorQueue =
+      new ArrayBlockingQueue<>(MAX_NEW_CONNECTIONS);
+  /** Create new connections asynchronously. */
+  private final ConnectionCreator creator;
+  /** Periodic executor to remove stale connection pools. */
+  private final ScheduledThreadPoolExecutor cleaner =
+      new ScheduledThreadPoolExecutor(1);
+
+  /** If the connection manager is running. */
+  private boolean running = false;
+
+
+  /**
+   * Creates a proxy client connection pool manager.
+   *
+   * @param config Configuration for the connections.
+   * @param minPoolSize Min size of the connection pool.
+   * @param maxPoolSize Max size of the connection pool.
+   */
+  public ConnectionManager(Configuration config) {
+    this.conf = config;
+
+    // Configure minimum and maximum connection pools
+    this.maxSize = this.conf.getInt(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT);
+
+    // Map with the connections indexed by UGI and Namenode
+    this.pools = new HashMap<>();
+
+    // Create connections in a thread asynchronously
+    this.creator = new ConnectionCreator(creatorQueue);
+    this.creator.setDaemon(true);
+
+    // Cleanup periods
+    this.poolCleanupPeriodMs = this.conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN_DEFAULT);
+    LOG.info("Cleaning connection pools every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(this.poolCleanupPeriodMs));
+    this.connectionCleanupPeriodMs = this.conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT);
+    LOG.info("Cleaning connections every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(this.connectionCleanupPeriodMs));
+  }
+
+  /**
+   * Start the connection manager.
+   */
+  public void start() {
+    // Start the thread that creates connections asynchronously
+    this.creator.start();
+
+    // Schedule a task to remove stale connection pools and sockets
+    long recyleTimeMs = Math.min(
+        poolCleanupPeriodMs, connectionCleanupPeriodMs);
+    LOG.info("Cleaning every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(recyleTimeMs));
+    this.cleaner.scheduleAtFixedRate(
+        new CleanupTask(), 0, recyleTimeMs, TimeUnit.MILLISECONDS);
+
+    // Mark the manager as running
+    this.running = true;
+  }
+
+  /**
+   * Stop the connection manager by closing all the pools.
+   */
+  public void close() {
+    this.creator.shutdown();
+    this.cleaner.shutdown();
+    this.running = false;
+
+    writeLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        pool.close();
+      }
+      this.pools.clear();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Fetches the next available proxy client in the pool. Each client connection
+   * is reserved for a single user and cannot be reused until free.
+   *
+   * @param ugi User group information.
+   * @param nnAddress Namenode address for the connection.
+   * @return Proxy client to connect to nnId as UGI.
+   * @throws IOException If the connection cannot be obtained.
+   */
+  public ConnectionContext getConnection(
+      UserGroupInformation ugi, String nnAddress) throws IOException {
+
+    // Check if the manager is shutdown
+    if (!this.running) {
+      LOG.error(
+          "Cannot get a connection to {} because the manager isn't running",
+          nnAddress);
+      return null;
+    }
+
+    // Try to get the pool if created
+    ConnectionPoolId connectionId = new ConnectionPoolId(ugi, nnAddress);
+    ConnectionPool pool = null;
+    readLock.lock();
+    try {
+      pool = this.pools.get(connectionId);
+    } finally {
+      readLock.unlock();
+    }
+
+    // Create the pool if not created before
+    if (pool == null) {
+      writeLock.lock();
+      try {
+        pool = this.pools.get(connectionId);
+        if (pool == null) {
+          pool = new ConnectionPool(
+              this.conf, nnAddress, ugi, this.minSize, this.maxSize);
+          this.pools.put(connectionId, pool);
+        }
+      } finally {
+        writeLock.unlock();
+      }
+    }
+
+    ConnectionContext conn = pool.getConnection();
+
+    // Add a new connection to the pool if it wasn't usable
+    if (conn == null || !conn.isUsable()) {
+      if (!this.creatorQueue.offer(pool)) {
+        LOG.error("Cannot add more than {} connections at the same time",
+            MAX_NEW_CONNECTIONS);
+      }
+    }
+
+    if (conn != null && conn.isClosed()) {
+      LOG.error("We got a closed connection from {}", pool);
+      conn = null;
+    }
+
+    return conn;
+  }
+
+  /**
+   * Get the number of connection pools.
+   *
+   * @return Number of connection pools.
+   */
+  public int getNumConnectionPools() {
+    readLock.lock();
+    try {
+      return pools.size();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get number of open connections.
+   *
+   * @return Number of open connections.
+   */
+  public int getNumConnections() {
+    int total = 0;
+    readLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        total += pool.getNumConnections();
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return total;
+  }
+
+  /**
+   * Get number of active connections.
+   *
+   * @return Number of active connections.
+   */
+  public int getNumActiveConnections() {
+    int total = 0;
+    readLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        total += pool.getNumActiveConnections();
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return total;
+  }
+
+  /**
+   * Get the number of connections to be created.
+   *
+   * @return Number of connections to be created.
+   */
+  public int getNumCreatingConnections() {
+    return this.creatorQueue.size();
+  }
+
+  /**
+   * Removes stale connections not accessed recently from the pool. This is
+   * invoked periodically.
+   */
+  private class CleanupTask implements Runnable {
+
+    @Override
+    public void run() {
+      long currentTime = Time.now();
+      List<ConnectionPoolId> toRemove = new LinkedList<>();
+
+      // Look for stale pools
+      readLock.lock();
+      try {
+        for (Entry<ConnectionPoolId, ConnectionPool> entry : pools.entrySet()) {
+          ConnectionPool pool = entry.getValue();
+          long lastTimeActive = pool.getLastActiveTime();
+          boolean isStale =
+              currentTime > (lastTimeActive + poolCleanupPeriodMs);
+          if (lastTimeActive > 0 && isStale) {
+            // Remove this pool
+            LOG.debug("Closing and removing stale pool {}", pool);
+            pool.close();
+            ConnectionPoolId poolId = entry.getKey();
+            toRemove.add(poolId);
+          } else {
+            // Keep this pool but clean connections inside
+            LOG.debug("Cleaning up {}", pool);
+            cleanup(pool);
+          }
+        }
+      } finally {
+        readLock.unlock();
+      }
+
+      // Remove stale pools
+      if (!toRemove.isEmpty()) {
+        writeLock.lock();
+        try {
+          for (ConnectionPoolId poolId : toRemove) {
+            pools.remove(poolId);
+          }
+        } finally {
+          writeLock.unlock();
+        }
+      }
+    }
+
+    /**
+     * Clean the unused connections for this pool.
+     *
+     * @param pool Connection pool to cleanup.
+     */
+    private void cleanup(ConnectionPool pool) {
+      if (pool.getNumConnections() > pool.getMinSize()) {
+        // Check if the pool hasn't been active in a while or not 50% are used
+        long timeSinceLastActive = Time.now() - pool.getLastActiveTime();
+        int total = pool.getNumConnections();
+        int active = getNumActiveConnections();
+        if (timeSinceLastActive > connectionCleanupPeriodMs ||
+            active < MIN_ACTIVE_RATIO * total) {
+          // Remove and close 1 connection
+          List<ConnectionContext> conns = pool.removeConnections(1);
+          for (ConnectionContext conn : conns) {
+            conn.close();
+          }
+          LOG.debug("Removed connection {} used {} seconds ago. " +
+              "Pool has {}/{} connections", pool.getConnectionPoolId(),
+              TimeUnit.MILLISECONDS.toSeconds(timeSinceLastActive),
+              pool.getNumConnections(), pool.getMaxSize());
+        }
+      }
+    }
+  }
+
+  /**
+   * Thread that creates connections asynchronously.
+   */
+  private static class ConnectionCreator extends Thread {
+    /** If the creator is running. */
+    private boolean running = true;
+    /** Queue to push work to. */
+    private BlockingQueue<ConnectionPool> queue;
+
+    ConnectionCreator(BlockingQueue<ConnectionPool> blockingQueue) {
+      super("Connection creator");
+      this.queue = blockingQueue;
+    }
+
+    @Override
+    public void run() {
+      while (this.running) {
+        try {
+          ConnectionPool pool = this.queue.take();
+          try {
+            int total = pool.getNumConnections();
+            int active = pool.getNumActiveConnections();
+            if (pool.getNumConnections() < pool.getMaxSize() &&
+                active >= MIN_ACTIVE_RATIO * total) {
+              ConnectionContext conn = pool.newConnection();
+              pool.addConnection(conn);
+            } else {
+              LOG.debug("Cannot add more than {} connections to {}",
+                  pool.getMaxSize(), pool);
+            }
+          } catch (IOException e) {
+            LOG.error("Cannot create a new connection", e);
+          }
+        } catch (InterruptedException e) {
+          LOG.error("The connection creator was interrupted");
+          this.running = false;
+        }
+      }
+    }
+
+    /**
+     * Stop this connection creator.
+     */
+    public void shutdown() {
+      this.running = false;
+      this.interrupt();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
new file mode 100644
index 0000000..f76f621
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
@@ -0,0 +1,314 @@
+/**
+ * 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 java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+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.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryUtils;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maintains a pool of connections for each User (including tokens) + NN. The
+ * RPC client maintains a single socket, to achieve throughput similar to a NN,
+ * each request is multiplexed across multiple sockets/connections from a
+ * pool.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ConnectionPool {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectionPool.class);
+
+
+  /** Configuration settings for the connection pool. */
+  private final Configuration conf;
+
+  /** Identifier for this connection pool. */
+  private final ConnectionPoolId connectionPoolId;
+  /** Namenode this pool connects to. */
+  private final String namenodeAddress;
+  /** User for this connections. */
+  private final UserGroupInformation ugi;
+
+  /** Pool of connections. We mimic a COW array. */
+  private volatile List<ConnectionContext> connections = new ArrayList<>();
+  /** Connection index for round-robin. */
+  private final AtomicInteger clientIndex = new AtomicInteger(0);
+
+  /** Min number of connections per user. */
+  private final int minSize;
+  /** Max number of connections per user. */
+  private final int maxSize;
+
+  /** The last time a connection was active. */
+  private volatile long lastActiveTime = 0;
+
+
+  protected ConnectionPool(Configuration config, String address,
+      UserGroupInformation user, int minPoolSize, int maxPoolSize)
+          throws IOException {
+
+    this.conf = config;
+
+    // Connection pool target
+    this.ugi = user;
+    this.namenodeAddress = address;
+    this.connectionPoolId =
+        new ConnectionPoolId(this.ugi, this.namenodeAddress);
+
+    // Set configuration parameters for the pool
+    this.minSize = minPoolSize;
+    this.maxSize = maxPoolSize;
+
+    // Add minimum connections to the pool
+    for (int i=0; i<this.minSize; i++) {
+      ConnectionContext newConnection = newConnection();
+      this.connections.add(newConnection);
+    }
+    LOG.debug("Created connection pool \"{}\" with {} connections",
+        this.connectionPoolId, this.minSize);
+  }
+
+  /**
+   * Get the maximum number of connections allowed in this pool.
+   *
+   * @return Maximum number of connections.
+   */
+  protected int getMaxSize() {
+    return this.maxSize;
+  }
+
+  /**
+   * Get the minimum number of connections in this pool.
+   *
+   * @return Minimum number of connections.
+   */
+  protected int getMinSize() {
+    return this.minSize;
+  }
+
+  /**
+   * Get the connection pool identifier.
+   *
+   * @return Connection pool identifier.
+   */
+  protected ConnectionPoolId getConnectionPoolId() {
+    return this.connectionPoolId;
+  }
+
+  /**
+   * Return the next connection round-robin.
+   *
+   * @return Connection context.
+   */
+  protected ConnectionContext getConnection() {
+
+    this.lastActiveTime = Time.now();
+
+    // Get a connection from the pool following round-robin
+    ConnectionContext conn = null;
+    List<ConnectionContext> tmpConnections = this.connections;
+    int size = tmpConnections.size();
+    int threadIndex = this.clientIndex.getAndIncrement();
+    for (int i=0; i<size; i++) {
+      int index = (threadIndex + i) % size;
+      conn = tmpConnections.get(index);
+      if (conn != null && !conn.isUsable()) {
+        return conn;
+      }
+    }
+
+    // We return a connection even if it's active
+    return conn;
+  }
+
+  /**
+   * Add a connection to the current pool. It uses a Copy-On-Write approach.
+   *
+   * @param conns New connections to add to the pool.
+   */
+  public synchronized void addConnection(ConnectionContext conn) {
+    List<ConnectionContext> tmpConnections = new ArrayList<>(this.connections);
+    tmpConnections.add(conn);
+    this.connections = tmpConnections;
+
+    this.lastActiveTime = Time.now();
+  }
+
+  /**
+   * Remove connections from the current pool.
+   *
+   * @param num Number of connections to remove.
+   * @return Removed connections.
+   */
+  public synchronized List<ConnectionContext> removeConnections(int num) {
+    List<ConnectionContext> removed = new LinkedList<>();
+
+    // Remove and close the last connection
+    List<ConnectionContext> tmpConnections = new ArrayList<>();
+    for (int i=0; i<this.connections.size(); i++) {
+      ConnectionContext conn = this.connections.get(i);
+      if (i < this.minSize || i < this.connections.size() - num) {
+        tmpConnections.add(conn);
+      } else {
+        removed.add(conn);
+      }
+    }
+    this.connections = tmpConnections;
+
+    return removed;
+  }
+
+  /**
+   * Close the connection pool.
+   */
+  protected synchronized void close() {
+    long timeSinceLastActive = TimeUnit.MILLISECONDS.toSeconds(
+        Time.now() - getLastActiveTime());
+    LOG.debug("Shutting down connection pool \"{}\" used {} seconds ago",
+        this.connectionPoolId, timeSinceLastActive);
+
+    for (ConnectionContext connection : this.connections) {
+      connection.close();
+    }
+    this.connections.clear();
+  }
+
+  /**
+   * Number of connections in the pool.
+   *
+   * @return Number of connections.
+   */
+  protected int getNumConnections() {
+    return this.connections.size();
+  }
+
+  /**
+   * Number of active connections in the pool.
+   *
+   * @return Number of active connections.
+   */
+  protected int getNumActiveConnections() {
+    int ret = 0;
+
+    List<ConnectionContext> tmpConnections = this.connections;
+    for (ConnectionContext conn : tmpConnections) {
+      if (conn.isActive()) {
+        ret++;
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Get the last time the connection pool was used.
+   *
+   * @return Last time the connection pool was used.
+   */
+  protected long getLastActiveTime() {
+    return this.lastActiveTime;
+  }
+
+  @Override
+  public String toString() {
+    return this.connectionPoolId.toString();
+  }
+
+  /**
+   * Create a new proxy wrapper for a client NN connection.
+   * @return Proxy for the target ClientProtocol that contains the user's
+   *         security context.
+   * @throws IOException
+   */
+  public ConnectionContext newConnection() throws IOException {
+    return newConnection(this.conf, this.namenodeAddress, this.ugi);
+  }
+
+  /**
+   * Creates a proxy wrapper for a client NN connection. Each proxy contains
+   * context for a single user/security context. To maximize throughput it is
+   * recommended to use multiple connection per user+server, allowing multiple
+   * writes and reads to be dispatched in parallel.
+   *
+   * @param conf Configuration for the connection.
+   * @param nnAddress Address of server supporting the ClientProtocol.
+   * @param ugi User context.
+   * @return Proxy for the target ClientProtocol that contains the user's
+   *         security context.
+   * @throws IOException If it cannot be created.
+   */
+  protected static ConnectionContext newConnection(Configuration conf,
+      String nnAddress, UserGroupInformation ugi)
+          throws IOException {
+    RPC.setProtocolEngine(
+        conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine.class);
+
+    final RetryPolicy defaultPolicy = RetryUtils.getDefaultRetryPolicy(
+        conf,
+        HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY,
+        HdfsClientConfigKeys.Retry.POLICY_ENABLED_DEFAULT,
+        HdfsClientConfigKeys.Retry.POLICY_SPEC_KEY,
+        HdfsClientConfigKeys.Retry.POLICY_SPEC_DEFAULT,
+        HdfsConstants.SAFEMODE_EXCEPTION_CLASS_NAME);
+
+    SocketFactory factory = SocketFactory.getDefault();
+    if (UserGroupInformation.isSecurityEnabled()) {
+      SaslRpcServer.init(conf);
+    }
+    InetSocketAddress socket = NetUtils.createSocketAddr(nnAddress);
+    final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
+    ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
+        ClientNamenodeProtocolPB.class, version, socket, ugi, conf,
+        factory, RPC.getRpcTimeout(conf), defaultPolicy, null).getProxy();
+    ClientProtocol client = new ClientNamenodeProtocolTranslatorPB(proxy);
+    Text dtService = SecurityUtil.buildTokenService(socket);
+
+    ProxyAndInfo<ClientProtocol> clientProxy =
+        new ProxyAndInfo<ClientProtocol>(client, dtService, socket);
+    ConnectionContext connection = new ConnectionContext(clientProxy);
+    return connection;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
new file mode 100644
index 0000000..a3a78de
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * Identifier for a connection for a user to a namenode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ConnectionPoolId implements Comparable<ConnectionPoolId> {
+
+  /** Namenode identifier. */
+  private final String nnId;
+  /** Information about the user. */
+  private final UserGroupInformation ugi;
+
+  /**
+   * New connection pool identifier.
+   *
+   * @param ugi Information of the user issuing the request.
+   * @param nnId Namenode address with port.
+   */
+  public ConnectionPoolId(final UserGroupInformation ugi, final String nnId) {
+    this.nnId = nnId;
+    this.ugi = ugi;
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = new HashCodeBuilder(17, 31)
+        .append(this.nnId)
+        .append(this.ugi.toString())
+        .append(this.getTokenIds())
+        .toHashCode();
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ConnectionPoolId) {
+      ConnectionPoolId other = (ConnectionPoolId) o;
+      if (!this.nnId.equals(other.nnId)) {
+        return false;
+      }
+      if (!this.ugi.toString().equals(other.ugi.toString())) {
+        return false;
+      }
+      String thisTokens = this.getTokenIds().toString();
+      String otherTokens = other.getTokenIds().toString();
+      return thisTokens.equals(otherTokens);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return this.ugi + " " + this.getTokenIds() + "->" + this.nnId;
+  }
+
+  @Override
+  public int compareTo(ConnectionPoolId other) {
+    int ret = this.nnId.compareTo(other.nnId);
+    if (ret == 0) {
+      ret = this.ugi.toString().compareTo(other.ugi.toString());
+    }
+    if (ret == 0) {
+      String thisTokens = this.getTokenIds().toString();
+      String otherTokens = other.getTokenIds().toString();
+      ret = thisTokens.compareTo(otherTokens);
+    }
+    return ret;
+  }
+
+  /**
+   * Get the token identifiers for this connection.
+   * @return List with the token identifiers.
+   */
+  private List<String> getTokenIds() {
+    List<String> tokenIds = new ArrayList<>();
+    Collection<Token<? extends TokenIdentifier>> tokens = this.ugi.getTokens();
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      byte[] tokenIdBytes = token.getIdentifier();
+      String tokenId = Arrays.toString(tokenIdBytes);
+      tokenIds.add(tokenId);
+    }
+    Collections.sort(tokenIds);
+    return tokenIds;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
index da6066b..a90c460 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
@@ -17,22 +17,52 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
 /**
- * Interface for objects that are unique to a namespace.
+ * Base class for objects that are unique to a namespace.
  */
-public interface RemoteLocationContext {
+public abstract class RemoteLocationContext
+    implements Comparable<RemoteLocationContext> {
 
   /**
    * Returns an identifier for a unique namespace.
    *
    * @return Namespace identifier.
    */
-  String getNameserviceId();
+  public abstract String getNameserviceId();
 
   /**
    * Destination in this location. For example the path in a remote namespace.
    *
    * @return Destination in this location.
    */
-  String getDest();
+  public abstract String getDest();
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 31)
+        .append(getNameserviceId())
+        .append(getDest())
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof RemoteLocationContext) {
+      RemoteLocationContext other = (RemoteLocationContext) obj;
+      return this.getNameserviceId().equals(other.getNameserviceId()) &&
+          this.getDest().equals(other.getDest());
+    }
+    return false;
+  }
+
+  @Override
+  public int compareTo(RemoteLocationContext info) {
+    int ret = this.getNameserviceId().compareTo(info.getNameserviceId());
+    if (ret == 0) {
+      ret = this.getDest().compareTo(info.getDest());
+    }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
new file mode 100644
index 0000000..cd57d45
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
@@ -0,0 +1,164 @@
+/**
+ * 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 java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Arrays;
+
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Determines the remote client protocol method and the parameter list for a
+ * specific location.
+ */
+public class RemoteMethod {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteMethod.class);
+
+
+  /** List of parameters: static and dynamic values, matchings types. */
+  private final Object[] params;
+  /** List of method parameters types, matches parameters. */
+  private final Class<?>[] types;
+  /** String name of the ClientProtocol method. */
+  private final String methodName;
+
+  /**
+   * Create a method with no parameters.
+   *
+   * @param method The string name of the ClientProtocol method.
+   */
+  public RemoteMethod(String method) {
+    this.params = null;
+    this.types = null;
+    this.methodName = method;
+  }
+
+  /**
+   * Creates a remote method generator.
+   *
+   * @param method The string name of the ClientProtocol method.
+   * @param pTypes A list of types to use to locate the specific method.
+   * @param pParams A list of parameters for the method. The order of the
+   *          parameter list must match the order and number of the types.
+   *          Parameters are grouped into 2 categories:
+   *          <ul>
+   *          <li>Static parameters that are immutable across locations.
+   *          <li>Dynamic parameters that are determined for each location by a
+   *          RemoteParam object. To specify a dynamic parameter, pass an
+   *          instance of RemoteParam in place of the parameter value.
+   *          </ul>
+   * @throws IOException If the types and parameter lists are not valid.
+   */
+  public RemoteMethod(String method, Class<?>[] pTypes, Object... pParams)
+      throws IOException {
+
+    if (pParams.length != pTypes.length) {
+      throw new IOException("Invalid parameters for method " + method);
+    }
+
+    this.params = pParams;
+    this.types = pTypes;
+    this.methodName = method;
+  }
+
+  /**
+   * Get the represented java method.
+   *
+   * @return Method
+   * @throws IOException If the method cannot be found.
+   */
+  public Method getMethod() throws IOException {
+    try {
+      if (types != null) {
+        return ClientProtocol.class.getDeclaredMethod(methodName, types);
+      } else {
+        return ClientProtocol.class.getDeclaredMethod(methodName);
+      }
+    } catch (NoSuchMethodException e) {
+      // Re-throw as an IOException
+      LOG.error("Cannot get method {} with types {}",
+          methodName, Arrays.toString(types), e);
+      throw new IOException(e);
+    } catch (SecurityException e) {
+      LOG.error("Cannot access method {} with types {}",
+          methodName, Arrays.toString(types), e);
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Get the calling types for this method.
+   *
+   * @return An array of calling types.
+   */
+  public Class<?>[] getTypes() {
+    return this.types;
+  }
+
+  /**
+   * Generate a list of parameters for this specific location using no context.
+   *
+   * @return A list of parameters for the method customized for the location.
+   */
+  public Object[] getParams() {
+    return this.getParams(null);
+  }
+
+  /**
+   * Get the name of the method.
+   *
+   * @return Name of the method.
+   */
+  public String getMethodName() {
+    return this.methodName;
+  }
+
+  /**
+   * Generate a list of parameters for this specific location. Parameters are
+   * grouped into 2 categories:
+   * <ul>
+   * <li>Static parameters that are immutable across locations.
+   * <li>Dynamic parameters that are determined for each location by a
+   * RemoteParam object.
+   * </ul>
+   *
+   * @param context The context identifying the location.
+   * @return A list of parameters for the method customized for the location.
+   */
+  public Object[] getParams(RemoteLocationContext context) {
+    if (this.params == null) {
+      return new Object[] {};
+    }
+    Object[] objList = new Object[this.params.length];
+    for (int i = 0; i < this.params.length; i++) {
+      Object currentObj = this.params[i];
+      if (currentObj instanceof RemoteParam) {
+        // Map the parameter using the context
+        RemoteParam paramGetter = (RemoteParam) currentObj;
+        objList[i] = paramGetter.getParameterForContext(context);
+      } else {
+        objList[i] = currentObj;
+      }
+    }
+    return objList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
new file mode 100644
index 0000000..8816ff6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
@@ -0,0 +1,71 @@
+/**
+ * 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 java.util.Map;
+
+/**
+ * A dynamically assignable parameter that is location-specific.
+ * <p>
+ * There are 2 ways this mapping is determined:
+ * <ul>
+ * <li>Default: Uses the RemoteLocationContext's destination
+ * <li>Map: Uses the value of the RemoteLocationContext key provided in the
+ * parameter map.
+ * </ul>
+ */
+public class RemoteParam {
+
+  private final Map<? extends Object, ? extends Object> paramMap;
+
+  /**
+   * Constructs a default remote parameter. Always maps the value to the
+   * destination of the provided RemoveLocationContext.
+   */
+  public RemoteParam() {
+    this.paramMap = null;
+  }
+
+  /**
+   * Constructs a map based remote parameter. Determines the value using the
+   * provided RemoteLocationContext as a key into the map.
+   *
+   * @param map Map with RemoteLocationContext keys.
+   */
+  public RemoteParam(
+      Map<? extends RemoteLocationContext, ? extends Object> map) {
+    this.paramMap = map;
+  }
+
+  /**
+   * Determine the appropriate value for this parameter based on the location.
+   *
+   * @param context Context identifying the location.
+   * @return A parameter specific to this location.
+   */
+  public Object getParameterForContext(RemoteLocationContext context) {
+    if (context == null) {
+      return null;
+    } else if (this.paramMap != null) {
+      return this.paramMap.get(context);
+    } else {
+      // Default case
+      return context.getDest();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index fe0d02a..019a5cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -22,12 +22,14 @@ import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.new
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
@@ -36,6 +38,8 @@ import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Router that provides a unified view of multiple federated HDFS clusters. It
@@ -60,7 +64,7 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceStability.Evolving
 public class Router extends CompositeService {
 
-  private static final Log LOG = LogFactory.getLog(Router.class);
+  private static final Logger LOG = LoggerFactory.getLogger(Router.class);
 
 
   /** Configuration for the Router. */
@@ -71,6 +75,7 @@ public class Router extends CompositeService {
 
   /** RPC interface to the client. */
   private RouterRpcServer rpcServer;
+  private InetSocketAddress rpcAddress;
 
   /** Interface with the State Store. */
   private StateStoreService stateStore;
@@ -105,9 +110,6 @@ public class Router extends CompositeService {
   protected void serviceInit(Configuration configuration) throws Exception {
     this.conf = configuration;
 
-    // TODO Interface to the State Store
-    this.stateStore = null;
-
     // Resolver to track active NNs
     this.namenodeResolver = newActiveNamenodeResolver(
         this.conf, this.stateStore);
@@ -122,6 +124,15 @@ public class Router extends CompositeService {
       throw new IOException("Cannot find subcluster resolver");
     }
 
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_RPC_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_RPC_ENABLE_DEFAULT)) {
+      // Create RPC server
+      this.rpcServer = createRpcServer();
+      addService(this.rpcServer);
+      this.setRpcServerAddress(rpcServer.getRpcAddress());
+    }
+
     super.serviceInit(conf);
   }
 
@@ -171,11 +182,13 @@ public class Router extends CompositeService {
       router.init(conf);
       router.start();
     } catch (Throwable e) {
-      LOG.error("Failed to start router.", e);
+      LOG.error("Failed to start router", e);
       terminate(1, e);
     }
   }
 
+
+
   /////////////////////////////////////////////////////////
   // RPC Server
   /////////////////////////////////////////////////////////
@@ -183,7 +196,7 @@ public class Router extends CompositeService {
   /**
    * Create a new Router RPC server to proxy ClientProtocol requests.
    *
-   * @return RouterRpcServer
+   * @return New Router RPC Server.
    * @throws IOException If the router RPC server was not started.
    */
   protected RouterRpcServer createRpcServer() throws IOException {
@@ -200,6 +213,35 @@ public class Router extends CompositeService {
     return this.rpcServer;
   }
 
+  /**
+   * Set the current RPC socket for the router.
+   *
+   * @param rpcAddress RPC address.
+   */
+  protected void setRpcServerAddress(InetSocketAddress address) {
+    this.rpcAddress = address;
+
+    // Use the RPC address as our unique router Id
+    if (this.rpcAddress != null) {
+      try {
+        String hostname = InetAddress.getLocalHost().getHostName();
+        setRouterId(hostname + ":" + this.rpcAddress.getPort());
+      } catch (UnknownHostException ex) {
+        LOG.error("Cannot set unique router ID, address not resolvable {}",
+            this.rpcAddress);
+      }
+    }
+  }
+
+  /**
+   * Get the current RPC socket address for the router.
+   *
+   * @return InetSocketAddress
+   */
+  public InetSocketAddress getRpcServerAddress() {
+    return this.rpcAddress;
+  }
+
   /////////////////////////////////////////////////////////
   // Submodule getters
   /////////////////////////////////////////////////////////


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


[19/29] hadoop git commit: HDFS-10646. Federation admin tool. Contributed by Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdmin.java
new file mode 100644
index 0000000..170247f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdmin.java
@@ -0,0 +1,261 @@
+/**
+ * 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.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.synchronizeRecords;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+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.resolver.MountTableManager;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+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.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.util.Time;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The administrator interface of the {@link Router} implemented by
+ * {@link RouterAdminServer}.
+ */
+public class TestRouterAdmin {
+
+  private static StateStoreDFSCluster cluster;
+  private static RouterContext routerContext;
+  public static final String RPC_BEAN =
+      "Hadoop:service=Router,name=FederationRPC";
+  private static List<MountTable> mockMountTable;
+  private static StateStoreService stateStore;
+
+  @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);
+    cluster.startRouters();
+    routerContext = cluster.getRandomRouter();
+    mockMountTable = cluster.generateMockMountTable();
+    Router router = routerContext.getRouter();
+    stateStore = router.getStateStore();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    cluster.stopRouter(routerContext);
+  }
+
+  @Before
+  public void testSetup() throws Exception {
+    assertTrue(
+        synchronizeRecords(stateStore, mockMountTable, MountTable.class));
+  }
+
+  @Test
+  public void testAddMountTable() throws IOException {
+    MountTable newEntry = MountTable.newInstance(
+        "/testpath", Collections.singletonMap("ns0", "/testdir"),
+        Time.now(), Time.now());
+
+    RouterClient client = routerContext.getAdminClient();
+    MountTableManager mountTable = client.getMountTableManager();
+
+    // Existing mount table size
+    List<MountTable> records = getMountTableEntries(mountTable);
+    assertEquals(records.size(), mockMountTable.size());
+
+    // Add
+    AddMountTableEntryRequest addRequest =
+        AddMountTableEntryRequest.newInstance(newEntry);
+    AddMountTableEntryResponse addResponse =
+        mountTable.addMountTableEntry(addRequest);
+    assertTrue(addResponse.getStatus());
+
+    // New mount table size
+    List<MountTable> records2 = getMountTableEntries(mountTable);
+    assertEquals(records2.size(), mockMountTable.size() + 1);
+  }
+
+  @Test
+  public void testAddDuplicateMountTable() throws IOException {
+    MountTable newEntry = MountTable.newInstance("/testpath",
+        Collections.singletonMap("ns0", "/testdir"), Time.now(), Time.now());
+
+    RouterClient client = routerContext.getAdminClient();
+    MountTableManager mountTable = client.getMountTableManager();
+
+    // Existing mount table size
+    List<MountTable> entries1 = getMountTableEntries(mountTable);
+    assertEquals(entries1.size(), mockMountTable.size());
+
+    // Add
+    AddMountTableEntryRequest addRequest =
+        AddMountTableEntryRequest.newInstance(newEntry);
+    AddMountTableEntryResponse addResponse =
+        mountTable.addMountTableEntry(addRequest);
+    assertTrue(addResponse.getStatus());
+
+    // New mount table size
+    List<MountTable> entries2 = getMountTableEntries(mountTable);
+    assertEquals(entries2.size(), mockMountTable.size() + 1);
+
+    // Add again, should fail
+    AddMountTableEntryResponse addResponse2 =
+        mountTable.addMountTableEntry(addRequest);
+    assertFalse(addResponse2.getStatus());
+  }
+
+  @Test
+  public void testRemoveMountTable() throws IOException {
+
+    RouterClient client = routerContext.getAdminClient();
+    MountTableManager mountTable = client.getMountTableManager();
+
+    // Existing mount table size
+    List<MountTable> entries1 = getMountTableEntries(mountTable);
+    assertEquals(entries1.size(), mockMountTable.size());
+
+    // Remove an entry
+    RemoveMountTableEntryRequest removeRequest =
+        RemoveMountTableEntryRequest.newInstance("/");
+    mountTable.removeMountTableEntry(removeRequest);
+
+    // New mount table size
+    List<MountTable> entries2 = getMountTableEntries(mountTable);
+    assertEquals(entries2.size(), mockMountTable.size() - 1);
+  }
+
+  @Test
+  public void testEditMountTable() throws IOException {
+
+    RouterClient client = routerContext.getAdminClient();
+    MountTableManager mountTable = client.getMountTableManager();
+
+    // Verify starting condition
+    MountTable entry = getMountTableEntry("/");
+    assertEquals(
+        Collections.singletonList(new RemoteLocation("ns0", "/")),
+        entry.getDestinations());
+
+    // Edit the entry for /
+    MountTable updatedEntry = MountTable.newInstance(
+        "/", Collections.singletonMap("ns1", "/"), Time.now(), Time.now());
+    UpdateMountTableEntryRequest updateRequest =
+        UpdateMountTableEntryRequest.newInstance(updatedEntry);
+    mountTable.updateMountTableEntry(updateRequest);
+
+    // Verify edited condition
+    entry = getMountTableEntry("/");
+    assertEquals(
+        Collections.singletonList(new RemoteLocation("ns1", "/")),
+        entry.getDestinations());
+  }
+
+  @Test
+  public void testGetMountTable() throws IOException {
+
+    RouterClient client = routerContext.getAdminClient();
+    MountTableManager mountTable = client.getMountTableManager();
+
+    // Verify size of table
+    List<MountTable> entries = getMountTableEntries(mountTable);
+    assertEquals(mockMountTable.size(), entries.size());
+
+    // Verify all entries are present
+    int matches = 0;
+    for (MountTable e : entries) {
+      for (MountTable entry : mockMountTable) {
+        assertEquals(e.getDestinations().size(), 1);
+        assertNotNull(e.getDateCreated());
+        assertNotNull(e.getDateModified());
+        if (entry.getSourcePath().equals(e.getSourcePath())) {
+          matches++;
+        }
+      }
+    }
+    assertEquals(matches, mockMountTable.size());
+  }
+
+  @Test
+  public void testGetSingleMountTableEntry() throws IOException {
+    MountTable entry = getMountTableEntry("/ns0");
+    assertNotNull(entry);
+    assertEquals(entry.getSourcePath(), "/ns0");
+  }
+
+  /**
+   * Gets an existing mount table record in the state store.
+   *
+   * @param mount The mount point of the record to remove.
+   * @return The matching record if found, null if it is not found.
+   * @throws IOException If the state store could not be accessed.
+   */
+  private MountTable getMountTableEntry(final String mount) throws IOException {
+    // Refresh the cache
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+
+    GetMountTableEntriesRequest request =
+        GetMountTableEntriesRequest.newInstance(mount);
+    RouterClient client = routerContext.getAdminClient();
+    MountTableManager mountTable = client.getMountTableManager();
+    List<MountTable> results = getMountTableEntries(mountTable, request);
+    if (results.size() > 0) {
+      // First result is sorted to have the shortest mount string length
+      return results.get(0);
+    }
+    return null;
+  }
+
+  private List<MountTable> getMountTableEntries(MountTableManager mountTable)
+      throws IOException {
+    GetMountTableEntriesRequest request =
+        GetMountTableEntriesRequest.newInstance("/");
+    return getMountTableEntries(mountTable, request);
+  }
+
+  private List<MountTable> getMountTableEntries(MountTableManager mountTable,
+      GetMountTableEntriesRequest request) throws IOException {
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    GetMountTableEntriesResponse response =
+        mountTable.getMountTableEntries(request);
+    return response.getEntries();
+  }
+}
\ 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


[10/29] hadoop git commit: HDFS-10629. Federation Roter. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10629. Federation Roter. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: f7ffbf3f92ec530b36f017302cc7605033974ba5
Parents: 1f3bc63
Author: Inigo <in...@apache.org>
Authored: Tue Mar 28 14:30:59 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:43 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/hdfs               |   5 +
 .../hadoop-hdfs/src/main/bin/hdfs.cmd           |   8 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  17 +
 .../resolver/ActiveNamenodeResolver.java        | 117 +++
 .../resolver/FederationNamenodeContext.java     |  87 +++
 .../FederationNamenodeServiceState.java         |  46 ++
 .../resolver/FederationNamespaceInfo.java       |  99 +++
 .../resolver/FileSubclusterResolver.java        |  75 ++
 .../resolver/NamenodePriorityComparator.java    |  63 ++
 .../resolver/NamenodeStatusReport.java          | 195 +++++
 .../federation/resolver/PathLocation.java       | 122 +++
 .../federation/resolver/RemoteLocation.java     |  74 ++
 .../federation/resolver/package-info.java       |  41 +
 .../federation/router/FederationUtil.java       | 117 +++
 .../router/RemoteLocationContext.java           |  38 +
 .../hdfs/server/federation/router/Router.java   | 263 +++++++
 .../federation/router/RouterRpcServer.java      | 102 +++
 .../server/federation/router/package-info.java  |  31 +
 .../federation/store/StateStoreService.java     |  77 ++
 .../server/federation/store/package-info.java   |  62 ++
 .../src/main/resources/hdfs-default.xml         |  16 +
 .../server/federation/FederationTestUtils.java  | 233 ++++++
 .../hdfs/server/federation/MockResolver.java    | 290 +++++++
 .../server/federation/RouterConfigBuilder.java  |  40 +
 .../server/federation/RouterDFSCluster.java     | 767 +++++++++++++++++++
 .../server/federation/router/TestRouter.java    |  96 +++
 26 files changed, 3080 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index e6405b5..b1f44a4 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -57,6 +57,7 @@ function hadoop_usage
   hadoop_add_subcommand "oiv" admin "apply the offline fsimage viewer to an fsimage"
   hadoop_add_subcommand "oiv_legacy" admin "apply the offline fsimage viewer to a legacy fsimage"
   hadoop_add_subcommand "portmap" daemon "run a portmap service"
+  hadoop_add_subcommand "router" daemon "run the DFS router"
   hadoop_add_subcommand "secondarynamenode" daemon "run the DFS secondary namenode"
   hadoop_add_subcommand "snapshotDiff" client "diff two snapshots of a directory or diff the current directory contents with a snapshot"
   hadoop_add_subcommand "storagepolicies" admin "list/get/set block storage policies"
@@ -176,6 +177,10 @@ function hdfscmd_case
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME=org.apache.hadoop.portmap.Portmap
     ;;
+    router)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.federation.router.Router'
+    ;;
     secondarynamenode)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode'

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
index 2181e47..b9853d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
@@ -59,7 +59,7 @@ if "%1" == "--loglevel" (
     )
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto debug
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto router debug
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -179,6 +179,11 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.CryptoAdmin
   goto :eof
 
+:router
+  set CLASS=org.apache.hadoop.hdfs.server.federation.router.Router
+  set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ROUTER_OPTS%
+  goto :eof
+
 :debug
   set CLASS=org.apache.hadoop.hdfs.tools.DebugAdmin
   goto :eof
@@ -219,6 +224,7 @@ goto :eof
   @echo   secondarynamenode    run the DFS secondary namenode
   @echo   namenode             run the DFS namenode
   @echo   journalnode          run the DFS journalnode
+  @echo   router               run the DFS router
   @echo   zkfc                 run the ZK Failover Controller daemon
   @echo   datanode             run a DFS datanode
   @echo   dfsadmin             run a DFS admin client

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/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 17cabad..ce0a17a 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
@@ -1106,6 +1106,23 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.use.dfs.network.topology";
   public static final boolean DFS_USE_DFS_NETWORK_TOPOLOGY_DEFAULT = true;
 
+  // HDFS federation
+  public static final String FEDERATION_PREFIX = "dfs.federation.";
+
+  // HDFS Router-based federation
+  public static final String FEDERATION_ROUTER_PREFIX =
+      "dfs.federation.router.";
+
+  // HDFS Router State Store connection
+  public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =
+      FEDERATION_ROUTER_PREFIX + "file.resolver.client.class";
+  public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT =
+      "org.apache.hadoop.hdfs.server.federation.MockResolver";
+  public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS =
+      FEDERATION_ROUTER_PREFIX + "namenode.resolver.client.class";
+  public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
+      "org.apache.hadoop.hdfs.server.federation.MockResolver";
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
new file mode 100644
index 0000000..477053d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Locates the most active NN for a given nameservice ID or blockpool ID. This
+ * interface is used by the {@link org.apache.hadoop.hdfs.server.federation.
+ * router.RouterRpcServer RouterRpcServer} to:
+ * <ul>
+ * <li>Determine the target NN for a given subcluster.
+ * <li>List of all namespaces discovered/active in the federation.
+ * <li>Update the currently active NN empirically.
+ * </ul>
+ * The interface is also used by the {@link org.apache.hadoop.hdfs.server.
+ * federation.router.NamenodeHeartbeatService NamenodeHeartbeatService} to
+ * register a discovered NN.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ActiveNamenodeResolver {
+
+  /**
+   * Report a successful, active NN address for a nameservice or blockPool.
+   *
+   * @param ns Nameservice identifier.
+   * @param successfulAddress The address the successful responded to the
+   *                          command.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  void updateActiveNamenode(
+      String ns, InetSocketAddress successfulAddress) throws IOException;
+
+  /**
+   * Returns a prioritized list of the most recent cached registration entries
+   * for a single nameservice ID.
+   * Returns an empty list if none are found. Returns entries in preference of:
+   * <ul>
+   * <li>The most recent ACTIVE NN
+   * <li>The most recent STANDBY NN
+   * <li>The most recent UNAVAILABLE NN
+   * </ul>
+   *
+   * @param nameserviceId Nameservice identifier.
+   * @return Prioritized list of namenode contexts.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  List<? extends FederationNamenodeContext>
+      getNamenodesForNameserviceId(String nameserviceId) throws IOException;
+
+  /**
+   * Returns a prioritized list of the most recent cached registration entries
+   * for a single block pool ID.
+   * Returns an empty list if none are found. Returns entries in preference of:
+   * <ul>
+   * <li>The most recent ACTIVE NN
+   * <li>The most recent STANDBY NN
+   * <li>The most recent UNAVAILABLE NN
+   * </ul>
+   *
+   * @param blockPoolId Block pool identifier for the nameservice.
+   * @return Prioritized list of namenode contexts.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  List<? extends FederationNamenodeContext>
+      getNamenodesForBlockPoolId(String blockPoolId) throws IOException;
+
+  /**
+   * Register a namenode in the State Store.
+   *
+   * @param report Namenode status report.
+   * @return True if the node was registered and successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the namenode could not be
+   *           registered.
+   */
+  boolean registerNamenode(NamenodeStatusReport report) throws IOException;
+
+  /**
+   * Get a list of all namespaces that are registered and active in the
+   * federation.
+   *
+   * @return List of name spaces in the federation
+   * @throws Throws exception if the namespace list is not available.
+   */
+  Set<FederationNamespaceInfo> getNamespaces() throws IOException;
+
+  /**
+   * Assign a unique identifier for the parent router service.
+   * Required to report the status to the namenode resolver.
+   *
+   * @param router Unique string identifier for the router.
+   */
+  void setRouterId(String routerId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java
new file mode 100644
index 0000000..68ef02a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.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.federation.resolver;
+
+/**
+ * Interface for a discovered NN and its current server endpoints.
+ */
+public interface FederationNamenodeContext {
+
+  /**
+   * Get the RPC server address of the namenode.
+   *
+   * @return RPC server address in the form of host:port.
+   */
+  String getRpcAddress();
+
+  /**
+   * Get the Service RPC server address of the namenode.
+   *
+   * @return Service RPC server address in the form of host:port.
+   */
+  String getServiceAddress();
+
+  /**
+   * Get the Lifeline RPC server address of the namenode.
+   *
+   * @return Lifeline RPC server address in the form of host:port.
+   */
+  String getLifelineAddress();
+
+  /**
+   * Get the HTTP server address of the namenode.
+   *
+   * @return HTTP address in the form of host:port.
+   */
+  String getWebAddress();
+
+  /**
+   * Get the unique key representing the namenode.
+   *
+   * @return Combination of the nameservice and the namenode IDs.
+   */
+  String getNamenodeKey();
+
+  /**
+   * Identifier for the nameservice/namespace.
+   *
+   * @return Namenode nameservice identifier.
+   */
+  String getNameserviceId();
+
+  /**
+   * Identifier for the namenode.
+   *
+   * @return String
+   */
+  String getNamenodeId();
+
+  /**
+   * The current state of the namenode (active, standby, etc).
+   *
+   * @return FederationNamenodeServiceState State of the namenode.
+   */
+  FederationNamenodeServiceState getState();
+
+  /**
+   * The update date.
+   *
+   * @return Long with the update date.
+   */
+  long getDateModified();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
new file mode 100644
index 0000000..c773f82
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+
+/**
+ * Namenode state in the federation. The order of this enum is used to evaluate
+ * NN priority for RPC calls.
+ */
+public enum FederationNamenodeServiceState {
+  ACTIVE, // HAServiceState.ACTIVE or operational.
+  STANDBY, // HAServiceState.STANDBY.
+  UNAVAILABLE, // When the namenode cannot be reached.
+  EXPIRED; // When the last update is too old.
+
+  public static FederationNamenodeServiceState getState(HAServiceState state) {
+    switch(state) {
+    case ACTIVE:
+      return FederationNamenodeServiceState.ACTIVE;
+    case STANDBY:
+      return FederationNamenodeServiceState.STANDBY;
+    case INITIALIZING:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    case STOPPING:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    default:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
new file mode 100644
index 0000000..bbaeca3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
+
+/**
+ * Represents information about a single nameservice/namespace in a federated
+ * HDFS cluster.
+ */
+public class FederationNamespaceInfo
+    implements Comparable<FederationNamespaceInfo>, RemoteLocationContext {
+
+  /** Block pool identifier. */
+  private String blockPoolId;
+  /** Cluster identifier. */
+  private String clusterId;
+  /** Nameservice identifier. */
+  private String nameserviceId;
+
+  public FederationNamespaceInfo(String bpId, String clId, String nsId) {
+    this.blockPoolId = bpId;
+    this.clusterId = clId;
+    this.nameserviceId = nsId;
+  }
+
+  /**
+   * The HDFS nameservice id for this namespace.
+   *
+   * @return Nameservice identifier.
+   */
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  /**
+   * The HDFS cluster id for this namespace.
+   *
+   * @return Cluster identifier.
+   */
+  public String getClusterId() {
+    return this.clusterId;
+  }
+
+  /**
+   * The HDFS block pool id for this namespace.
+   *
+   * @return Block pool identifier.
+   */
+  public String getBlockPoolId() {
+    return this.blockPoolId;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.nameserviceId.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    } else if (obj instanceof FederationNamespaceInfo) {
+      return this.compareTo((FederationNamespaceInfo) obj) == 0;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int compareTo(FederationNamespaceInfo info) {
+    return this.nameserviceId.compareTo(info.getNameserviceId());
+  }
+
+  @Override
+  public String toString() {
+    return this.nameserviceId + "->" + this.blockPoolId + ":" + this.clusterId;
+  }
+
+  @Override
+  public String getDest() {
+    return this.nameserviceId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
new file mode 100644
index 0000000..af9f493
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
@@ -0,0 +1,75 @@
+/**
+ * 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.resolver;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface to map a file path in the global name space to a specific
+ * subcluster and path in an HDFS name space.
+ * <p>
+ * Each path in the global/federated namespace may map to 1-N different HDFS
+ * locations.  Each location specifies a single nameservice and a single HDFS
+ * path.  The behavior is similar to MergeFS and Nfly and allows the merger
+ * of multiple HDFS locations into a single path.  See HADOOP-8298 and
+ * HADOOP-12077
+ * <p>
+ * For example, a directory listing will fetch listings for each destination
+ * path and combine them into a single set of results.
+ * <p>
+ * When multiple destinations are available for a path, the destinations are
+ * prioritized in a consistent manner.  This allows the proxy server to
+ * guess the best/most likely destination and attempt it first.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FileSubclusterResolver {
+
+  /**
+   * Get the destinations for a global path. Results are from the mount table
+   * cache.  If multiple destinations are available, the first result is the
+   * highest priority destination.
+   *
+   * @param path Global path.
+   * @return Location in a destination namespace or null if it does not exist.
+   * @throws IOException Throws exception if the data is not available.
+   */
+  PathLocation getDestinationForPath(String path) throws IOException;
+
+  /**
+   * Get a list of mount points for a path. Results are from the mount table
+   * cache.
+   *
+   * @return List of mount points present at this path or zero-length list if
+   *         none are found.
+   * @throws IOException Throws exception if the data is not available.
+   */
+  List<String> getMountPoints(String path) throws IOException;
+
+  /**
+   * Get the default namespace for the cluster.
+   *
+   * @return Default namespace identifier.
+   */
+  String getDefaultNamespace();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
new file mode 100644
index 0000000..fe82f29
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
@@ -0,0 +1,63 @@
+/**
+ * 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.resolver;
+
+import java.util.Comparator;
+
+/**
+ * Compares NNs in the same namespace and prioritizes by their status. The
+ * priorities are:
+ * <ul>
+ * <li>ACTIVE
+ * <li>STANDBY
+ * <li>UNAVAILABLE
+ * </ul>
+ * When two NNs have the same state, the last modification date is the tie
+ * breaker, newest has priority. Expired NNs are excluded.
+ */
+public class NamenodePriorityComparator
+    implements Comparator<FederationNamenodeContext> {
+
+  @Override
+  public int compare(FederationNamenodeContext o1,
+      FederationNamenodeContext o2) {
+    FederationNamenodeServiceState state1 = o1.getState();
+    FederationNamenodeServiceState state2 = o2.getState();
+
+    if (state1 == state2) {
+      // Both have the same state, use mode dates
+      return compareModDates(o1, o2);
+    } else {
+      // Enum is ordered by priority
+      return state1.compareTo(state2);
+    }
+  }
+
+  /**
+   * Compare the modification dates.
+   *
+   * @param o1 Context 1.
+   * @param o2 Context 2.
+   * @return Comparison between dates.
+   */
+  private int compareModDates(FederationNamenodeContext o1,
+      FederationNamenodeContext o2) {
+    // Reverse sort, lowest position is highest priority.
+    return (int) (o2.getDateModified() - o1.getDateModified());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/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
new file mode 100644
index 0000000..9259048
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+/**
+ * Status of the namenode.
+ */
+public class NamenodeStatusReport {
+
+  /** Namenode information. */
+  private String nameserviceId = "";
+  private String namenodeId = "";
+  private String clusterId = "";
+  private String blockPoolId = "";
+  private String rpcAddress = "";
+  private String serviceAddress = "";
+  private String lifelineAddress = "";
+  private String webAddress = "";
+
+  /** Namenode state. */
+  private HAServiceState status = HAServiceState.STANDBY;
+  private boolean safeMode = false;
+
+  /** If the fields are valid. */
+  private boolean registrationValid = false;
+  private boolean haStateValid = false;
+
+  public NamenodeStatusReport(String ns, String nn, String rpc, String service,
+      String lifeline, String web) {
+    this.nameserviceId = ns;
+    this.namenodeId = nn;
+    this.rpcAddress = rpc;
+    this.serviceAddress = service;
+    this.lifelineAddress = lifeline;
+    this.webAddress = web;
+  }
+
+  /**
+   * If the registration is valid.
+   *
+   * @return If the registration is valid.
+   */
+  public boolean registrationValid() {
+    return this.registrationValid;
+  }
+
+  /**
+   * If the HA state is valid.
+   *
+   * @return If the HA state is valid.
+   */
+  public boolean haStateValid() {
+    return this.haStateValid;
+  }
+
+  /**
+   * Get the state of the Namenode being monitored.
+   *
+   * @return State of the Namenode.
+   */
+  public FederationNamenodeServiceState getState() {
+    if (!registrationValid) {
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    } else if (haStateValid) {
+      return FederationNamenodeServiceState.getState(status);
+    } else {
+      return FederationNamenodeServiceState.ACTIVE;
+    }
+  }
+
+  /**
+   * Get the name service identifier.
+   *
+   * @return The name service identifier.
+   */
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  /**
+   * Get the namenode identifier.
+   *
+   * @return The namenode identifier.
+   */
+  public String getNamenodeId() {
+    return this.namenodeId;
+  }
+
+  /**
+   * Get the cluster identifier.
+   *
+   * @return The cluster identifier.
+   */
+  public String getClusterId() {
+    return this.clusterId;
+  }
+
+  /**
+   * Get the block pool identifier.
+   *
+   * @return The block pool identifier.
+   */
+  public String getBlockPoolId() {
+    return this.blockPoolId;
+  }
+
+  /**
+   * Get the RPC address.
+   *
+   * @return The RPC address.
+   */
+  public String getRpcAddress() {
+    return this.rpcAddress;
+  }
+
+  /**
+   * Get the Service RPC address.
+   *
+   * @return The Service RPC address.
+   */
+  public String getServiceAddress() {
+    return this.serviceAddress;
+  }
+
+  /**
+   * Get the Lifeline RPC address.
+   *
+   * @return The Lifeline RPC address.
+   */
+  public String getLifelineAddress() {
+    return this.lifelineAddress;
+  }
+
+  /**
+   * Get the web address.
+   *
+   * @return The web address.
+   */
+  public String getWebAddress() {
+    return this.webAddress;
+  }
+
+  /**
+   * Get the HA service state.
+   *
+   * @return The HA service state.
+   */
+  public void setHAServiceState(HAServiceState state) {
+    this.status = state;
+    this.haStateValid = true;
+  }
+
+  /**
+   * Set the namespace information.
+   *
+   * @param info Namespace information.
+   */
+  public void setNamespaceInfo(NamespaceInfo info) {
+    this.clusterId = info.getClusterID();
+    this.blockPoolId = info.getBlockPoolID();
+    this.registrationValid = true;
+  }
+
+  public void setSafeMode(boolean safemode) {
+    this.safeMode = safemode;
+  }
+
+  public boolean getSafemode() {
+    return this.safeMode;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s-%s:%s",
+        nameserviceId, namenodeId, serviceAddress);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
new file mode 100644
index 0000000..d90565c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
@@ -0,0 +1,122 @@
+/**
+ * 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.resolver;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A map of the properties and target destinations (name space + path) for
+ * a path in the global/federated namespace.
+ * This data is generated from the @see MountTable records.
+ */
+public class PathLocation {
+
+  /** Source path in global namespace. */
+  private final String sourcePath;
+
+  /** Remote paths in the target namespaces. */
+  private final List<RemoteLocation> destinations;
+
+  /** List of name spaces present. */
+  private final Set<String> namespaces;
+
+
+  /**
+   * Create a new PathLocation.
+   *
+   * @param source Source path in the global name space.
+   * @param dest Destinations of the mount table entry.
+   * @param namespaces Unique identifier representing the combination of
+   *          name spaces present in the destination list.
+   */
+  public PathLocation(
+      String source, List<RemoteLocation> dest, Set<String> nss) {
+    this.sourcePath = source;
+    this.destinations = dest;
+    this.namespaces = nss;
+  }
+
+  /**
+   * Create a path location from another path.
+   *
+   * @param other Other path location to copy from.
+   */
+  public PathLocation(PathLocation other) {
+    this.sourcePath = other.sourcePath;
+    this.destinations = new LinkedList<RemoteLocation>(other.destinations);
+    this.namespaces = new HashSet<String>(other.namespaces);
+  }
+
+  /**
+   * Get the source path in the global namespace for this path location.
+   *
+   * @return The path in the global namespace.
+   */
+  public String getSourcePath() {
+    return this.sourcePath;
+  }
+
+  /**
+   * Get the list of subclusters defined for the destinations.
+   */
+  public Set<String> getNamespaces() {
+    return Collections.unmodifiableSet(this.namespaces);
+  }
+
+  @Override
+  public String toString() {
+    RemoteLocation loc = getDefaultLocation();
+    return loc.getNameserviceId() + "->" + loc.getDest();
+  }
+
+  /**
+   * Check if this location supports multiple clusters/paths.
+   *
+   * @return If it has multiple destinations.
+   */
+  public boolean hasMultipleDestinations() {
+    return this.destinations.size() > 1;
+  }
+
+  /**
+   * Get the list of locations found in the mount table.
+   * The first result is the highest priority path.
+   *
+   * @return List of remote locations.
+   */
+  public List<RemoteLocation> getDestinations() {
+    return Collections.unmodifiableList(this.destinations);
+  }
+
+  /**
+   * Get the default or highest priority location.
+   *
+   * @return The default location.
+   */
+  public RemoteLocation getDefaultLocation() {
+    if (destinations.isEmpty() || destinations.get(0).getDest() == null) {
+      throw new UnsupportedOperationException(
+          "Unsupported path " + sourcePath + " please check mount table");
+    }
+    return destinations.get(0);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
new file mode 100644
index 0000000..eef136d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
@@ -0,0 +1,74 @@
+/**
+ * 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.resolver;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
+
+/**
+ * A single in a remote namespace consisting of a nameservice ID
+ * and a HDFS path.
+ */
+public class RemoteLocation implements RemoteLocationContext {
+
+  /** Identifier of the remote namespace for this location. */
+  private String nameserviceId;
+  /** Path in the remote location. */
+  private String path;
+
+  /**
+   * Create a new remote location.
+   *
+   * @param nsId Destination namespace.
+   * @param pPath Path in the destination namespace.
+   */
+  public RemoteLocation(String nsId, String pPath) {
+    this.nameserviceId = nsId;
+    this.path = pPath;
+  }
+
+  @Override
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  @Override
+  public String getDest() {
+    return this.path;
+  }
+
+  @Override
+  public String toString() {
+    return this.nameserviceId + "->" + this.path;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 31)
+        .append(this.nameserviceId)
+        .append(this.path)
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return (obj != null &&
+        obj.getClass() == this.getClass() &&
+        obj.hashCode() == this.hashCode());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java
new file mode 100644
index 0000000..d8be9e3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+/**
+ * The resolver package contains indepedent data resolvers used in HDFS
+ * federation. The data resolvers collect data from the cluster, including from
+ * the state store. The resolvers expose APIs used by HDFS federation to collect
+ * aggregated, cached data for use in Real-time request processing. The
+ * resolvers are perf-sensitive and are used in the flow of the
+ * {@link RouterRpcServer} request path.
+ * <p>
+ * The principal resolvers are:
+ * <ul>
+ * <li>{@link ActiveNamenodeResolver} Real-time interface for locating the most
+ * recently active NN for a nameservice.
+ * <li>{@link FileSubclusterResolver} Real-time interface for determining the NN
+ * and local file path for a given file/folder based on the global namespace
+ * path.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
new file mode 100644
index 0000000..6e7e865
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import java.lang.reflect.Constructor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+
+/**
+ * Utilities for managing HDFS federation.
+ */
+public final class FederationUtil {
+
+  private static final Log LOG = LogFactory.getLog(FederationUtil.class);
+
+  private FederationUtil() {
+    // Utility Class
+  }
+
+  /**
+   * Create an instance of an interface with a constructor using a state store
+   * constructor.
+   *
+   * @param conf Configuration
+   * @param context Context object to pass to the instance.
+   * @param contextType Type of the context passed to the constructor.
+   * @param configurationKeyName Configuration key to retrieve the class to load
+   * @param defaultClassName Default class to load if the configuration key is
+   *          not set
+   * @param clazz Class/interface that must be implemented by the instance.
+   * @return New instance of the specified class that implements the desired
+   *         interface and a single parameter constructor containing a
+   *         StateStore reference.
+   */
+  private static <T, R> T newInstance(final Configuration conf,
+      final R context, final Class<R> contextClass,
+      final String configKeyName, final String defaultClassName,
+      final Class<T> clazz) {
+
+    String className = conf.get(configKeyName, defaultClassName);
+    try {
+      Class<?> instance = conf.getClassByName(className);
+      if (clazz.isAssignableFrom(instance)) {
+        if (contextClass == null) {
+          // Default constructor if no context
+          @SuppressWarnings("unchecked")
+          Constructor<T> constructor =
+              (Constructor<T>) instance.getConstructor();
+          return constructor.newInstance();
+        } else {
+          // Constructor with context
+          @SuppressWarnings("unchecked")
+          Constructor<T> constructor = (Constructor<T>) instance.getConstructor(
+              Configuration.class, contextClass);
+          return constructor.newInstance(conf, context);
+        }
+      } else {
+        throw new RuntimeException("Class " + className + " not instance of "
+            + clazz.getCanonicalName());
+      }
+    } catch (ReflectiveOperationException e) {
+      LOG.error("Could not instantiate: " + className, e);
+      return null;
+    }
+  }
+
+  /**
+   * Creates an instance of a FileSubclusterResolver from the configuration.
+   *
+   * @param conf Configuration that defines the file resolver class.
+   * @param obj Context object passed to class constructor.
+   * @return FileSubclusterResolver
+   */
+  public static FileSubclusterResolver newFileSubclusterResolver(
+      Configuration conf, StateStoreService stateStore) {
+    return newInstance(conf, stateStore, StateStoreService.class,
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT,
+        FileSubclusterResolver.class);
+  }
+
+  /**
+   * Creates an instance of an ActiveNamenodeResolver from the configuration.
+   *
+   * @param conf Configuration that defines the namenode resolver class.
+   * @param obj Context object passed to class constructor.
+   * @return ActiveNamenodeResolver
+   */
+  public static ActiveNamenodeResolver newActiveNamenodeResolver(
+      Configuration conf, StateStoreService stateStore) {
+    return newInstance(conf, stateStore, StateStoreService.class,
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT,
+        ActiveNamenodeResolver.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
new file mode 100644
index 0000000..da6066b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
@@ -0,0 +1,38 @@
+/**
+ * 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;
+
+/**
+ * Interface for objects that are unique to a namespace.
+ */
+public interface RemoteLocationContext {
+
+  /**
+   * Returns an identifier for a unique namespace.
+   *
+   * @return Namespace identifier.
+   */
+  String getNameserviceId();
+
+  /**
+   * Destination in this location. For example the path in a remote namespace.
+   *
+   * @return Destination in this location.
+   */
+  String getDest();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
new file mode 100644
index 0000000..fe0d02a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -0,0 +1,263 @@
+/**
+ * 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.apache.hadoop.hdfs.server.federation.router.FederationUtil.newActiveNamenodeResolver;
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newFileSubclusterResolver;
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Router that provides a unified view of multiple federated HDFS clusters. It
+ * has two main roles: (1) federated interface and (2) NameNode heartbeat.
+ * <p>
+ * For the federated interface, the Router receives a client request, checks the
+ * State Store for the correct subcluster, and forwards the request to the
+ * active Namenode of that subcluster. The reply from the Namenode then flows in
+ * the opposite direction. The Routers are stateless and can be behind a load
+ * balancer. HDFS clients connect to the router using the same interfaces as are
+ * used to communicate with a namenode, namely the ClientProtocol RPC interface
+ * and the WebHdfs HTTP interface exposed by the router. {@link RouterRpcServer}
+ * {@link RouterHttpServer}
+ * <p>
+ * For NameNode heartbeat, the Router periodically checks the state of a
+ * NameNode (usually on the same server) and reports their high availability
+ * (HA) state and load/space status to the State Store. Note that this is an
+ * optional role as a Router can be independent of any subcluster.
+ * {@link StateStoreService} {@link NamenodeHeartbeatService}
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class Router extends CompositeService {
+
+  private static final Log LOG = LogFactory.getLog(Router.class);
+
+
+  /** Configuration for the Router. */
+  private Configuration conf;
+
+  /** Router address/identifier. */
+  private String routerId;
+
+  /** RPC interface to the client. */
+  private RouterRpcServer rpcServer;
+
+  /** Interface with the State Store. */
+  private StateStoreService stateStore;
+
+  /** Interface to map global name space to HDFS subcluster name spaces. */
+  private FileSubclusterResolver subclusterResolver;
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private ActiveNamenodeResolver namenodeResolver;
+
+
+  /** Usage string for help message. */
+  private static final String USAGE = "Usage: java Router";
+
+  /** Priority of the Router shutdown hook. */
+  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+
+
+  /////////////////////////////////////////////////////////
+  // Constructor
+  /////////////////////////////////////////////////////////
+
+  public Router() {
+    super(Router.class.getName());
+  }
+
+  /////////////////////////////////////////////////////////
+  // Service management
+  /////////////////////////////////////////////////////////
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
+
+    // TODO Interface to the State Store
+    this.stateStore = null;
+
+    // Resolver to track active NNs
+    this.namenodeResolver = newActiveNamenodeResolver(
+        this.conf, this.stateStore);
+    if (this.namenodeResolver == null) {
+      throw new IOException("Cannot find namenode resolver.");
+    }
+
+    // Lookup interface to map between the global and subcluster name spaces
+    this.subclusterResolver = newFileSubclusterResolver(
+        this.conf, this.stateStore);
+    if (this.subclusterResolver == null) {
+      throw new IOException("Cannot find subcluster resolver");
+    }
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+
+    super.serviceStop();
+  }
+
+  /**
+   * Shutdown the router.
+   */
+  public void shutDown() {
+    new Thread() {
+      @Override
+      public void run() {
+        Router.this.stop();
+      }
+    }.start();
+  }
+
+  /**
+   * Main run loop for the router.
+   *
+   * @param argv parameters.
+   */
+  public static void main(String[] argv) {
+    if (DFSUtil.parseHelpArgument(argv, Router.USAGE, System.out, true)) {
+      System.exit(0);
+    }
+
+    try {
+      StringUtils.startupShutdownMessage(Router.class, argv, LOG);
+
+      Router router = new Router();
+
+      ShutdownHookManager.get().addShutdownHook(
+          new CompositeServiceShutdownHook(router), SHUTDOWN_HOOK_PRIORITY);
+
+      Configuration conf = new HdfsConfiguration();
+      router.init(conf);
+      router.start();
+    } catch (Throwable e) {
+      LOG.error("Failed to start router.", e);
+      terminate(1, e);
+    }
+  }
+
+  /////////////////////////////////////////////////////////
+  // RPC Server
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create a new Router RPC server to proxy ClientProtocol requests.
+   *
+   * @return RouterRpcServer
+   * @throws IOException If the router RPC server was not started.
+   */
+  protected RouterRpcServer createRpcServer() throws IOException {
+    return new RouterRpcServer(this.conf, this, this.getNamenodeResolver(),
+        this.getSubclusterResolver());
+  }
+
+  /**
+   * Get the Router RPC server.
+   *
+   * @return Router RPC server.
+   */
+  public RouterRpcServer getRpcServer() {
+    return this.rpcServer;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Submodule getters
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Get the State Store service.
+   *
+   * @return State Store service.
+   */
+  public StateStoreService getStateStore() {
+    return this.stateStore;
+  }
+
+  /**
+   * Get the subcluster resolver for files.
+   *
+   * @return Subcluster resolver for files.
+   */
+  public FileSubclusterResolver getSubclusterResolver() {
+    return this.subclusterResolver;
+  }
+
+  /**
+   * Get the namenode resolver for a subcluster.
+   *
+   * @return The namenode resolver for a subcluster.
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return this.namenodeResolver;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Router info
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Unique ID for the router, typically the hostname:port string for the
+   * router's RPC server. This ID may be null on router startup before the RPC
+   * server has bound to a port.
+   *
+   * @return Router identifier.
+   */
+  public String getRouterId() {
+    return this.routerId;
+  }
+
+  /**
+   * Sets a unique ID for this router.
+   *
+   * @param router Identifier of the Router.
+   */
+  public void setRouterId(String id) {
+    this.routerId = id;
+    if (this.stateStore != null) {
+      this.stateStore.setIdentifier(this.routerId);
+    }
+    if (this.namenodeResolver != null) {
+      this.namenodeResolver.setRouterId(this.routerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
new file mode 100644
index 0000000..24792bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -0,0 +1,102 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.service.AbstractService;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is responsible for handling all of the RPC calls to the It is
+ * created, started, and stopped by {@link Router}. It implements the
+ * {@link ClientProtocol} to mimic a
+ * {@link org.apache.hadoop.hdfs.server.namenode.NameNode NameNode} and proxies
+ * the requests to the active
+ * {@link org.apache.hadoop.hdfs.server.namenode.NameNode NameNode}.
+ */
+public class RouterRpcServer extends AbstractService {
+
+  /** The RPC server that listens to requests from clients. */
+  private final Server rpcServer;
+
+  /**
+   * Construct a router RPC server.
+   *
+   * @param configuration HDFS Configuration.
+   * @param nnResolver The NN resolver instance to determine active NNs in HA.
+   * @param fileResolver File resolver to resolve file paths to subclusters.
+   * @throws IOException If the RPC server could not be created.
+   */
+  public RouterRpcServer(Configuration configuration, Router router,
+      ActiveNamenodeResolver nnResolver, FileSubclusterResolver fileResolver)
+          throws IOException {
+    super(RouterRpcServer.class.getName());
+
+    this.rpcServer = null;
+  }
+
+  /**
+   * Allow access to the client RPC server for testing.
+   *
+   * @return The RPC server.
+   */
+  @VisibleForTesting
+  public Server getServer() {
+    return this.rpcServer;
+  }
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    super.serviceInit(configuration);
+  }
+
+  /**
+   * Start client and service RPC servers.
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    if (this.rpcServer != null) {
+      this.rpcServer.start();
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (this.rpcServer != null) {
+      this.rpcServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  /**
+   * Wait until the RPC servers have shutdown.
+   */
+  void join() throws InterruptedException {
+    if (this.rpcServer != null) {
+      this.rpcServer.join();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java
new file mode 100644
index 0000000..327f39b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * The router package includes the core services for a HDFS federation router.
+ * The {@link Router} acts as a transparent proxy in front of a cluster of
+ * multiple NameNodes and nameservices. The {@link RouterRpcServer} exposes the
+ * NameNode clientProtocol and is the primary contact point for DFS clients in a
+ * federated cluster.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
new file mode 100644
index 0000000..866daa3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
@@ -0,0 +1,77 @@
+/**
+ * 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.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.CompositeService;
+
+/**
+ * A service to initialize a
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} and maintain the connection to the data store. There
+ * are multiple state store driver connections supported:
+ * <ul>
+ * <li>File {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
+ * StateStoreFileImpl StateStoreFileImpl}
+ * <li>ZooKeeper {@link org.apache.hadoop.hdfs.server.federation.store.driver.
+ * impl.StateStoreZooKeeperImpl StateStoreZooKeeperImpl}
+ * </ul>
+ * <p>
+ * The service also supports the dynamic registration of data interfaces such as
+ * the following:
+ * <ul>
+ * <li>{@link MembershipStateStore}: state of the Namenodes in the
+ * federation.
+ * <li>{@link MountTableStore}: Mount table between to subclusters.
+ * See {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
+ * <li>{@link RouterStateStore}: State of the routers in the federation.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class StateStoreService extends CompositeService {
+
+  /** Identifier for the service. */
+  private String identifier;
+
+  // Stub class
+  public StateStoreService(String name) {
+    super(name);
+  }
+
+  /**
+   * Fetch a unique identifier for this state store instance. Typically it is
+   * the address of the router.
+   *
+   * @return Unique identifier for this store.
+   */
+  public String getIdentifier() {
+    return this.identifier;
+  }
+
+  /**
+   * Set a unique synchronization identifier for this store.
+   *
+   * @param id Unique identifier, typically the router's RPC address.
+   */
+  public void setIdentifier(String id) {
+    this.identifier = id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java
new file mode 100644
index 0000000..949ec7c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java
@@ -0,0 +1,62 @@
+/**
+ * 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.
+ */
+
+/**
+ * The federation state store tracks persistent values that are shared between
+ * multiple routers.
+ * <p>
+ * Data is stored in data records that inherit from a common class. Data records
+ * are serialized when written to the data store using a modular serialization
+ * implementation. The default is profobuf serialization. Data is stored as rows
+ * of records of the same type with each data member in a record representing a
+ * column.
+ * <p>
+ * The state store uses a modular data storage
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} to handle querying, updating and deleting data records. The
+ * data storage driver is initialized and maintained by the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.
+ * StateStoreService FederationStateStoreService}. The state store
+ * supports fetching all records of a type, filtering by column values or
+ * fetching a single record by its primary key.
+ * <p>
+ * The state store contains several API interfaces, one for each data records
+ * type.
+ * <p>
+ * <ul>
+ * <li>FederationMembershipStateStore: state of all Namenodes in the federation.
+ * Uses the MembershipState record.
+ * <li>FederationMountTableStore: Mount table mapping paths in the global
+ * namespace to individual subcluster paths. Uses the MountTable record.
+ * <li>RouterStateStore: State of all routers in the federation. Uses the
+ * RouterState record.
+ * </ul>
+ * <p>
+ * Each API is defined in a separate interface. The implementations of these
+ * interfaces are responsible for accessing the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} to query, update and delete data records.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/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 bbe2eca..6e31388 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
@@ -4619,4 +4619,20 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.file.resolver.client.class</name>
+    <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
+    <description>
+      Class to resolve files to subclusters.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.namenode.resolver.client.class</name>
+    <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
+    <description>
+      Class to resolve the namenode for a subcluster.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
new file mode 100644
index 0000000..8674682
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
@@ -0,0 +1,233 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Date;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.security.AccessControlException;
+
+/**
+ * Helper utilities for testing HDFS Federation.
+ */
+public final class FederationTestUtils {
+
+  public final static String NAMESERVICE1 = "ns0";
+  public final static String NAMESERVICE2 = "ns1";
+  public final static String NAMENODE1 = "nn0";
+  public final static String NAMENODE2 = "nn1";
+  public final static String NAMENODE3 = "nn2";
+  public final static String NAMENODE4 = "nn3";
+  public final static String ROUTER1 = "router0";
+  public final static String ROUTER2 = "router1";
+  public final static String ROUTER3 = "router2";
+  public final static String ROUTER4 = "router3";
+  public final static long BLOCK_SIZE_BYTES = 134217728;
+
+  private FederationTestUtils() {
+    // Utility class
+  }
+
+  public static void verifyException(Object obj, String methodName,
+      Class<? extends Exception> exceptionClass, Class<?>[] parameterTypes,
+      Object[] arguments) {
+
+    Throwable triggeredException = null;
+    try {
+      Method m = obj.getClass().getMethod(methodName, parameterTypes);
+      m.invoke(obj, arguments);
+    } catch (InvocationTargetException ex) {
+      triggeredException = ex.getTargetException();
+    } catch (Exception e) {
+      triggeredException = e;
+    }
+    if (exceptionClass != null) {
+      assertNotNull("No exception was triggered, expected exception - "
+          + exceptionClass.getName(), triggeredException);
+      assertEquals(exceptionClass, triggeredException.getClass());
+    } else {
+      assertNull("Exception was triggered but no exception was expected",
+          triggeredException);
+    }
+  }
+
+  public static NamenodeStatusReport createNamenodeReport(String ns, String nn,
+      HAServiceState state) {
+    Random rand = new Random();
+    NamenodeStatusReport report = new NamenodeStatusReport(ns, nn,
+        "localhost:" + rand.nextInt(10000), "localhost:" + rand.nextInt(10000),
+        "localhost:" + rand.nextInt(10000), "testwebaddress-" + ns + nn);
+    if (state == null) {
+      // Unavailable, no additional info
+      return report;
+    }
+    report.setHAServiceState(state);
+    report.setNamespaceInfo(new NamespaceInfo(1, "tesclusterid", ns, 0,
+            "testbuildvesion", "testsoftwareversion"));
+    return report;
+  }
+
+  public static void waitNamenodeRegistered(ActiveNamenodeResolver resolver,
+      String nameserviceId, String namenodeId,
+      FederationNamenodeServiceState finalState)
+          throws InterruptedException, IllegalStateException, IOException {
+
+    for (int loopCount = 0; loopCount < 20; loopCount++) {
+
+      if (loopCount > 0) {
+        Thread.sleep(1000);
+      }
+
+      List<? extends FederationNamenodeContext> namenodes;
+      namenodes =
+          resolver.getNamenodesForNameserviceId(nameserviceId);
+      for (FederationNamenodeContext namenode : namenodes) {
+        if (namenodeId != null
+            && !namenode.getNamenodeId().equals(namenodeId)) {
+          // Keep looking
+          continue;
+        }
+        if (finalState != null && !namenode.getState().equals(finalState)) {
+          // Wrong state, wait a bit more
+          break;
+        }
+        // Found
+        return;
+      }
+    }
+    assertTrue("Failed to verify state store registration for state - "
+        + finalState + " - " + " - " + nameserviceId + " - ", false);
+  }
+
+  public static boolean verifyDate(Date d1, Date d2, long precision) {
+    if (Math.abs(d1.getTime() - d2.getTime()) < precision) {
+      return true;
+    }
+    return false;
+  }
+
+  public static boolean addDirectory(FileSystem context, String path)
+      throws IOException {
+    context.mkdirs(new Path(path), new FsPermission("777"));
+    return verifyFileExists(context, path);
+  }
+
+  public static FileStatus getFileStatus(FileSystem context, String path)
+      throws IOException {
+    return context.getFileStatus(new Path(path));
+  }
+
+  public static boolean verifyFileExists(FileSystem context, String path) {
+    try {
+      FileStatus status = getFileStatus(context, path);
+      if (status != null) {
+        return true;
+      }
+    } catch (Exception e) {
+      return false;
+    }
+
+    return false;
+  }
+
+  public static boolean checkForFileInDirectory(FileSystem context,
+      String testPath, String targetFile) throws AccessControlException,
+          FileNotFoundException,
+          UnsupportedFileSystemException, IllegalArgumentException,
+          IOException {
+    FileStatus[] fileStatus = context.listStatus(new Path(testPath));
+    String file = null;
+    String verifyPath = testPath + "/" + targetFile;
+    if (testPath.equals("/")) {
+      verifyPath = testPath + targetFile;
+    }
+
+    Boolean found = false;
+    for (int i = 0; i < fileStatus.length; i++) {
+      FileStatus f = fileStatus[i];
+      file = Path.getPathWithoutSchemeAndAuthority(f.getPath()).toString();
+      if (file.equals(verifyPath)) {
+        found = true;
+      }
+    }
+    return found;
+  }
+
+  public static int countContents(FileSystem context, String testPath)
+      throws IOException {
+    FileStatus[] fileStatus = context.listStatus(new Path(testPath));
+    return fileStatus.length;
+  }
+
+  public static void createFile(FileSystem fs, String path, long length)
+      throws IOException {
+    FsPermission permissions = new FsPermission("700");
+    FSDataOutputStream writeStream = fs.create(new Path(path), permissions,
+        true, 1000, (short) 1, BLOCK_SIZE_BYTES, null);
+    for (int i = 0; i < length; i++) {
+      writeStream.write(i);
+    }
+    writeStream.close();
+  }
+
+  public static String readFile(FileSystem fs, String path) throws IOException {
+    // Read the file from the filesystem via the active namenode
+    Path fileName = new Path(path);
+    InputStreamReader reader = new InputStreamReader(fs.open(fileName));
+    BufferedReader bufferedReader = new BufferedReader(reader);
+    StringBuilder data = new StringBuilder();
+    String line;
+
+    while ((line = bufferedReader.readLine()) != null) {
+      data.append(line);
+    }
+
+    bufferedReader.close();
+    reader.close();
+    return data.toString();
+  }
+
+  public static boolean deleteFile(FileSystem fs, String path)
+      throws IOException {
+    return fs.delete(new Path(path), true);
+  }
+}


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


[28/29] hadoop git commit: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
new file mode 100644
index 0000000..3a32be1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -0,0 +1,856 @@
+/**
+ * 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 java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A client proxy for Router -> NN communication using the NN ClientProtocol.
+ * <p>
+ * Provides routers to invoke remote ClientProtocol methods and handle
+ * retries/failover.
+ * <ul>
+ * <li>invokeSingle Make a single request to a single namespace
+ * <li>invokeSequential Make a sequential series of requests to multiple
+ * ordered namespaces until a condition is met.
+ * <li>invokeConcurrent Make concurrent requests to multiple namespaces and
+ * return all of the results.
+ * </ul>
+ * Also maintains a cached pool of connections to NNs. Connections are managed
+ * by the ConnectionManager and are unique to each user + NN. The size of the
+ * connection pool can be configured. Larger pools allow for more simultaneous
+ * requests to a single NN from a single user.
+ */
+public class RouterRpcClient {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterRpcClient.class);
+
+
+  /** Router identifier. */
+  private final String routerId;
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+  /** Connection pool to the Namenodes per user for performance. */
+  private final ConnectionManager connectionManager;
+  /** Service to run asynchronous calls. */
+  private final ExecutorService executorService;
+  /** Retry policy for router -> NN communication. */
+  private final RetryPolicy retryPolicy;
+
+  /** Pattern to parse a stack trace line. */
+  private static final Pattern STACK_TRACE_PATTERN =
+      Pattern.compile("\\tat (.*)\\.(.*)\\((.*):(\\d*)\\)");
+
+
+  /**
+   * Create a router RPC client to manage remote procedure calls to NNs.
+   *
+   * @param conf Hdfs Configuation.
+   * @param resolver A NN resolver to determine the currently active NN in HA.
+   * @param monitor Optional performance monitor.
+   */
+  public RouterRpcClient(Configuration conf, String identifier,
+      ActiveNamenodeResolver resolver) {
+
+    this.routerId = identifier;
+
+    this.namenodeResolver = resolver;
+
+    this.connectionManager = new ConnectionManager(conf);
+    this.connectionManager.start();
+
+    ThreadFactory threadFactory = new ThreadFactoryBuilder()
+        .setNameFormat("RPC Router Client-%d")
+        .build();
+    this.executorService = Executors.newCachedThreadPool(threadFactory);
+
+    int maxFailoverAttempts = conf.getInt(
+        HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
+        HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
+    int maxRetryAttempts = conf.getInt(
+        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
+        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+    int failoverSleepBaseMillis = conf.getInt(
+        HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
+        HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
+    int failoverSleepMaxMillis = conf.getInt(
+        HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
+        HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
+    this.retryPolicy = RetryPolicies.failoverOnNetworkException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts, maxRetryAttempts,
+        failoverSleepBaseMillis, failoverSleepMaxMillis);
+  }
+
+  /**
+   * Shutdown the client.
+   */
+  public void shutdown() {
+    if (this.connectionManager != null) {
+      this.connectionManager.close();
+    }
+    if (this.executorService != null) {
+      this.executorService.shutdownNow();
+    }
+  }
+
+  /**
+   * Total number of available sockets between the router and NNs.
+   *
+   * @return Number of namenode clients.
+   */
+  public int getNumConnections() {
+    return this.connectionManager.getNumConnections();
+  }
+
+  /**
+   * Total number of available sockets between the router and NNs.
+   *
+   * @return Number of namenode clients.
+   */
+  public int getNumActiveConnections() {
+    return this.connectionManager.getNumActiveConnections();
+  }
+
+  /**
+   * Total number of open connection pools to a NN. Each connection pool.
+   * represents one user + one NN.
+   *
+   * @return Number of connection pools.
+   */
+  public int getNumConnectionPools() {
+    return this.connectionManager.getNumConnectionPools();
+  }
+
+  /**
+   * Number of connections between the router and NNs being created sockets.
+   *
+   * @return Number of connections waiting to be created.
+   */
+  public int getNumCreatingConnections() {
+    return this.connectionManager.getNumCreatingConnections();
+  }
+
+  /**
+   * Get ClientProtocol proxy client for a NameNode. Each combination of user +
+   * NN must use a unique proxy client. Previously created clients are cached
+   * and stored in a connection pool by the ConnectionManager.
+   *
+   * @param ugi User group information.
+   * @param nsId Nameservice identifier.
+   * @param rpcAddress ClientProtocol RPC server address of the NN.
+   * @return ConnectionContext containing a ClientProtocol proxy client for the
+   *         NN + current user.
+   * @throws IOException If we cannot get a connection to the NameNode.
+   */
+  private ConnectionContext getConnection(
+      UserGroupInformation ugi, String nsId, String rpcAddress)
+          throws IOException {
+    ConnectionContext connection = null;
+    try {
+      // Each proxy holds the UGI info for the current user when it is created.
+      // This cache does not scale very well, one entry per user per namenode,
+      // and may need to be adjusted and/or selectively pruned. The cache is
+      // important due to the excessive overhead of creating a new proxy wrapper
+      // for each individual request.
+
+      // TODO Add tokens from the federated UGI
+      connection = this.connectionManager.getConnection(ugi, rpcAddress);
+      LOG.debug("User {} NN {} is using connection {}",
+          ugi.getUserName(), rpcAddress, connection);
+    } catch (Exception ex) {
+      LOG.error("Cannot open NN client to address: {}", rpcAddress, ex);
+    }
+
+    if (connection == null) {
+      throw new IOException("Cannot get a connection to " + rpcAddress);
+    }
+    return connection;
+  }
+
+  /**
+   * Convert an exception to an IOException.
+   *
+   * For a non-IOException, wrap it with IOException. For a RemoteException,
+   * unwrap it. For an IOException which is not a RemoteException, return it.
+   *
+   * @param e Exception to convert into an exception.
+   * @return Created IO exception.
+   */
+  private static IOException toIOException(Exception e) {
+    if (e instanceof RemoteException) {
+      return ((RemoteException) e).unwrapRemoteException();
+    }
+    if (e instanceof IOException) {
+      return (IOException)e;
+    }
+    return new IOException(e);
+  }
+
+  /**
+   * If we should retry the RPC call.
+   *
+   * @param ex Exception reported.
+   * @param retryCount Number of retries.
+   * @return Retry decision.
+   * @throws IOException Original exception if the retry policy generates one.
+   */
+  private RetryDecision shouldRetry(final IOException ioe, final int retryCount)
+      throws IOException {
+    try {
+      final RetryPolicy.RetryAction a =
+          this.retryPolicy.shouldRetry(ioe, retryCount, 0, true);
+      return a.action;
+    } catch (Exception ex) {
+      LOG.error("Re-throwing API exception, no more retries", ex);
+      throw toIOException(ex);
+    }
+  }
+
+  /**
+   * Invokes a method against the ClientProtocol proxy server. If a standby
+   * exception is generated by the call to the client, retries using the
+   * alternate server.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param ugi User group information.
+   * @param namenodes A prioritized list of namenodes within the same
+   *                  nameservice.
+   * @param method Remote ClientProtcol method to invoke.
+   * @param params Variable list of parameters matching the method.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  private Object invokeMethod(
+      final UserGroupInformation ugi,
+      final List<? extends FederationNamenodeContext> namenodes,
+      final Method method, final Object... params) throws IOException {
+
+    if (namenodes == null || namenodes.isEmpty()) {
+      throw new IOException("No namenodes to invoke " + method.getName() +
+          " with params " + Arrays.toString(params) + " from " + this.routerId);
+    }
+
+    Object ret = null;
+    boolean failover = false;
+    Map<FederationNamenodeContext, IOException> ioes = new LinkedHashMap<>();
+    for (FederationNamenodeContext namenode : namenodes) {
+      ConnectionContext connection = null;
+      try {
+        String nsId = namenode.getNameserviceId();
+        String rpcAddress = namenode.getRpcAddress();
+        connection = this.getConnection(ugi, nsId, rpcAddress);
+        ProxyAndInfo<ClientProtocol> client = connection.getClient();
+        ClientProtocol proxy = client.getProxy();
+        ret = invoke(0, method, proxy, params);
+        if (failover) {
+          // Success on alternate server, update
+          InetSocketAddress address = client.getAddress();
+          namenodeResolver.updateActiveNamenode(nsId, address);
+        }
+        return ret;
+      } catch (IOException ioe) {
+        ioes.put(namenode, ioe);
+        if (ioe instanceof StandbyException) {
+          // Fail over indicated by retry policy and/or NN
+          failover = true;
+        } else if (ioe instanceof RemoteException) {
+          // RemoteException returned by NN
+          throw (RemoteException) ioe;
+        } else {
+          // Other communication error, this is a failure
+          // Communication retries are handled by the retry policy
+          throw ioe;
+        }
+      } finally {
+        if (connection != null) {
+          connection.release();
+        }
+      }
+    }
+
+    // All namenodes were unavailable or in standby
+    String msg = "No namenode available to invoke " + method.getName() + " " +
+        Arrays.toString(params);
+    LOG.error(msg);
+    for (Entry<FederationNamenodeContext, IOException> entry :
+        ioes.entrySet()) {
+      FederationNamenodeContext namenode = entry.getKey();
+      String nsId = namenode.getNameserviceId();
+      String nnId = namenode.getNamenodeId();
+      String addr = namenode.getRpcAddress();
+      IOException ioe = entry.getValue();
+      if (ioe instanceof StandbyException) {
+        LOG.error("{} {} at {} is in Standby", nsId, nnId, addr);
+      } else {
+        LOG.error("{} {} at {} error: \"{}\"",
+            nsId, nnId, addr, ioe.getMessage());
+      }
+    }
+    throw new StandbyException(msg);
+  }
+
+  /**
+   * Invokes a method on the designated object. Catches exceptions specific to
+   * the invocation.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param method Method to invoke
+   * @param obj Target object for the method
+   * @param params Variable parameters
+   * @return Response from the remote server
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private Object invoke(int retryCount, final Method method, final Object obj,
+      final Object... params) throws IOException {
+    try {
+      return method.invoke(obj, params);
+    } catch (IllegalAccessException e) {
+      LOG.error("Unexpected exception while proxying API", e);
+      return null;
+    } catch (IllegalArgumentException e) {
+      LOG.error("Unexpected exception while proxying API", e);
+      return null;
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof IOException) {
+        IOException ioe = (IOException) cause;
+        // Check if we should retry.
+        RetryDecision decision = shouldRetry(ioe, retryCount);
+        if (decision == RetryDecision.RETRY) {
+          // retry
+          return invoke(++retryCount, method, obj, params);
+        } else if (decision == RetryDecision.FAILOVER_AND_RETRY) {
+          // failover, invoker looks for standby exceptions for failover.
+          if (ioe instanceof StandbyException) {
+            throw ioe;
+          } else {
+            throw new StandbyException(ioe.getMessage());
+          }
+        } else {
+          if (ioe instanceof RemoteException) {
+            RemoteException re = (RemoteException) ioe;
+            ioe = re.unwrapRemoteException();
+            ioe = getCleanException(ioe);
+          }
+          throw ioe;
+        }
+      } else {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  /**
+   * Get a clean copy of the exception. Sometimes the exceptions returned by the
+   * server contain the full stack trace in the message.
+   *
+   * @param ioe Exception to clean up.
+   * @return Copy of the original exception with a clean message.
+   */
+  private static IOException getCleanException(IOException ioe) {
+    IOException ret = null;
+
+    String msg = ioe.getMessage();
+    Throwable cause = ioe.getCause();
+    StackTraceElement[] stackTrace = ioe.getStackTrace();
+
+    // Clean the message by removing the stack trace
+    int index = msg.indexOf("\n");
+    if (index > 0) {
+      String[] msgSplit = msg.split("\n");
+      msg = msgSplit[0];
+
+      // Parse stack trace from the message
+      List<StackTraceElement> elements = new LinkedList<>();
+      for (int i=1; i<msgSplit.length; i++) {
+        String line = msgSplit[i];
+        Matcher matcher = STACK_TRACE_PATTERN.matcher(line);
+        if (matcher.find()) {
+          String declaringClass = matcher.group(1);
+          String methodName = matcher.group(2);
+          String fileName = matcher.group(3);
+          int lineNumber = Integer.parseInt(matcher.group(4));
+          StackTraceElement element = new StackTraceElement(
+              declaringClass, methodName, fileName, lineNumber);
+          elements.add(element);
+        }
+      }
+      stackTrace = elements.toArray(new StackTraceElement[elements.size()]);
+    }
+
+    // Create the new output exception
+    if (ioe instanceof RemoteException) {
+      RemoteException re = (RemoteException)ioe;
+      ret = new RemoteException(re.getClassName(), msg);
+    } else {
+      // Try the simple constructor and initialize the fields
+      Class<? extends IOException> ioeClass = ioe.getClass();
+      try {
+        Constructor<? extends IOException> constructor =
+            ioeClass.getDeclaredConstructor(String.class);
+        ret = constructor.newInstance(msg);
+      } catch (ReflectiveOperationException e) {
+        // If there are errors, just use the input one
+        LOG.error("Could not create exception {}", ioeClass.getSimpleName(), e);
+        ret = ioe;
+      }
+    }
+    if (ret != null) {
+      ret.initCause(cause);
+      ret.setStackTrace(stackTrace);
+    }
+
+    return ret;
+  }
+
+  /**
+   * Invokes a ClientProtocol method. Determines the target nameservice via a
+   * provided block.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param block Block used to determine appropriate nameservice.
+   * @param method The remote method and parameters to invoke.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  public Object invokeSingle(final ExtendedBlock block, RemoteMethod method)
+      throws IOException {
+    String bpId = block.getBlockPoolId();
+    return invokeSingleBlockPool(bpId, method);
+  }
+
+  /**
+   * Invokes a ClientProtocol method. Determines the target nameservice using
+   * the block pool id.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param bpId Block pool identifier.
+   * @param method The remote method and parameters to invoke.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  public Object invokeSingleBlockPool(final String bpId, RemoteMethod method)
+      throws IOException {
+    String nsId = getNameserviceForBlockPoolId(bpId);
+    return invokeSingle(nsId, method);
+  }
+
+  /**
+   * Invokes a ClientProtocol method against the specified namespace.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param nsId Target namespace for the method.
+   * @param method The remote method and parameters to invoke.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  public Object invokeSingle(final String nsId, RemoteMethod method)
+      throws IOException {
+    UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+    List<? extends FederationNamenodeContext> nns =
+        getNamenodesForNameservice(nsId);
+    RemoteLocationContext loc = new RemoteLocation(nsId, "/");
+    return invokeMethod(ugi, nns, method.getMethod(), method.getParams(loc));
+  }
+
+  /**
+   * Invokes a single proxy call for a single location.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param location RemoteLocation to invoke.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @return The result of invoking the method if successful.
+   * @throws IOException
+   */
+  public Object invokeSingle(final RemoteLocationContext location,
+      RemoteMethod remoteMethod) throws IOException {
+    List<RemoteLocationContext> locations = Collections.singletonList(location);
+    return invokeSequential(locations, remoteMethod);
+  }
+
+  /**
+   * Invokes sequential proxy calls to different locations. Continues to invoke
+   * calls until a call returns without throwing a remote exception.
+   *
+   * @param locations List of locations/nameservices to call concurrently.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @return The result of the first successful call, or if no calls are
+   *         successful, the result of the last RPC call executed.
+   * @throws IOException if the success condition is not met and one of the RPC
+   *           calls generated a remote exception.
+   */
+  public Object invokeSequential(
+      final List<? extends RemoteLocationContext> locations,
+      final RemoteMethod remoteMethod) throws IOException {
+    return invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  /**
+   * Invokes sequential proxy calls to different locations. Continues to invoke
+   * calls until the success condition is met, or until all locations have been
+   * attempted.
+   *
+   * The success condition may be specified by:
+   * <ul>
+   * <li>An expected result class
+   * <li>An expected result value
+   * </ul>
+   *
+   * If no expected result class/values are specified, the success condition is
+   * a call that does not throw a remote exception.
+   *
+   * @param locations List of locations/nameservices to call concurrently.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @param expectedResultClass In order to be considered a positive result, the
+   *          return type must be of this class.
+   * @param expectedResultValue In order to be considered a positive result, the
+   *          return value must equal the value of this object.
+   * @return The result of the first successful call, or if no calls are
+   *         successful, the result of the first RPC call executed.
+   * @throws IOException if the success condition is not met, return the first
+   *                     remote exception generated.
+   */
+  public Object invokeSequential(
+      final List<? extends RemoteLocationContext> locations,
+      final RemoteMethod remoteMethod, Class<?> expectedResultClass,
+      Object expectedResultValue) throws IOException {
+
+    final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+    final Method m = remoteMethod.getMethod();
+    IOException firstThrownException = null;
+    IOException lastThrownException = null;
+    Object firstResult = null;
+    // Invoke in priority order
+    for (final RemoteLocationContext loc : locations) {
+      String ns = loc.getNameserviceId();
+      List<? extends FederationNamenodeContext> namenodes =
+          getNamenodesForNameservice(ns);
+      try {
+        Object[] params = remoteMethod.getParams(loc);
+        Object result = invokeMethod(ugi, namenodes, m, params);
+        // Check if the result is what we expected
+        if (isExpectedClass(expectedResultClass, result) &&
+            isExpectedValue(expectedResultValue, result)) {
+          // Valid result, stop here
+          return result;
+        }
+        if (firstResult == null) {
+          firstResult = result;
+        }
+      } catch (IOException ioe) {
+        // Record it and move on
+        lastThrownException = (IOException) ioe;
+        if (firstThrownException == null) {
+          firstThrownException = lastThrownException;
+        }
+      } catch (Exception e) {
+        // Unusual error, ClientProtocol calls always use IOException (or
+        // RemoteException). Re-wrap in IOException for compatibility with
+        // ClientProtcol.
+        LOG.error("Unexpected exception {} proxying {} to {}",
+            e.getClass(), m.getName(), ns, e);
+        lastThrownException = new IOException(
+            "Unexpected exception proxying API " + e.getMessage(), e);
+        if (firstThrownException == null) {
+          firstThrownException = lastThrownException;
+        }
+      }
+    }
+
+    if (firstThrownException != null) {
+      // re-throw the last exception thrown for compatibility
+      throw firstThrownException;
+    }
+    // Return the last result, whether it is the value we are looking for or a
+    return firstResult;
+  }
+
+  /**
+   * Checks if a result matches the required result class.
+   *
+   * @param expectedResultClass Required result class, null to skip the check.
+   * @param result The result to check.
+   * @return True if the result is an instance of the required class or if the
+   *         expected class is null.
+   */
+  private static boolean isExpectedClass(Class<?> expectedClass, Object clazz) {
+    if (expectedClass == null) {
+      return true;
+    } else if (clazz == null) {
+      return false;
+    } else {
+      return expectedClass.isInstance(clazz);
+    }
+  }
+
+  /**
+   * Checks if a result matches the expected value.
+   *
+   * @param expectedResultValue The expected value, null to skip the check.
+   * @param result The result to check.
+   * @return True if the result is equals to the expected value or if the
+   *         expected value is null.
+   */
+  private static boolean isExpectedValue(Object expectedValue, Object value) {
+    if (expectedValue == null) {
+      return true;
+    } else if (value == null) {
+      return false;
+    } else {
+      return value.equals(expectedValue);
+    }
+  }
+
+  /**
+   * Invokes multiple concurrent proxy calls to different clients. Returns an
+   * array of results.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param locations List of remote locations to call concurrently.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @param requireResponse If true an exception will be thrown if all calls do
+   *          not complete. If false exceptions are ignored and all data results
+   *          successfully received are returned.
+   * @param standby If the requests should go to the standby namenodes too.
+   * @return Result of invoking the method per subcluster: nsId -> result.
+   * @throws IOException If requiredResponse=true and any of the calls throw an
+   *           exception.
+   */
+  @SuppressWarnings("unchecked")
+  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+      final Collection<T> locations, final RemoteMethod method,
+      boolean requireResponse, boolean standby) throws IOException {
+
+    final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+    final Method m = method.getMethod();
+
+    if (locations.size() == 1) {
+      // Shortcut, just one call
+      T location = locations.iterator().next();
+      String ns = location.getNameserviceId();
+      final List<? extends FederationNamenodeContext> namenodes =
+          getNamenodesForNameservice(ns);
+      Object[] paramList = method.getParams(location);
+      Object result = invokeMethod(ugi, namenodes, m, paramList);
+      return Collections.singletonMap(location, result);
+    }
+
+    List<T> orderedLocations = new LinkedList<>();
+    Set<Callable<Object>> callables = new HashSet<>();
+    for (final T location : locations) {
+      String nsId = location.getNameserviceId();
+      final List<? extends FederationNamenodeContext> namenodes =
+          getNamenodesForNameservice(nsId);
+      final Object[] paramList = method.getParams(location);
+      if (standby) {
+        // Call the objectGetter to all NNs (including standby)
+        for (final FederationNamenodeContext nn : namenodes) {
+          String nnId = nn.getNamenodeId();
+          final List<FederationNamenodeContext> nnList =
+              Collections.singletonList(nn);
+          T nnLocation = location;
+          if (location instanceof RemoteLocation) {
+            nnLocation = (T)new RemoteLocation(nsId, nnId, location.getDest());
+          }
+          orderedLocations.add(nnLocation);
+          callables.add(new Callable<Object>() {
+            public Object call() throws Exception {
+              return invokeMethod(ugi, nnList, m, paramList);
+            }
+          });
+        }
+      } else {
+        // Call the objectGetter in order of nameservices in the NS list
+        orderedLocations.add(location);
+        callables.add(new Callable<Object>() {
+          public Object call() throws Exception {
+            return invokeMethod(ugi, namenodes, m, paramList);
+          }
+        });
+      }
+    }
+
+    try {
+      List<Future<Object>> futures = executorService.invokeAll(callables);
+      Map<T, Object> results = new TreeMap<>();
+      Map<T, IOException> exceptions = new TreeMap<>();
+      for (int i=0; i<futures.size(); i++) {
+        T location = orderedLocations.get(i);
+        try {
+          Future<Object> future = futures.get(i);
+          Object result = future.get();
+          results.put(location, result);
+        } catch (ExecutionException ex) {
+          Throwable cause = ex.getCause();
+          LOG.debug("Canot execute {} in {}: {}",
+              m.getName(), location, cause.getMessage());
+
+          // Convert into IOException if needed
+          IOException ioe = null;
+          if (cause instanceof IOException) {
+            ioe = (IOException) cause;
+          } else {
+            ioe = new IOException("Unhandled exception while proxying API " +
+                m.getName() + ": " + cause.getMessage(), cause);
+          }
+
+          // Response from all servers required, use this error.
+          if (requireResponse) {
+            throw ioe;
+          }
+
+          // Store the exceptions
+          exceptions.put(location, ioe);
+        }
+      }
+
+      // Throw the exception for the first location if there are no results
+      if (results.isEmpty()) {
+        T location = orderedLocations.get(0);
+        IOException ioe = exceptions.get(location);
+        if (ioe != null) {
+          throw ioe;
+        }
+      }
+
+      return results;
+    } catch (InterruptedException ex) {
+      LOG.error("Unexpected error while invoking API: {}", ex.getMessage());
+      throw new IOException(
+          "Unexpected error while invoking API " + ex.getMessage(), ex);
+    }
+  }
+
+  /**
+   * Get a prioritized list of NNs that share the same nameservice ID (in the
+   * same namespace). NNs that are reported as ACTIVE will be first in the list.
+   *
+   * @param nameserviceId The nameservice ID for the namespace.
+   * @return A prioritized list of NNs to use for communication.
+   * @throws IOException If a NN cannot be located for the nameservice ID.
+   */
+  private List<? extends FederationNamenodeContext> getNamenodesForNameservice(
+      final String nsId) throws IOException {
+
+    final List<? extends FederationNamenodeContext> namenodes =
+        namenodeResolver.getNamenodesForNameserviceId(nsId);
+
+    if (namenodes == null || namenodes.isEmpty()) {
+      throw new IOException("Cannot locate a registered namenode for " + nsId +
+          " from " + this.routerId);
+    }
+    return namenodes;
+  }
+
+  /**
+   * Get a prioritized list of NNs that share the same block pool ID (in the
+   * same namespace). NNs that are reported as ACTIVE will be first in the list.
+   *
+   * @param blockPoolId The blockpool ID for the namespace.
+   * @return A prioritized list of NNs to use for communication.
+   * @throws IOException If a NN cannot be located for the block pool ID.
+   */
+  private List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
+      final String bpId) throws IOException {
+
+    List<? extends FederationNamenodeContext> namenodes =
+        namenodeResolver.getNamenodesForBlockPoolId(bpId);
+
+    if (namenodes == null || namenodes.isEmpty()) {
+      throw new IOException("Cannot locate a registered namenode for " + bpId +
+          " from " + this.routerId);
+    }
+    return namenodes;
+  }
+
+  /**
+   * Get the nameservice identifier for a block pool.
+   *
+   * @param bpId Identifier of the block pool.
+   * @return Nameservice identifier.
+   * @throws IOException If a NN cannot be located for the block pool ID.
+   */
+  private String getNameserviceForBlockPoolId(final String bpId)
+      throws IOException {
+    List<? extends FederationNamenodeContext> namenodes =
+        getNamenodesForBlockPoolId(bpId);
+    FederationNamenodeContext namenode = namenodes.get(0);
+    return namenode.getNameserviceId();
+  }
+}
\ 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


[26/29] hadoop git commit: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
index ee6f57d..2875750 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.util.Time;
 
 /**
  * In-memory cache/mock of a namenode and file resolver. Stores the most
- * recently updated NN information for each nameservice and block pool. Also
+ * recently updated NN information for each nameservice and block pool. It also
  * stores a virtual mount table for resolving global namespace paths to local NN
  * paths.
  */
@@ -51,82 +51,93 @@ public class MockResolver
     implements ActiveNamenodeResolver, FileSubclusterResolver {
 
   private Map<String, List<? extends FederationNamenodeContext>> resolver =
-      new HashMap<String, List<? extends FederationNamenodeContext>>();
-  private Map<String, List<RemoteLocation>> locations =
-      new HashMap<String, List<RemoteLocation>>();
-  private Set<FederationNamespaceInfo> namespaces =
-      new HashSet<FederationNamespaceInfo>();
+      new HashMap<>();
+  private Map<String, List<RemoteLocation>> locations = new HashMap<>();
+  private Set<FederationNamespaceInfo> namespaces = new HashSet<>();
   private String defaultNamespace = null;
 
+
   public MockResolver(Configuration conf, StateStoreService store) {
     this.cleanRegistrations();
   }
 
-  public void addLocation(String mount, String nameservice, String location) {
-    RemoteLocation remoteLocation = new RemoteLocation(nameservice, location);
-    List<RemoteLocation> locationsList = locations.get(mount);
+  public void addLocation(String mount, String nsId, String location) {
+    List<RemoteLocation> locationsList = this.locations.get(mount);
     if (locationsList == null) {
-      locationsList = new LinkedList<RemoteLocation>();
-      locations.put(mount, locationsList);
+      locationsList = new LinkedList<>();
+      this.locations.put(mount, locationsList);
     }
+
+    final RemoteLocation remoteLocation = new RemoteLocation(nsId, location);
     if (!locationsList.contains(remoteLocation)) {
       locationsList.add(remoteLocation);
     }
 
     if (this.defaultNamespace == null) {
-      this.defaultNamespace = nameservice;
+      this.defaultNamespace = nsId;
     }
   }
 
   public synchronized void cleanRegistrations() {
-    this.resolver =
-        new HashMap<String, List<? extends FederationNamenodeContext>>();
-    this.namespaces = new HashSet<FederationNamespaceInfo>();
+    this.resolver = new HashMap<>();
+    this.namespaces = new HashSet<>();
   }
 
   @Override
   public void updateActiveNamenode(
-      String ns, InetSocketAddress successfulAddress) {
+      String nsId, InetSocketAddress successfulAddress) {
 
     String address = successfulAddress.getHostName() + ":" +
         successfulAddress.getPort();
-    String key = ns;
+    String key = nsId;
     if (key != null) {
       // Update the active entry
       @SuppressWarnings("unchecked")
-      List<FederationNamenodeContext> iterator =
-          (List<FederationNamenodeContext>) resolver.get(key);
-      for (FederationNamenodeContext namenode : iterator) {
+      List<FederationNamenodeContext> namenodes =
+          (List<FederationNamenodeContext>) this.resolver.get(key);
+      for (FederationNamenodeContext namenode : namenodes) {
         if (namenode.getRpcAddress().equals(address)) {
           MockNamenodeContext nn = (MockNamenodeContext) namenode;
           nn.setState(FederationNamenodeServiceState.ACTIVE);
           break;
         }
       }
-      Collections.sort(iterator, new NamenodePriorityComparator());
+      // This operation modifies the list so we need to be careful
+      synchronized(namenodes) {
+        Collections.sort(namenodes, new NamenodePriorityComparator());
+      }
     }
   }
 
   @Override
   public List<? extends FederationNamenodeContext>
       getNamenodesForNameserviceId(String nameserviceId) {
-    return resolver.get(nameserviceId);
+    // Return a copy of the list because it is updated periodically
+    List<? extends FederationNamenodeContext> namenodes =
+        this.resolver.get(nameserviceId);
+    return Collections.unmodifiableList(new ArrayList<>(namenodes));
   }
 
   @Override
   public List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
       String blockPoolId) {
-    return resolver.get(blockPoolId);
+    // Return a copy of the list because it is updated periodically
+    List<? extends FederationNamenodeContext> namenodes =
+        this.resolver.get(blockPoolId);
+    return Collections.unmodifiableList(new ArrayList<>(namenodes));
   }
 
   private static class MockNamenodeContext
       implements FederationNamenodeContext {
+
+    private String namenodeId;
+    private String nameserviceId;
+
     private String webAddress;
     private String rpcAddress;
     private String serviceAddress;
     private String lifelineAddress;
-    private String namenodeId;
-    private String nameserviceId;
+
     private FederationNamenodeServiceState state;
     private long dateModified;
 
@@ -197,6 +208,7 @@ public class MockResolver
   @Override
   public synchronized boolean registerNamenode(NamenodeStatusReport report)
       throws IOException {
+
     MockNamenodeContext context = new MockNamenodeContext(
         report.getRpcAddress(), report.getServiceAddress(),
         report.getLifelineAddress(), report.getWebAddress(),
@@ -205,13 +217,14 @@ public class MockResolver
     String nsId = report.getNameserviceId();
     String bpId = report.getBlockPoolId();
     String cId = report.getClusterId();
+
     @SuppressWarnings("unchecked")
     List<MockNamenodeContext> existingItems =
-        (List<MockNamenodeContext>) resolver.get(nsId);
+        (List<MockNamenodeContext>) this.resolver.get(nsId);
     if (existingItems == null) {
-      existingItems = new ArrayList<MockNamenodeContext>();
-      resolver.put(bpId, existingItems);
-      resolver.put(nsId, existingItems);
+      existingItems = new ArrayList<>();
+      this.resolver.put(bpId, existingItems);
+      this.resolver.put(nsId, existingItems);
     }
     boolean added = false;
     for (int i=0; i<existingItems.size() && !added; i++) {
@@ -227,7 +240,7 @@ public class MockResolver
     Collections.sort(existingItems, new NamenodePriorityComparator());
 
     FederationNamespaceInfo info = new FederationNamespaceInfo(bpId, cId, nsId);
-    namespaces.add(info);
+    this.namespaces.add(info);
     return true;
   }
 
@@ -238,16 +251,13 @@ public class MockResolver
 
   @Override
   public PathLocation getDestinationForPath(String path) throws IOException {
-    String finalPath = null;
-    String nameservice = null;
-    Set<String> namespaceSet = new HashSet<String>();
-    LinkedList<RemoteLocation> remoteLocations =
-        new LinkedList<RemoteLocation>();
-    for(String key : this.locations.keySet()) {
-      if(path.startsWith(key)) {
+    Set<String> namespaceSet = new HashSet<>();
+    List<RemoteLocation> remoteLocations = new LinkedList<>();
+    for (String key : this.locations.keySet()) {
+      if (path.startsWith(key)) {
         for (RemoteLocation location : this.locations.get(key)) {
-          finalPath = location.getDest() + path.substring(key.length());
-          nameservice = location.getNameserviceId();
+          String finalPath = location.getDest() + path.substring(key.length());
+          String nameservice = location.getNameserviceId();
           RemoteLocation remoteLocation =
               new RemoteLocation(nameservice, finalPath);
           remoteLocations.add(remoteLocation);
@@ -265,7 +275,7 @@ public class MockResolver
 
   @Override
   public List<String> getMountPoints(String path) throws IOException {
-    List<String> mounts = new ArrayList<String>();
+    List<String> mounts = new ArrayList<>();
     if (path.equals("/")) {
       // Mounts only supported under root level
       for (String mount : this.locations.keySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
index 16d624c..39fcf7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /**
  * Constructs a router configuration with individual features enabled/disabled.
@@ -26,15 +27,32 @@ public class RouterConfigBuilder {
 
   private Configuration conf;
 
+  private boolean enableRpcServer = false;
+
   public RouterConfigBuilder(Configuration configuration) {
     this.conf = configuration;
   }
 
   public RouterConfigBuilder() {
-    this.conf = new Configuration();
+    this.conf = new Configuration(false);
+  }
+
+  public RouterConfigBuilder all() {
+    this.enableRpcServer = true;
+    return this;
+  }
+
+  public RouterConfigBuilder rpc(boolean enable) {
+    this.enableRpcServer = enable;
+    return this;
+  }
+
+  public RouterConfigBuilder rpc() {
+    return this.rpc(true);
   }
 
   public Configuration build() {
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_RPC_ENABLE, this.enableRpcServer);
     return conf;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
index 55d04ad..4031b7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -17,27 +17,44 @@
  */
 package org.apache.hadoop.hdfs.server.federation;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.addDirectory;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.waitNamenodeRegistered;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -46,16 +63,49 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service.STATE;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Test utility to mimic a federated HDFS cluster with a router.
+ * Test utility to mimic a federated HDFS cluster with multiple routers.
  */
 public class RouterDFSCluster {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterDFSCluster.class);
+
+  public static final String TEST_STRING = "teststring";
+  public static final String TEST_DIR = "testdir";
+  public static final String TEST_FILE = "testfile";
+
+
+  /** Nameservices in the federated cluster. */
+  private List<String> nameservices;
+  /** Namenodes in the federated cluster. */
+  private List<NamenodeContext> namenodes;
+  /** Routers in the federated cluster. */
+  private List<RouterContext> routers;
+  /** If the Namenodes are in high availability.*/
+  private boolean highAvailability;
+
+  /** Mini cluster. */
+  private MiniDFSCluster cluster;
+
+  /** Router configuration overrides. */
+  private Configuration routerOverrides;
+  /** Namenode configuration overrides. */
+  private Configuration namenodeOverrides;
+
+
   /**
    * Router context.
    */
@@ -69,13 +119,14 @@ public class RouterDFSCluster {
     private Configuration conf;
     private URI fileSystemUri;
 
-    public RouterContext(Configuration conf, String ns, String nn)
+    public RouterContext(Configuration conf, String nsId, String nnId)
         throws URISyntaxException {
-      this.namenodeId = nn;
-      this.nameserviceId = ns;
       this.conf = conf;
-      router = new Router();
-      router.init(conf);
+      this.nameserviceId = nsId;
+      this.namenodeId = nnId;
+
+      this.router = new Router();
+      this.router.init(conf);
     }
 
     public Router getRouter() {
@@ -99,18 +150,30 @@ public class RouterDFSCluster {
     }
 
     public void initRouter() throws URISyntaxException {
+      // Store the bound points for the router interfaces
+      InetSocketAddress rpcAddress = router.getRpcServerAddress();
+      if (rpcAddress != null) {
+        this.rpcPort = rpcAddress.getPort();
+        this.fileSystemUri =
+            URI.create("hdfs://" + NetUtils.getHostPortString(rpcAddress));
+        // Override the default FS to point to the router RPC
+        DistributedFileSystem.setDefaultUri(conf, fileSystemUri);
+        try {
+          this.fileContext = FileContext.getFileContext(conf);
+        } catch (UnsupportedFileSystemException e) {
+          this.fileContext = null;
+        }
+      }
     }
 
-    public DistributedFileSystem getFileSystem() throws IOException {
-      DistributedFileSystem fs =
-          (DistributedFileSystem) DistributedFileSystem.get(conf);
-      return fs;
+    public FileSystem getFileSystem() throws IOException {
+      return DistributedFileSystem.get(conf);
     }
 
     public DFSClient getClient(UserGroupInformation user)
         throws IOException, URISyntaxException, InterruptedException {
 
-      LOG.info("Connecting to router at " + fileSystemUri);
+      LOG.info("Connecting to router at {}", fileSystemUri);
       return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
         @Override
         public DFSClient run() throws IOException {
@@ -120,9 +183,8 @@ public class RouterDFSCluster {
     }
 
     public DFSClient getClient() throws IOException, URISyntaxException {
-
       if (client == null) {
-        LOG.info("Connecting to router at " + fileSystemUri);
+        LOG.info("Connecting to router at {}", fileSystemUri);
         client = new DFSClient(fileSystemUri, conf);
       }
       return client;
@@ -130,9 +192,10 @@ public class RouterDFSCluster {
   }
 
   /**
-   * Namenode context.
+   * Namenode context in the federated cluster.
    */
   public class NamenodeContext {
+    private Configuration conf;
     private NameNode namenode;
     private String nameserviceId;
     private String namenodeId;
@@ -143,14 +206,13 @@ public class RouterDFSCluster {
     private int httpPort;
     private URI fileSystemUri;
     private int index;
-    private Configuration conf;
     private DFSClient client;
 
-    public NamenodeContext(Configuration conf, String ns, String nn,
-        int index) {
+    public NamenodeContext(
+        Configuration conf, String nsId, String nnId, int index) {
       this.conf = conf;
-      this.namenodeId = nn;
-      this.nameserviceId = ns;
+      this.nameserviceId = nsId;
+      this.namenodeId = nnId;
       this.index = index;
     }
 
@@ -170,20 +232,19 @@ public class RouterDFSCluster {
       return this.fileContext;
     }
 
-    public void setNamenode(NameNode n) throws URISyntaxException {
-      namenode = n;
+    public void setNamenode(NameNode nn) throws URISyntaxException {
+      this.namenode = nn;
 
-      // Store the bound ports and override the default FS with the local NN's
-      // RPC
-      rpcPort = n.getNameNodeAddress().getPort();
-      servicePort = n.getServiceRpcAddress().getPort();
-      lifelinePort = n.getServiceRpcAddress().getPort();
-      httpPort = n.getHttpAddress().getPort();
-      fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
-      DistributedFileSystem.setDefaultUri(conf, fileSystemUri);
+      // Store the bound ports and override the default FS with the local NN RPC
+      this.rpcPort = nn.getNameNodeAddress().getPort();
+      this.servicePort = nn.getServiceRpcAddress().getPort();
+      this.lifelinePort = nn.getServiceRpcAddress().getPort();
+      this.httpPort = nn.getHttpAddress().getPort();
+      this.fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
+      DistributedFileSystem.setDefaultUri(this.conf, this.fileSystemUri);
 
       try {
-        this.fileContext = FileContext.getFileContext(conf);
+        this.fileContext = FileContext.getFileContext(this.conf);
       } catch (UnsupportedFileSystemException e) {
         this.fileContext = null;
       }
@@ -205,10 +266,8 @@ public class RouterDFSCluster {
       return namenode.getHttpAddress().getHostName() + ":" + httpPort;
     }
 
-    public DistributedFileSystem getFileSystem() throws IOException {
-      DistributedFileSystem fs =
-          (DistributedFileSystem) DistributedFileSystem.get(conf);
-      return fs;
+    public FileSystem getFileSystem() throws IOException {
+      return DistributedFileSystem.get(conf);
     }
 
     public void resetClient() {
@@ -218,7 +277,7 @@ public class RouterDFSCluster {
     public DFSClient getClient(UserGroupInformation user)
         throws IOException, URISyntaxException, InterruptedException {
 
-      LOG.info("Connecting to namenode at " + fileSystemUri);
+      LOG.info("Connecting to namenode at {}", fileSystemUri);
       return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
         @Override
         public DFSClient run() throws IOException {
@@ -229,7 +288,7 @@ public class RouterDFSCluster {
 
     public DFSClient getClient() throws IOException, URISyntaxException {
       if (client == null) {
-        LOG.info("Connecting to namenode at " + fileSystemUri);
+        LOG.info("Connecting to namenode at {}", fileSystemUri);
         client = new DFSClient(fileSystemUri, conf);
       }
       return client;
@@ -244,36 +303,20 @@ public class RouterDFSCluster {
     }
   }
 
-  public static final String NAMENODE1 = "nn0";
-  public static final String NAMENODE2 = "nn1";
-  public static final String NAMENODE3 = "nn2";
-  public static final String TEST_STRING = "teststring";
-  public static final String TEST_DIR = "testdir";
-  public static final String TEST_FILE = "testfile";
-
-  private List<String> nameservices;
-  private List<RouterContext> routers;
-  private List<NamenodeContext> namenodes;
-  private static final Log LOG = LogFactory.getLog(RouterDFSCluster.class);
-  private MiniDFSCluster cluster;
-  private boolean highAvailability;
-
-  protected static final int DEFAULT_HEARTBEAT_INTERVAL = 5;
-  protected static final int DEFAULT_CACHE_INTERVAL_SEC = 5;
-  private Configuration routerOverrides;
-  private Configuration namenodeOverrides;
-
-  private static final String NAMENODES = NAMENODE1 + "," + NAMENODE2;
-
-  public RouterDFSCluster(boolean ha, int numNameservices) {
-    this(ha, numNameservices, 2);
-  }
-
   public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
     this.highAvailability = ha;
     configureNameservices(numNameservices, numNamenodes);
   }
 
+  public RouterDFSCluster(boolean ha, int numNameservices) {
+    this(ha, numNameservices, 2);
+  }
+
+  /**
+   * Add configuration settings to override default Router settings.
+   *
+   * @param conf Router configuration overrides.
+   */
   public void addRouterOverrides(Configuration conf) {
     if (this.routerOverrides == null) {
       this.routerOverrides = conf;
@@ -282,6 +325,11 @@ public class RouterDFSCluster {
     }
   }
 
+  /**
+   * Add configuration settings to override default Namenode settings.
+   *
+   * @param conf Namenode configuration overrides.
+   */
   public void addNamenodeOverrides(Configuration conf) {
     if (this.namenodeOverrides == null) {
       this.namenodeOverrides = conf;
@@ -290,124 +338,134 @@ public class RouterDFSCluster {
     }
   }
 
-  public Configuration generateNamenodeConfiguration(
-      String defaultNameserviceId) {
-    Configuration c = new HdfsConfiguration();
+  /**
+   * Generate the configuration for a client.
+   *
+   * @param nsId Nameservice identifier.
+   * @return New namenode configuration.
+   */
+  public Configuration generateNamenodeConfiguration(String nsId) {
+    Configuration conf = new HdfsConfiguration();
 
-    c.set(DFSConfigKeys.DFS_NAMESERVICES, getNameservicesKey());
-    c.set("fs.defaultFS", "hdfs://" + defaultNameserviceId);
+    conf.set(DFS_NAMESERVICES, getNameservicesKey());
+    conf.set(FS_DEFAULT_NAME_KEY, "hdfs://" + nsId);
 
     for (String ns : nameservices) {
       if (highAvailability) {
-        c.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns, NAMENODES);
+        conf.set(
+            DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,
+            NAMENODES[0] + "," + NAMENODES[1]);
       }
 
       for (NamenodeContext context : getNamenodes(ns)) {
         String suffix = context.getConfSuffix();
 
-        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix,
+        conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix,
             "127.0.0.1:" + context.rpcPort);
-        c.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix,
+        conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix,
             "127.0.0.1:" + context.httpPort);
-        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
+        conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
             "0.0.0.0");
       }
     }
 
-    if (namenodeOverrides != null) {
-      c.addResource(namenodeOverrides);
+    if (this.namenodeOverrides != null) {
+      conf.addResource(this.namenodeOverrides);
     }
-    return c;
+    return conf;
   }
 
+  /**
+   * Generate the configuration for a client.
+   *
+   * @return New configuration for a client.
+   */
   public Configuration generateClientConfiguration() {
-    Configuration conf = new HdfsConfiguration();
-    conf.addResource(generateNamenodeConfiguration(getNameservices().get(0)));
+    Configuration conf = new HdfsConfiguration(false);
+    String ns0 = getNameservices().get(0);
+    conf.addResource(generateNamenodeConfiguration(ns0));
     return conf;
   }
 
-  public Configuration generateRouterConfiguration(String localNameserviceId,
-      String localNamenodeId) throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    conf.addResource(generateNamenodeConfiguration(localNameserviceId));
+  /**
+   * Generate the configuration for a Router.
+   *
+   * @param nsId Nameservice identifier.
+   * @param nnId Namenode identifier.
+   * @return New configuration for a Router.
+   */
+  public Configuration generateRouterConfiguration(String nsId, String nnId) {
+
+    Configuration conf = new HdfsConfiguration(false);
+    conf.addResource(generateNamenodeConfiguration(nsId));
+
+    conf.setInt(DFS_ROUTER_HANDLER_COUNT_KEY, 10);
+    conf.set(DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(DFS_ROUTER_RPC_BIND_HOST_KEY, "0.0.0.0");
+
+    conf.set(DFS_ROUTER_DEFAULT_NAMESERVICE, nameservices.get(0));
 
     // Use mock resolver classes
-    conf.set(DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
-        MockResolver.class.getCanonicalName());
-    conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
-        MockResolver.class.getCanonicalName());
+    conf.setClass(FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class, ActiveNamenodeResolver.class);
+    conf.setClass(FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class, FileSubclusterResolver.class);
 
     // Set the nameservice ID for the default NN monitor
-    conf.set(DFSConfigKeys.DFS_NAMESERVICE_ID, localNameserviceId);
-
-    if (localNamenodeId != null) {
-      conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, localNamenodeId);
+    conf.set(DFS_NAMESERVICE_ID, nsId);
+    if (nnId != null) {
+      conf.set(DFS_HA_NAMENODE_ID_KEY, nnId);
     }
 
-    StringBuilder routerBuilder = new StringBuilder();
-    for (String ns : nameservices) {
-      for (NamenodeContext context : getNamenodes(ns)) {
-        String suffix = context.getConfSuffix();
-
-        if (routerBuilder.length() != 0) {
-          routerBuilder.append(",");
-        }
-        routerBuilder.append(suffix);
+    // Add custom overrides if available
+    if (this.routerOverrides != null) {
+      for (Entry<String, String> entry : this.routerOverrides) {
+        String confKey = entry.getKey();
+        String confValue = entry.getValue();
+        conf.set(confKey, confValue);
       }
     }
-
     return conf;
   }
 
   public void configureNameservices(int numNameservices, int numNamenodes) {
-    nameservices = new ArrayList<String>();
-    for (int i = 0; i < numNameservices; i++) {
-      nameservices.add("ns" + i);
-    }
-    namenodes = new ArrayList<NamenodeContext>();
-    int index = 0;
-    for (String ns : nameservices) {
-      Configuration nnConf = generateNamenodeConfiguration(ns);
-      if (highAvailability) {
-        NamenodeContext context =
-            new NamenodeContext(nnConf, ns, NAMENODE1, index);
-        namenodes.add(context);
-        index++;
-
-        if (numNamenodes > 1) {
-          context = new NamenodeContext(nnConf, ns, NAMENODE2, index + 1);
-          namenodes.add(context);
-          index++;
-        }
+    this.nameservices = new ArrayList<>();
+    this.namenodes = new ArrayList<>();
 
-        if (numNamenodes > 2) {
-          context = new NamenodeContext(nnConf, ns, NAMENODE3, index + 1);
-          namenodes.add(context);
-          index++;
-        }
+    NamenodeContext context = null;
+    int nnIndex = 0;
+    for (int i=0; i<numNameservices; i++) {
+      String ns = "ns" + i;
+      this.nameservices.add("ns" + i);
 
+      Configuration nnConf = generateNamenodeConfiguration(ns);
+      if (!highAvailability) {
+        context = new NamenodeContext(nnConf, ns, null, nnIndex++);
+        this.namenodes.add(context);
       } else {
-        NamenodeContext context = new NamenodeContext(nnConf, ns, null, index);
-        namenodes.add(context);
-        index++;
+        for (int j=0; j<numNamenodes; j++) {
+          context = new NamenodeContext(nnConf, ns, NAMENODES[j], nnIndex++);
+          this.namenodes.add(context);
+        }
       }
     }
   }
 
   public String getNameservicesKey() {
-    StringBuilder ns = new StringBuilder();
-    for (int i = 0; i < nameservices.size(); i++) {
-      if (i > 0) {
-        ns.append(",");
+    StringBuilder sb = new StringBuilder();
+    for (String nsId : this.nameservices) {
+      if (sb.length() > 0) {
+        sb.append(",");
       }
-      ns.append(nameservices.get(i));
+      sb.append(nsId);
     }
-    return ns.toString();
+    return sb.toString();
   }
 
   public String getRandomNameservice() {
     Random r = new Random();
-    return nameservices.get(r.nextInt(nameservices.size()));
+    int randIndex = r.nextInt(nameservices.size());
+    return nameservices.get(randIndex);
   }
 
   public List<String> getNameservices() {
@@ -415,7 +473,7 @@ public class RouterDFSCluster {
   }
 
   public List<NamenodeContext> getNamenodes(String nameservice) {
-    ArrayList<NamenodeContext> nns = new ArrayList<NamenodeContext>();
+    List<NamenodeContext> nns = new ArrayList<>();
     for (NamenodeContext c : namenodes) {
       if (c.nameserviceId.equals(nameservice)) {
         nns.add(c);
@@ -426,23 +484,23 @@ public class RouterDFSCluster {
 
   public NamenodeContext getRandomNamenode() {
     Random rand = new Random();
-    return namenodes.get(rand.nextInt(namenodes.size()));
+    int i = rand.nextInt(this.namenodes.size());
+    return this.namenodes.get(i);
   }
 
   public List<NamenodeContext> getNamenodes() {
-    return namenodes;
+    return this.namenodes;
   }
 
   public boolean isHighAvailability() {
     return highAvailability;
   }
 
-  public NamenodeContext getNamenode(String nameservice,
-      String namenode) {
-    for (NamenodeContext c : namenodes) {
+  public NamenodeContext getNamenode(String nameservice, String namenode) {
+    for (NamenodeContext c : this.namenodes) {
       if (c.nameserviceId.equals(nameservice)) {
-        if (namenode == null || c.namenodeId == null || namenode.isEmpty()
-            || c.namenodeId.isEmpty()) {
+        if (namenode == null || namenode.isEmpty() ||
+            c.namenodeId == null || c.namenodeId.isEmpty()) {
           return c;
         } else if (c.namenodeId.equals(namenode)) {
           return c;
@@ -453,7 +511,7 @@ public class RouterDFSCluster {
   }
 
   public List<RouterContext> getRouters(String nameservice) {
-    ArrayList<RouterContext> nns = new ArrayList<RouterContext>();
+    List<RouterContext> nns = new ArrayList<>();
     for (RouterContext c : routers) {
       if (c.nameserviceId.equals(nameservice)) {
         nns.add(c);
@@ -462,14 +520,13 @@ public class RouterDFSCluster {
     return nns;
   }
 
-  public RouterContext getRouterContext(String nameservice,
-      String namenode) {
+  public RouterContext getRouterContext(String nsId, String nnId) {
     for (RouterContext c : routers) {
-      if (namenode == null) {
+      if (nnId == null) {
         return c;
       }
-      if (c.namenodeId.equals(namenode)
-          && c.nameserviceId.equals(nameservice)) {
+      if (c.namenodeId.equals(nnId) &&
+          c.nameserviceId.equals(nsId)) {
         return c;
       }
     }
@@ -485,10 +542,10 @@ public class RouterDFSCluster {
     return routers;
   }
 
-  public RouterContext buildRouter(String nameservice, String namenode)
+  public RouterContext buildRouter(String nsId, String nnId)
       throws URISyntaxException, IOException {
-    Configuration config = generateRouterConfiguration(nameservice, namenode);
-    RouterContext rc = new RouterContext(config, nameservice, namenode);
+    Configuration config = generateRouterConfiguration(nsId, nnId);
+    RouterContext rc = new RouterContext(config, nsId, nnId);
     return rc;
   }
 
@@ -500,10 +557,9 @@ public class RouterDFSCluster {
     try {
       MiniDFSNNTopology topology = new MiniDFSNNTopology();
       for (String ns : nameservices) {
-
         NSConf conf = new MiniDFSNNTopology.NSConf(ns);
         if (highAvailability) {
-          for(int i = 0; i < namenodes.size()/nameservices.size(); i++) {
+          for (int i=0; i<namenodes.size()/nameservices.size(); i++) {
             NNConf nnConf = new MiniDFSNNTopology.NNConf("nn" + i);
             conf.addNN(nnConf);
           }
@@ -516,11 +572,15 @@ public class RouterDFSCluster {
       topology.setFederation(true);
 
       // Start mini DFS cluster
-      Configuration nnConf = generateNamenodeConfiguration(nameservices.get(0));
+      String ns0 = nameservices.get(0);
+      Configuration nnConf = generateNamenodeConfiguration(ns0);
       if (overrideConf != null) {
         nnConf.addResource(overrideConf);
       }
-      cluster = new MiniDFSCluster.Builder(nnConf).nnTopology(topology).build();
+      cluster = new MiniDFSCluster.Builder(nnConf)
+          .numDataNodes(nameservices.size()*2)
+          .nnTopology(topology)
+          .build();
       cluster.waitActive();
 
       // Store NN pointers
@@ -530,28 +590,32 @@ public class RouterDFSCluster {
       }
 
     } catch (Exception e) {
-      LOG.error("Cannot start Router DFS cluster: " + e.getMessage(), e);
-      cluster.shutdown();
+      LOG.error("Cannot start Router DFS cluster: {}", e.getMessage(), e);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 
   public void startRouters()
       throws InterruptedException, URISyntaxException, IOException {
-    // Create routers
-    routers = new ArrayList<RouterContext>();
-    for (String ns : nameservices) {
 
+    // Create one router per nameservice
+    this.routers = new ArrayList<>();
+    for (String ns : this.nameservices) {
       for (NamenodeContext context : getNamenodes(ns)) {
-        routers.add(buildRouter(ns, context.namenodeId));
+        RouterContext router = buildRouter(ns, context.namenodeId);
+        this.routers.add(router);
       }
     }
 
     // Start all routers
-    for (RouterContext router : routers) {
+    for (RouterContext router : this.routers) {
       router.router.start();
     }
+
     // Wait until all routers are active and record their ports
-    for (RouterContext router : routers) {
+    for (RouterContext router : this.routers) {
       waitActive(router);
       router.initRouter();
     }
@@ -570,22 +634,21 @@ public class RouterDFSCluster {
       }
       Thread.sleep(1000);
     }
-    assertFalse(
-        "Timeout waiting for " + router.router.toString() + " to activate.",
-        true);
+    fail("Timeout waiting for " + router.router + " to activate");
   }
 
-
   public void registerNamenodes() throws IOException {
-    for (RouterContext r : routers) {
+    for (RouterContext r : this.routers) {
       ActiveNamenodeResolver resolver = r.router.getNamenodeResolver();
-      for (NamenodeContext nn : namenodes) {
+      for (NamenodeContext nn : this.namenodes) {
         // Generate a report
-        NamenodeStatusReport report = new NamenodeStatusReport(nn.nameserviceId,
-            nn.namenodeId, nn.getRpcAddress(), nn.getServiceAddress(),
+        NamenodeStatusReport report = new NamenodeStatusReport(
+            nn.nameserviceId, nn.namenodeId,
+            nn.getRpcAddress(), nn.getServiceAddress(),
             nn.getLifelineAddress(), nn.getHttpAddress());
-        report.setNamespaceInfo(nn.namenode.getNamesystem().getFSImage()
-            .getStorage().getNamespaceInfo());
+        FSImage fsImage = nn.namenode.getNamesystem().getFSImage();
+        NamespaceInfo nsInfo = fsImage.getStorage().getNamespaceInfo();
+        report.setNamespaceInfo(nsInfo);
 
         // Determine HA state from nn public state string
         String nnState = nn.namenode.getState();
@@ -606,74 +669,97 @@ public class RouterDFSCluster {
 
   public void waitNamenodeRegistration()
       throws InterruptedException, IllegalStateException, IOException {
-    for (RouterContext r : routers) {
-      for (NamenodeContext nn : namenodes) {
-        FederationTestUtils.waitNamenodeRegistered(
-            r.router.getNamenodeResolver(), nn.nameserviceId, nn.namenodeId,
-            null);
+    for (RouterContext r : this.routers) {
+      Router router = r.router;
+      for (NamenodeContext nn : this.namenodes) {
+        ActiveNamenodeResolver nnResolver = router.getNamenodeResolver();
+        waitNamenodeRegistered(
+            nnResolver, nn.nameserviceId, nn.namenodeId, null);
       }
     }
   }
 
   public void waitRouterRegistrationQuorum(RouterContext router,
-      FederationNamenodeServiceState state, String nameservice, String namenode)
+      FederationNamenodeServiceState state, String nsId, String nnId)
           throws InterruptedException, IOException {
-    LOG.info("Waiting for NN - " + nameservice + ":" + namenode
-        + " to transition to state - " + state);
-    FederationTestUtils.waitNamenodeRegistered(
-        router.router.getNamenodeResolver(), nameservice, namenode, state);
+    LOG.info("Waiting for NN {} {} to transition to {}", nsId, nnId, state);
+    ActiveNamenodeResolver nnResolver = router.router.getNamenodeResolver();
+    waitNamenodeRegistered(nnResolver, nsId, nnId, state);
   }
 
-  public String getFederatedPathForNameservice(String ns) {
-    return "/" + ns;
+  /**
+   * Get the federated path for a nameservice.
+   * @param nsId Nameservice identifier.
+   * @return Path in the Router.
+   */
+  public String getFederatedPathForNS(String nsId) {
+    return "/" + nsId;
   }
 
-  public String getNamenodePathForNameservice(String ns) {
-    return "/target-" + ns;
+  /**
+   * Get the namenode path for a nameservice.
+   * @param nsId Nameservice identifier.
+   * @return Path in the Namenode.
+   */
+  public String getNamenodePathForNS(String nsId) {
+    return "/target-" + nsId;
   }
 
   /**
-   * @return example:
+   * Get the federated test directory for a nameservice.
+   * @param nsId Nameservice identifier.
+   * @return Example:
    *         <ul>
    *         <li>/ns0/testdir which maps to ns0->/target-ns0/testdir
    *         </ul>
    */
-  public String getFederatedTestDirectoryForNameservice(String ns) {
-    return getFederatedPathForNameservice(ns) + "/" + TEST_DIR;
+  public String getFederatedTestDirectoryForNS(String nsId) {
+    return getFederatedPathForNS(nsId) + "/" + TEST_DIR;
   }
 
   /**
+   * Get the namenode test directory for a nameservice.
+   * @param nsId Nameservice identifier.
    * @return example:
    *         <ul>
    *         <li>/target-ns0/testdir
    *         </ul>
    */
-  public String getNamenodeTestDirectoryForNameservice(String ns) {
-    return getNamenodePathForNameservice(ns) + "/" + TEST_DIR;
+  public String getNamenodeTestDirectoryForNS(String nsId) {
+    return getNamenodePathForNS(nsId) + "/" + TEST_DIR;
   }
 
   /**
+   * Get the federated test file for a nameservice.
+   * @param nsId Nameservice identifier.
    * @return example:
    *         <ul>
    *         <li>/ns0/testfile which maps to ns0->/target-ns0/testfile
    *         </ul>
    */
-  public String getFederatedTestFileForNameservice(String ns) {
-    return getFederatedPathForNameservice(ns) + "/" + TEST_FILE;
+  public String getFederatedTestFileForNS(String nsId) {
+    return getFederatedPathForNS(nsId) + "/" + TEST_FILE;
   }
 
   /**
+   * Get the namenode test file for a nameservice.
+   * @param nsId Nameservice identifier.
    * @return example:
    *         <ul>
    *         <li>/target-ns0/testfile
    *         </ul>
    */
-  public String getNamenodeTestFileForNameservice(String ns) {
-    return getNamenodePathForNameservice(ns) + "/" + TEST_FILE;
+  public String getNamenodeTestFileForNS(String nsId) {
+    return getNamenodePathForNS(nsId) + "/" + TEST_FILE;
   }
 
+  /**
+   * Stop the federated HDFS cluster.
+   */
   public void shutdown() {
-    cluster.shutdown();
+    if (cluster != null) {
+      cluster.shutdown();
+    }
     if (routers != null) {
       for (RouterContext context : routers) {
         stopRouter(context);
@@ -681,9 +767,12 @@ public class RouterDFSCluster {
     }
   }
 
+  /**
+   * Stop a router.
+   * @param router Router context.
+   */
   public void stopRouter(RouterContext router) {
     try {
-
       router.router.shutDown();
 
       int loopCount = 0;
@@ -691,7 +780,7 @@ public class RouterDFSCluster {
         loopCount++;
         Thread.sleep(1000);
         if (loopCount > 20) {
-          LOG.error("Unable to shutdown router - " + router.rpcPort);
+          LOG.error("Cannot shutdown router {}", router.rpcPort);
           break;
         }
       }
@@ -714,26 +803,28 @@ public class RouterDFSCluster {
     for (String ns : getNameservices()) {
       NamenodeContext context = getNamenode(ns, null);
       if (!createTestDirectoriesNamenode(context)) {
-        throw new IOException("Unable to create test directory for ns - " + ns);
+        throw new IOException("Cannot create test directory for ns " + ns);
       }
     }
   }
 
   public boolean createTestDirectoriesNamenode(NamenodeContext nn)
       throws IOException {
-    return FederationTestUtils.addDirectory(nn.getFileSystem(),
-        getNamenodeTestDirectoryForNameservice(nn.nameserviceId));
+    FileSystem fs = nn.getFileSystem();
+    String testDir = getNamenodeTestDirectoryForNS(nn.nameserviceId);
+    return addDirectory(fs, testDir);
   }
 
   public void deleteAllFiles() throws IOException {
     // Delete all files via the NNs and verify
     for (NamenodeContext context : getNamenodes()) {
-      FileStatus[] status = context.getFileSystem().listStatus(new Path("/"));
-      for(int i = 0; i <status.length; i++) {
+      FileSystem fs = context.getFileSystem();
+      FileStatus[] status = fs.listStatus(new Path("/"));
+      for (int i = 0; i <status.length; i++) {
         Path p = status[i].getPath();
-        context.getFileSystem().delete(p, true);
+        fs.delete(p, true);
       }
-      status = context.getFileSystem().listStatus(new Path("/"));
+      status = fs.listStatus(new Path("/"));
       assertEquals(status.length, 0);
     }
   }
@@ -754,14 +845,34 @@ public class RouterDFSCluster {
       MockResolver resolver =
           (MockResolver) r.router.getSubclusterResolver();
       // create table entries
-      for (String ns : nameservices) {
+      for (String nsId : nameservices) {
         // Direct path
-        resolver.addLocation(getFederatedPathForNameservice(ns), ns,
-            getNamenodePathForNameservice(ns));
+        String routerPath = getFederatedPathForNS(nsId);
+        String nnPath = getNamenodePathForNS(nsId);
+        resolver.addLocation(routerPath, nsId, nnPath);
       }
 
-      // Root path goes to both NS1
-      resolver.addLocation("/", nameservices.get(0), "/");
+      // Root path points to both first nameservice
+      String ns0 = nameservices.get(0);
+      resolver.addLocation("/", ns0, "/");
+    }
+  }
+
+  public MiniDFSCluster getCluster() {
+    return cluster;
+  }
+
+  /**
+   * Wait until the federated cluster is up and ready.
+   * @throws IOException If we cannot wait for the cluster to be up.
+   */
+  public void waitClusterUp() throws IOException {
+    cluster.waitClusterUp();
+    registerNamenodes();
+    try {
+      waitNamenodeRegistration();
+    } catch (Exception e) {
+      throw new IOException("Cannot wait for the namenodes", e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
index 8c720c7..d8afb39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation.router;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
@@ -51,6 +52,10 @@ public class TestRouter {
     conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
         MockResolver.class.getCanonicalName());
 
+    // Bind to any available port
+    conf.set(DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY, "0.0.0.0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0");
+
     // Simulate a co-located NN
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns0");
     conf.set("fs.defaultFS", "hdfs://" + "ns0");
@@ -90,7 +95,31 @@ public class TestRouter {
   @Test
   public void testRouterService() throws InterruptedException, IOException {
 
+    // Rpc only
+    testRouterStartup(new RouterConfigBuilder(conf).rpc().build());
+
     // Run with all services
-    testRouterStartup((new RouterConfigBuilder(conf)).build());
+    testRouterStartup(new RouterConfigBuilder(conf).all().build());
+  }
+
+  @Test
+  public void testRouterRestartRpcService() throws IOException {
+
+    // Start
+    Router router = new Router();
+    router.init(new RouterConfigBuilder(conf).rpc().build());
+    router.start();
+
+    // Verify RPC server is running
+    assertNotNull(router.getRpcServerAddress());
+    RouterRpcServer rpcServer = router.getRpcServer();
+    assertNotNull(rpcServer);
+    assertEquals(STATE.STARTED, rpcServer.getServiceState());
+
+    // Stop router and RPC server
+    router.stop();
+    assertEquals(STATE.STOPPED, rpcServer.getServiceState());
+
+    router.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
new file mode 100644
index 0000000..af506c9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
@@ -0,0 +1,869 @@
+/**
+ * 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.apache.hadoop.hdfs.server.federation.FederationTestUtils.addDirectory;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.countContents;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.createFile;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.deleteFile;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.getFileStatus;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
+import static org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.TEST_STRING;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.Service.STATE;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The the RPC interface of the {@link Router} implemented by
+ * {@link RouterRpcServer}.
+ */
+public class TestRouterRpc {
+
+  /** Federated HDFS cluster. */
+  private static RouterDFSCluster cluster;
+
+  /** Random Router for this federated cluster. */
+  private RouterContext router;
+
+  /** Random nameservice in the federated cluster.  */
+  private String ns;
+  /** First namenode in the nameservice. */
+  private NamenodeContext namenode;
+
+  /** Client interface to the Router. */
+  private ClientProtocol routerProtocol;
+  /** Client interface to the Namenode. */
+  private ClientProtocol nnProtocol;
+
+  /** Filesystem interface to the Router. */
+  private FileSystem routerFS;
+  /** Filesystem interface to the Namenode. */
+  private FileSystem nnFS;
+
+  /** File in the Router. */
+  private String routerFile;
+  /** File in the Namenode. */
+  private String nnFile;
+
+
+  @BeforeClass
+  public static void globalSetUp() throws Exception {
+    cluster = new RouterDFSCluster(false, 2);
+
+    // Start NNs and DNs and wait until ready
+    cluster.startCluster();
+
+    // Start routers with only an RPC service
+    cluster.addRouterOverrides((new RouterConfigBuilder()).rpc().build());
+    cluster.startRouters();
+
+    // Register and verify all NNs with all routers
+    cluster.registerNamenodes();
+    cluster.waitNamenodeRegistration();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    cluster.shutdown();
+  }
+
+  @Before
+  public void testSetup() throws Exception {
+
+    // Create mock locations
+    cluster.installMockLocations();
+
+    // Delete all files via the NNs and verify
+    cluster.deleteAllFiles();
+
+    // Create test fixtures on NN
+    cluster.createTestDirectoriesNamenode();
+
+    // Wait to ensure NN has fully created its test directories
+    Thread.sleep(100);
+
+    // Pick a NS, namenode and router for this test
+    this.router = cluster.getRandomRouter();
+    this.ns = cluster.getRandomNameservice();
+    this.namenode = cluster.getNamenode(ns, null);
+
+    // Handles to the ClientProtocol interface
+    this.routerProtocol = router.getClient().getNamenode();
+    this.nnProtocol = namenode.getClient().getNamenode();
+
+    // Handles to the filesystem client
+    this.nnFS = namenode.getFileSystem();
+    this.routerFS = router.getFileSystem();
+
+    // Create a test file on the NN
+    Random r = new Random();
+    String randomFile = "testfile-" + r.nextInt();
+    this.nnFile =
+        cluster.getNamenodeTestDirectoryForNS(ns) + "/" + randomFile;
+    this.routerFile =
+        cluster.getFederatedTestDirectoryForNS(ns) + "/" + randomFile;
+
+    createFile(nnFS, nnFile, 32);
+    verifyFileExists(nnFS, nnFile);
+  }
+
+  @Test
+  public void testRpcService() throws IOException {
+    Router testRouter = new Router();
+    List<String> nss = cluster.getNameservices();
+    String ns0 = nss.get(0);
+    Configuration routerConfig = cluster.generateRouterConfiguration(ns0, null);
+    RouterRpcServer server = new RouterRpcServer(routerConfig, testRouter,
+        testRouter.getNamenodeResolver(), testRouter.getSubclusterResolver());
+    server.init(routerConfig);
+    assertEquals(STATE.INITED, server.getServiceState());
+    server.start();
+    assertEquals(STATE.STARTED, server.getServiceState());
+    server.stop();
+    assertEquals(STATE.STOPPED, server.getServiceState());
+    server.close();
+    testRouter.close();
+  }
+
+  protected RouterDFSCluster getCluster() {
+    return TestRouterRpc.cluster;
+  }
+
+  protected RouterContext getRouterContext() {
+    return this.router;
+  }
+
+  protected void setRouter(RouterContext r)
+      throws IOException, URISyntaxException {
+    this.router = r;
+    this.routerProtocol = r.getClient().getNamenode();
+    this.routerFS = r.getFileSystem();
+  }
+
+  protected FileSystem getRouterFileSystem() {
+    return this.routerFS;
+  }
+
+  protected FileSystem getNamenodeFileSystem() {
+    return this.nnFS;
+  }
+
+  protected ClientProtocol getRouterProtocol() {
+    return this.routerProtocol;
+  }
+
+  protected ClientProtocol getNamenodeProtocol() {
+    return this.nnProtocol;
+  }
+
+  protected NamenodeContext getNamenode() {
+    return this.namenode;
+  }
+
+  protected void setNamenodeFile(String filename) {
+    this.nnFile = filename;
+  }
+
+  protected String getNamenodeFile() {
+    return this.nnFile;
+  }
+
+  protected void setRouterFile(String filename) {
+    this.routerFile = filename;
+  }
+
+  protected String getRouterFile() {
+    return this.routerFile;
+  }
+
+  protected void setNamenode(NamenodeContext nn)
+      throws IOException, URISyntaxException {
+    this.namenode = nn;
+    this.nnProtocol = nn.getClient().getNamenode();
+    this.nnFS = nn.getFileSystem();
+  }
+
+  protected String getNs() {
+    return this.ns;
+  }
+
+  protected void setNs(String nameservice) {
+    this.ns = nameservice;
+  }
+
+  protected static void compareResponses(
+      ClientProtocol protocol1, ClientProtocol protocol2,
+      Method m, Object[] paramList) {
+
+    Object return1 = null;
+    Exception exception1 = null;
+    try {
+      return1 = m.invoke(protocol1, paramList);
+    } catch (Exception ex) {
+      exception1 = ex;
+    }
+
+    Object return2 = null;
+    Exception exception2 = null;
+    try {
+      return2 = m.invoke(protocol2, paramList);
+    } catch (Exception ex) {
+      exception2 = ex;
+    }
+
+    assertEquals(return1, return2);
+    if (exception1 == null && exception2 == null) {
+      return;
+    }
+
+    assertEquals(
+        exception1.getCause().getClass(),
+        exception2.getCause().getClass());
+  }
+
+  @Test
+  public void testProxyListFiles() throws IOException, InterruptedException,
+      URISyntaxException, NoSuchMethodException, SecurityException {
+
+    // Verify that the root listing is a union of the mount table destinations
+    // and the files stored at all nameservices mounted at the root (ns0 + ns1)
+    //
+    // / -->
+    // /ns0 (from mount table)
+    // /ns1 (from mount table)
+    // all items in / of ns0 (default NS)
+
+    // Collect the mount table entries from the root mount point
+    Set<String> requiredPaths = new TreeSet<>();
+    FileSubclusterResolver fileResolver =
+        router.getRouter().getSubclusterResolver();
+    for (String mount : fileResolver.getMountPoints("/")) {
+      requiredPaths.add(mount);
+    }
+
+    // Collect all files/dirs on the root path of the default NS
+    String defaultNs = cluster.getNameservices().get(0);
+    NamenodeContext nn = cluster.getNamenode(defaultNs, null);
+    FileStatus[] iterator = nn.getFileSystem().listStatus(new Path("/"));
+    for (FileStatus file : iterator) {
+      requiredPaths.add(file.getPath().getName());
+    }
+
+    // Fetch listing
+    DirectoryListing listing =
+        routerProtocol.getListing("/", HdfsFileStatus.EMPTY_NAME, false);
+    Iterator<String> requiredPathsIterator = requiredPaths.iterator();
+    // Match each path returned and verify order returned
+    for(HdfsFileStatus f : listing.getPartialListing()) {
+      String fileName = requiredPathsIterator.next();
+      String currentFile = f.getFullPath(new Path("/")).getName();
+      assertEquals(currentFile, fileName);
+    }
+
+    // Verify the total number of results found/matched
+    assertEquals(requiredPaths.size(), listing.getPartialListing().length);
+
+    // List a path that doesn't exist and validate error response with NN
+    // behavior.
+    Method m = ClientProtocol.class.getMethod(
+        "getListing", String.class, byte[].class, boolean.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, HdfsFileStatus.EMPTY_NAME, false});
+  }
+
+  @Test
+  public void testProxyListFilesWithConflict()
+      throws IOException, InterruptedException {
+
+    // Add a directory to the namespace that conflicts with a mount point
+    NamenodeContext nn = cluster.getNamenode(ns, null);
+    FileSystem nnFs = nn.getFileSystem();
+    addDirectory(nnFs, cluster.getFederatedTestDirectoryForNS(ns));
+
+    FileSystem routerFs = router.getFileSystem();
+    int initialCount = countContents(routerFs, "/");
+
+    // Root file system now for NS X:
+    // / ->
+    // /ns0 (mount table)
+    // /ns1 (mount table)
+    // /target-ns0 (the target folder for the NS0 mapped to /
+    // /nsX (local directory that duplicates mount table)
+    int newCount = countContents(routerFs, "/");
+    assertEquals(initialCount, newCount);
+
+    // Verify that each root path is readable and contains one test directory
+    assertEquals(1, countContents(routerFs, cluster.getFederatedPathForNS(ns)));
+
+    // Verify that real folder for the ns contains a single test directory
+    assertEquals(1, countContents(nnFs, cluster.getNamenodePathForNS(ns)));
+
+  }
+
+  protected void testRename(RouterContext testRouter, String filename,
+      String renamedFile, boolean exceptionExpected) throws IOException {
+
+    createFile(testRouter.getFileSystem(), filename, 32);
+    // verify
+    verifyFileExists(testRouter.getFileSystem(), filename);
+    // rename
+    boolean exceptionThrown = false;
+    try {
+      DFSClient client = testRouter.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      clientProtocol.rename(filename, renamedFile);
+    } catch (Exception ex) {
+      exceptionThrown = true;
+    }
+    if (exceptionExpected) {
+      // Error was expected
+      assertTrue(exceptionThrown);
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(filename), true));
+    } else {
+      // No error was expected
+      assertFalse(exceptionThrown);
+      // verify
+      assertTrue(verifyFileExists(testRouter.getFileSystem(), renamedFile));
+      // delete
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(renamedFile), true));
+    }
+  }
+
+  protected void testRename2(RouterContext testRouter, String filename,
+      String renamedFile, boolean exceptionExpected) throws IOException {
+    createFile(testRouter.getFileSystem(), filename, 32);
+    // verify
+    verifyFileExists(testRouter.getFileSystem(), filename);
+    // rename
+    boolean exceptionThrown = false;
+    try {
+      DFSClient client = testRouter.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      clientProtocol.rename2(filename, renamedFile, new Options.Rename[] {});
+    } catch (Exception ex) {
+      exceptionThrown = true;
+    }
+    assertEquals(exceptionExpected, exceptionThrown);
+    if (exceptionExpected) {
+      // Error was expected
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(filename), true));
+    } else {
+      // verify
+      assertTrue(verifyFileExists(testRouter.getFileSystem(), renamedFile));
+      // delete
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(renamedFile), true));
+    }
+  }
+
+  @Test
+  public void testProxyRenameFiles() throws IOException, InterruptedException {
+
+    Thread.sleep(5000);
+    List<String> nss = cluster.getNameservices();
+    String ns0 = nss.get(0);
+    String ns1 = nss.get(1);
+
+    // Rename within the same namespace
+    // /ns0/testdir/testrename -> /ns0/testdir/testrename-append
+    String filename =
+        cluster.getFederatedTestDirectoryForNS(ns0) + "/testrename";
+    String renamedFile = filename + "-append";
+    testRename(router, filename, renamedFile, false);
+    testRename2(router, filename, renamedFile, false);
+
+    // Rename a file to a destination that is in a different namespace (fails)
+    filename = cluster.getFederatedTestDirectoryForNS(ns0) + "/testrename";
+    renamedFile = cluster.getFederatedTestDirectoryForNS(ns1) + "/testrename";
+    testRename(router, filename, renamedFile, true);
+    testRename2(router, filename, renamedFile, true);
+  }
+
+  @Test
+  public void testProxyChownFiles() throws Exception {
+
+    String newUsername = "TestUser";
+    String newGroup = "TestGroup";
+
+    // change owner
+    routerProtocol.setOwner(routerFile, newUsername, newGroup);
+
+    // Verify with NN
+    FileStatus file = getFileStatus(namenode.getFileSystem(), nnFile);
+    assertEquals(file.getOwner(), newUsername);
+    assertEquals(file.getGroup(), newGroup);
+
+    // Bad request and validate router response matches NN response.
+    Method m = ClientProtocol.class.getMethod("setOwner", String.class,
+        String.class, String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, newUsername, newGroup});
+  }
+
+  @Test
+  public void testProxyGetStats() throws Exception {
+
+    long[] combinedData = routerProtocol.getStats();
+
+    long[] individualData = new long[10];
+    for (String nameservice : cluster.getNameservices()) {
+      NamenodeContext n = cluster.getNamenode(nameservice, null);
+      DFSClient client = n.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      long[] data = clientProtocol.getStats();
+      for (int i = 0; i < data.length; i++) {
+        individualData[i] += data[i];
+      }
+      assert(data.length == combinedData.length);
+    }
+
+    for (int i = 0; i < combinedData.length && i < individualData.length; i++) {
+      if (i == ClientProtocol.GET_STATS_REMAINING_IDX) {
+        // Skip available storage as this fluctuates in mini cluster
+        continue;
+      }
+      assertEquals(combinedData[i], individualData[i]);
+    }
+  }
+
+  @Test
+  public void testProxyGetDatanodeReport() throws Exception {
+
+    DatanodeInfo[] combinedData =
+        routerProtocol.getDatanodeReport(DatanodeReportType.ALL);
+
+    Set<Integer> individualData = new HashSet<Integer>();
+    for (String nameservice : cluster.getNameservices()) {
+      NamenodeContext n = cluster.getNamenode(nameservice, null);
+      DFSClient client = n.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      DatanodeInfo[] data =
+          clientProtocol.getDatanodeReport(DatanodeReportType.ALL);
+      for (int i = 0; i < data.length; i++) {
+        // Collect unique DNs based on their xfer port
+        DatanodeInfo info = data[i];
+        individualData.add(info.getXferPort());
+      }
+    }
+    assertEquals(combinedData.length, individualData.size());
+  }
+
+  @Test
+  public void testProxyGetDatanodeStorageReport()
+      throws IOException, InterruptedException, URISyntaxException {
+
+    DatanodeStorageReport[] combinedData =
+        routerProtocol.getDatanodeStorageReport(DatanodeReportType.ALL);
+
+    Set<String> individualData = new HashSet<>();
+    for (String nameservice : cluster.getNameservices()) {
+      NamenodeContext n = cluster.getNamenode(nameservice, null);
+      DFSClient client = n.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      DatanodeStorageReport[] data =
+          clientProtocol.getDatanodeStorageReport(DatanodeReportType.ALL);
+      for (DatanodeStorageReport report : data) {
+        // Determine unique DN instances
+        DatanodeInfo dn = report.getDatanodeInfo();
+        individualData.add(dn.toString());
+      }
+    }
+    assertEquals(combinedData.length, individualData.size());
+  }
+
+  @Test
+  public void testProxyMkdir() throws Exception {
+
+    // Check the initial folders
+    FileStatus[] filesInitial = routerFS.listStatus(new Path("/"));
+
+    // Create a directory via the router at the root level
+    String dirPath = "/testdir";
+    FsPermission permission = new FsPermission("705");
+    routerProtocol.mkdirs(dirPath, permission, false);
+
+    // Verify the root listing has the item via the router
+    FileStatus[] files = routerFS.listStatus(new Path("/"));
+    assertEquals(Arrays.toString(files) + " should be " +
+        Arrays.toString(filesInitial) + " + " + dirPath,
+        filesInitial.length + 1, files.length);
+    assertTrue(verifyFileExists(routerFS, dirPath));
+
+    // Verify the directory is present in only 1 Namenode
+    int foundCount = 0;
+    for (NamenodeContext n : cluster.getNamenodes()) {
+      if (verifyFileExists(n.getFileSystem(), dirPath)) {
+        foundCount++;
+      }
+    }
+    assertEquals(1, foundCount);
+    assertTrue(deleteFile(routerFS, dirPath));
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod("mkdirs", String.class,
+        FsPermission.class, boolean.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, permission, false});
+  }
+
+  @Test
+  public void testProxyChmodFiles() throws Exception {
+
+    FsPermission permission = new FsPermission("444");
+
+    // change permissions
+    routerProtocol.setPermission(routerFile, permission);
+
+    // Validate permissions NN
+    FileStatus file = getFileStatus(namenode.getFileSystem(), nnFile);
+    assertEquals(permission, file.getPermission());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "setPermission", String.class, FsPermission.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, permission});
+  }
+
+  @Test
+  public void testProxySetReplication() throws Exception {
+
+    // Check current replication via NN
+    FileStatus file = getFileStatus(nnFS, nnFile);
+    assertEquals(1, file.getReplication());
+
+    // increment replication via router
+    routerProtocol.setReplication(routerFile, (short) 2);
+
+    // Verify via NN
+    file = getFileStatus(nnFS, nnFile);
+    assertEquals(2, file.getReplication());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "setReplication", String.class, short.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, (short) 2});
+  }
+
+  @Test
+  public void testProxyTruncateFile() throws Exception {
+
+    // Check file size via NN
+    FileStatus file = getFileStatus(nnFS, nnFile);
+    assertTrue(file.getLen() > 0);
+
+    // Truncate to 0 bytes via router
+    routerProtocol.truncate(routerFile, 0, "testclient");
+
+    // Verify via NN
+    file = getFileStatus(nnFS, nnFile);
+    assertEquals(0, file.getLen());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "truncate", String.class, long.class, String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, (long) 0, "testclient"});
+  }
+
+  @Test
+  public void testProxyGetBlockLocations() throws Exception {
+
+    // Fetch block locations via router
+    LocatedBlocks locations =
+        routerProtocol.getBlockLocations(routerFile, 0, 1024);
+    assertEquals(1, locations.getLocatedBlocks().size());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "getBlockLocations", String.class, long.class, long.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol,
+        m, new Object[] {badPath, (long) 0, (long) 0});
+  }
+
+  @Test
+  public void testProxyStoragePolicy() throws Exception {
+
+    // Query initial policy via NN
+    HdfsFileStatus status = namenode.getClient().getFileInfo(nnFile);
+
+    // Set a random policy via router
+    BlockStoragePolicy[] policies = namenode.getClient().getStoragePolicies();
+    BlockStoragePolicy policy = policies[0];
+
+    while (policy.isCopyOnCreateFile()) {
+      // Pick a non copy on create policy
+      Random rand = new Random();
+      int randIndex = rand.nextInt(policies.length);
+      policy = policies[randIndex];
+    }
+    routerProtocol.setStoragePolicy(routerFile, policy.getName());
+
+    // Verify policy via NN
+    HdfsFileStatus newStatus = namenode.getClient().getFileInfo(nnFile);
+    assertTrue(newStatus.getStoragePolicy() == policy.getId());
+    assertTrue(newStatus.getStoragePolicy() != status.getStoragePolicy());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod("setStoragePolicy", String.class,
+        String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol,
+        m, new Object[] {badPath, "badpolicy"});
+  }
+
+  @Test
+  public void testProxyGetPreferedBlockSize() throws Exception {
+
+    // Query via NN and Router and verify
+    long namenodeSize = nnProtocol.getPreferredBlockSize(nnFile);
+    long routerSize = routerProtocol.getPreferredBlockSize(routerFile);
+    assertEquals(routerSize, namenodeSize);
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "getPreferredBlockSize", String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(
+        routerProtocol, nnProtocol, m, new Object[] {badPath});
+  }
+
+  private void testConcat(
+      String source, String target, boolean failureExpected) {
+    boolean failure = false;
+    try {
+      // Concat test file with fill block length file via router
+      routerProtocol.concat(target, new String[] {source});
+    } catch (IOException ex) {
+      failure = true;
+    }
+    assertEquals(failureExpected, failure);
+  }
+
+  @Test
+  public void testProxyConcatFile() throws Exception {
+
+    // Create a stub file in the primary ns
+    String sameNameservice = ns;
+    String existingFile =
+        cluster.getFederatedTestDirectoryForNS(sameNameservice) +
+        "_concatfile";
+    int existingFileSize = 32;
+    createFile(routerFS, existingFile, existingFileSize);
+
+    // Identify an alternate nameservice that doesn't match the existing file
+    String alternateNameservice = null;
+    for (String n : cluster.getNameservices()) {
+      if (!n.equals(sameNameservice)) {
+        alternateNameservice = n;
+        break;
+      }
+    }
+
+    // Create new files, must be a full block to use concat. One file is in the
+    // same namespace as the target file, the other is in a different namespace.
+    String altRouterFile =
+        cluster.getFederatedTestDirectoryForNS(alternateNameservice) +
+        "_newfile";
+    String sameRouterFile =
+        cluster.getFederatedTestDirectoryForNS(sameNameservice) +
+        "_newfile";
+    createFile(routerFS, altRouterFile, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
+    createFile(routerFS, sameRouterFile, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
+
+    // Concat in different namespaces, fails
+    testConcat(existingFile, altRouterFile, true);
+
+    // Concat in same namespaces, succeeds
+    testConcat(existingFile, sameRouterFile, false);
+
+    // Check target file length
+    FileStatus status = getFileStatus(routerFS, sameRouterFile);
+    assertEquals(
+        existingFileSize + DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT,
+        status.getLen());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "concat", String.class, String[].class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, new String[] {routerFile}});
+  }
+
+  @Test
+  public void testProxyAppend() throws Exception {
+
+    // Append a test string via router
+    EnumSet<CreateFlag> createFlag = EnumSet.of(CreateFlag.APPEND);
+    DFSClient routerClient = getRouterContext().getClient();
+    HdfsDataOutputStream stream =
+        routerClient.append(routerFile, 1024, createFlag, null, null);
+    stream.writeBytes(TEST_STRING);
+    stream.close();
+
+    // Verify file size via NN
+    FileStatus status = getFileStatus(nnFS, nnFile);
+    assertTrue(status.getLen() > TEST_STRING.length());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod("append", String.class,
+        String.class, EnumSetWritable.class);
+    String badPath = "/unknownlocation/unknowndir";
+    EnumSetWritable<CreateFlag> createFlagWritable =
+        new EnumSetWritable<CreateFlag>(createFlag);
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, "testClient", createFlagWritable});
+  }
+
+  @Test
+  public void testProxyGetAdditionalDatanode()
+      throws IOException, InterruptedException, URISyntaxException {
+
+    // Use primitive APIs to open a file, add a block, and get datanode location
+    EnumSet<CreateFlag> createFlag = EnumSet.of(CreateFlag.CREATE);
+    String clientName = getRouterContext().getClient().getClientName();
+    String newRouterFile = routerFile + "_additionalDatanode";
+    HdfsFileStatus status = routerProtocol.create(
+        newRouterFile, new FsPermission("777"), clientName,
+        new EnumSetWritable<CreateFlag>(createFlag), true, (short) 1,
+        (long) 1024, CryptoProtocolVersion.supported(), null);
+
+    // Add a block via router (requires client to have same lease)
+    LocatedBlock block = routerProtocol.addBlock(
+        newRouterFile, clientName, null, null,
+        status.getFileId(), null, null);
+
+    DatanodeInfo[] exclusions = new DatanodeInfo[0];
+    LocatedBlock newBlock = routerProtocol.getAdditionalDatanode(
+        newRouterFile, status.getFileId(), block.getBlock(),
+        block.getLocations(), block.getStorageIDs(), exclusions, 1, clientName);
+    assertNotNull(newBlock);
+  }
+
+  @Test
+  public void testProxyCreateFileAlternateUser()
+      throws IOException, URISyntaxException, InterruptedException {
+
+    // Create via Router
+    String routerDir = cluster.getFederatedTestDirectoryForNS(ns);
+    String namenodeDir = cluster.getNamenodeTestDirectoryForNS(ns);
+    String newRouterFile = routerDir + "/unknownuser";
+    String newNamenodeFile = namenodeDir + "/unknownuser";
+    String username = "unknownuser";
+
+    // Allow all user access to dir
+    namenode.getFileContext().setPermission(
+        new Path(namenodeDir), new FsPermission("777"));
+
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser(username);
+    DFSClient client = getRouterContext().getClient(ugi);
+    client.create(newRouterFile, true);
+
+    // Fetch via NN and check user
+    FileStatus status = getFileStatus(nnFS, newNamenodeFile);
+    assertEquals(status.getOwner(), username);
+  }
+
+  @Test
+  public void testProxyGetFileInfoAcessException() throws IOException {
+
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser("unknownuser");
+
+    // List files from the NN and trap the exception
+    Exception nnFailure = null;
+    try {
+      String testFile = cluster.getNamenodeTestFileForNS(ns);
+      namenode.getClient(ugi).getLocatedBlocks(testFile, 0);
+    } catch (Exception e) {
+      nnFailure = e;
+    }
+    assertNotNull(nnFailure);
+
+    // List files from the router and trap the exception
+    Exception routerFailure = null;
+    try {
+      String testFile = cluster.getFederatedTestFileForNS(ns);
+      getRouterContext().getClient(ugi).getLocatedBlocks(testFile, 0);
+    } catch (Exception e) {
+      routerFailure = e;
+    }
+    assertNotNull(routerFailure);
+
+    assertEquals(routerFailure.getClass(), nnFailure.getClass());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b381a9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java
new file mode 100644
index 0000000..5489691
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java
@@ -0,0 +1,216 @@
+/**
+ * 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.apache.hadoop.hdfs.server.federation.FederationTestUtils.createFile;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.federation.MockResolver;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+
+/**
+ * The the RPC interface of the {@link getRouter()} implemented by
+ * {@link RouterRpcServer}.
+ */
+public class TestRouterRpcMultiDestination extends TestRouterRpc {
+
+  @Override
+  public void testSetup() throws Exception {
+
+    RouterDFSCluster cluster = getCluster();
+
+    // Create mock locations
+    getCluster().installMockLocations();
+    List<RouterContext> routers = cluster.getRouters();
+
+    // Add extra location to the root mount / such that the root mount points:
+    // /
+    //   ns0 -> /
+    //   ns1 -> /
+    for (RouterContext rc : routers) {
+      Router router = rc.getRouter();
+      MockResolver resolver = (MockResolver) router.getSubclusterResolver();
+      resolver.addLocation("/", cluster.getNameservices().get(1), "/");
+    }
+
+    // Create a mount that points to 2 dirs in the same ns:
+    // /same
+    //   ns0 -> /
+    //   ns0 -> /target-ns0
+    for (RouterContext rc : routers) {
+      Router router = rc.getRouter();
+      MockResolver resolver = (MockResolver) router.getSubclusterResolver();
+      List<String> nss = cluster.getNameservices();
+      String ns0 = nss.get(0);
+      resolver.addLocation("/same", ns0, "/");
+      resolver.addLocation("/same", ns0, cluster.getNamenodePathForNS(ns0));
+    }
+
+    // Delete all files via the NNs and verify
+    cluster.deleteAllFiles();
+
+    // Create test fixtures on NN
+    cluster.createTestDirectoriesNamenode();
+
+    // Wait to ensure NN has fully created its test directories
+    Thread.sleep(100);
+
+    // Pick a NS, namenode and getRouter() for this test
+    RouterContext router = cluster.getRandomRouter();
+    this.setRouter(router);
+
+    String ns = cluster.getRandomNameservice();
+    this.setNs(ns);
+    this.setNamenode(cluster.getNamenode(ns, null));
+
+    // Create a test file on a single NN that is accessed via a getRouter() path
+    // with 2 destinations. All tests should failover to the alternate
+    // destination if the wrong NN is attempted first.
+    Random r = new Random();
+    String randomString = "testfile-" + r.nextInt();
+    setNamenodeFile("/" + randomString);
+    setRouterFile("/" + randomString);
+
+    FileSystem nnFs = getNamenodeFileSystem();
+    FileSystem routerFs = getRouterFileSystem();
+    createFile(nnFs, getNamenodeFile(), 32);
+
+    verifyFileExists(nnFs, getNamenodeFile());
+    verifyFileExists(routerFs, getRouterFile());
+  }
+
+  private void testListing(String path) throws IOException {
+
+    // Collect the mount table entries for this path
+    Set<String> requiredPaths = new TreeSet<>();
+    RouterContext rc = getRouterContext();
+    Router router = rc.getRouter();
+    FileSubclusterResolver subclusterResolver = router.getSubclusterResolver();
+    for (String mount : subclusterResolver.getMountPoints(path)) {
+      requiredPaths.add(mount);
+    }
+
+    // Get files/dirs from the Namenodes
+    PathLocation location = subclusterResolver.getDestinationForPath(path);
+    for (RemoteLocation loc : location.getDestinations()) {
+      String nsId = loc.getNameserviceId();
+      String dest = loc.getDest();
+      NamenodeContext nn = getCluster().getNamenode(nsId, null);
+      FileSystem fs = nn.getFileSystem();
+      FileStatus[] files = fs.listStatus(new Path(dest));
+      for (FileStatus file : files) {
+        String pathName = file.getPath().getName();
+        requiredPaths.add(pathName);
+      }
+    }
+
+    // Get files/dirs from the Router
+    DirectoryListing listing =
+        getRouterProtocol().getListing(path, HdfsFileStatus.EMPTY_NAME, false);
+    Iterator<String> requiredPathsIterator = requiredPaths.iterator();
+
+    // Match each path returned and verify order returned
+    HdfsFileStatus[] partialListing = listing.getPartialListing();
+    for (HdfsFileStatus fileStatus : listing.getPartialListing()) {
+      String fileName = requiredPathsIterator.next();
+      String currentFile = fileStatus.getFullPath(new Path(path)).getName();
+      assertEquals(currentFile, fileName);
+    }
+
+    // Verify the total number of results found/matched
+    assertEquals(
+        requiredPaths + " doesn't match " + Arrays.toString(partialListing),
+        requiredPaths.size(), partialListing.length);
+  }
+
+  @Override
+  public void testProxyListFiles() throws IOException, InterruptedException,
+      URISyntaxException, NoSuchMethodException, SecurityException {
+
+    // Verify that the root listing is a union of the mount table destinations
+    // and the files stored at all nameservices mounted at the root (ns0 + ns1)
+    // / -->
+    // /ns0 (from mount table)
+    // /ns1 (from mount table)
+    // /same (from the mount table)
+    // all items in / of ns0 from mapping of / -> ns0:::/)
+    // all items in / of ns1 from mapping of / -> ns1:::/)
+    testListing("/");
+
+    // Verify that the "/same" mount point lists the contents of both dirs in
+    // the same ns
+    // /same -->
+    // /target-ns0 (from root of ns0)
+    // /testdir (from contents of /target-ns0)
+    testListing("/same");
+
+    // List a non-existing path and validate error response with NN behavior
+    ClientProtocol namenodeProtocol =
+        getCluster().getRandomNamenode().getClient().getNamenode();
+    Method m = ClientProtocol.class.getMethod(
+        "getListing", String.class,  byte[].class, boolean.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(getRouterProtocol(), namenodeProtocol, m,
+        new Object[] {badPath, HdfsFileStatus.EMPTY_NAME, false});
+  }
+
+  @Override
+  public void testProxyRenameFiles() throws IOException, InterruptedException {
+
+    super.testProxyRenameFiles();
+
+    List<String> nss = getCluster().getNameservices();
+    String ns0 = nss.get(0);
+    String ns1 = nss.get(1);
+
+    // Rename a file from ns0 into the root (mapped to both ns0 and ns1)
+    String testDir0 = getCluster().getFederatedTestDirectoryForNS(ns0);
+    String filename0 = testDir0 + "/testrename";
+    String renamedFile = "/testrename";
+    testRename(getRouterContext(), filename0, renamedFile, false);
+    testRename2(getRouterContext(), filename0, renamedFile, false);
+
+    // Rename a file from ns1 into the root (mapped to both ns0 and ns1)
+    String testDir1 = getCluster().getFederatedTestDirectoryForNS(ns1);
+    String filename1 = testDir1 + "/testrename";
+    testRename(getRouterContext(), filename1, renamedFile, false);
+    testRename2(getRouterContext(), filename1, renamedFile, false);
+  }
+}
\ 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


[18/29] hadoop git commit: HDFS-10880. Federation Mount Table State Store internal API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10880. Federation Mount Table State Store internal API. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 28bd21f4e5c911262d4e3dd207e86c48f2d9a2f4
Parents: 740ff17
Author: Inigo Goiri <in...@apache.org>
Authored: Fri Aug 4 18:00:12 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   7 +-
 .../federation/resolver/MountTableManager.java  |  80 +++
 .../federation/resolver/MountTableResolver.java | 544 +++++++++++++++++++
 .../federation/resolver/PathLocation.java       | 124 ++++-
 .../resolver/order/DestinationOrder.java        |  29 +
 .../federation/resolver/order/package-info.java |  29 +
 .../federation/router/FederationUtil.java       |  56 +-
 .../hdfs/server/federation/router/Router.java   |   3 +-
 .../federation/store/MountTableStore.java       |  49 ++
 .../federation/store/StateStoreService.java     |   2 +
 .../store/impl/MountTableStoreImpl.java         | 116 ++++
 .../protocol/AddMountTableEntryRequest.java     |  47 ++
 .../protocol/AddMountTableEntryResponse.java    |  42 ++
 .../protocol/GetMountTableEntriesRequest.java   |  49 ++
 .../protocol/GetMountTableEntriesResponse.java  |  53 ++
 .../protocol/RemoveMountTableEntryRequest.java  |  49 ++
 .../protocol/RemoveMountTableEntryResponse.java |  42 ++
 .../protocol/UpdateMountTableEntryRequest.java  |  51 ++
 .../protocol/UpdateMountTableEntryResponse.java |  43 ++
 .../pb/AddMountTableEntryRequestPBImpl.java     |  84 +++
 .../pb/AddMountTableEntryResponsePBImpl.java    |  76 +++
 .../pb/GetMountTableEntriesRequestPBImpl.java   |  76 +++
 .../pb/GetMountTableEntriesResponsePBImpl.java  | 104 ++++
 .../pb/RemoveMountTableEntryRequestPBImpl.java  |  76 +++
 .../pb/RemoveMountTableEntryResponsePBImpl.java |  76 +++
 .../pb/UpdateMountTableEntryRequestPBImpl.java  |  96 ++++
 .../pb/UpdateMountTableEntryResponsePBImpl.java |  76 +++
 .../federation/store/records/MountTable.java    | 301 ++++++++++
 .../store/records/impl/pb/MountTablePBImpl.java | 213 ++++++++
 .../src/main/proto/FederationProtocol.proto     |  61 ++-
 .../hdfs/server/federation/MockResolver.java    |   9 +-
 .../resolver/TestMountTableResolver.java        | 396 ++++++++++++++
 .../store/FederationStateStoreTestUtils.java    |  16 +
 .../store/TestStateStoreMountTable.java         | 250 +++++++++
 .../store/driver/TestStateStoreDriverBase.java  |  12 +
 .../store/records/TestMountTable.java           | 176 ++++++
 36 files changed, 3437 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/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 d1c2b41..5433df3 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
@@ -27,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
@@ -1175,8 +1177,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // HDFS Router State Store connection
   public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =
       FEDERATION_ROUTER_PREFIX + "file.resolver.client.class";
-  public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT =
-      "org.apache.hadoop.hdfs.server.federation.MockResolver";
+  public static final Class<? extends FileSubclusterResolver>
+      FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT =
+          MountTableResolver.class;
   public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS =
       FEDERATION_ROUTER_PREFIX + "namenode.resolver.client.class";
   public static final Class<? extends ActiveNamenodeResolver>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableManager.java
new file mode 100644
index 0000000..c2e4a5b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableManager.java
@@ -0,0 +1,80 @@
+/**
+ * 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.resolver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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;
+
+/**
+ * Manage a mount table.
+ */
+public interface MountTableManager {
+
+  /**
+   * Add an entry to the mount table.
+   *
+   * @param request Fully populated request object.
+   * @return True if the mount table entry was successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the data store is not initialized.
+   */
+  AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException;
+
+  /**
+   * Updates an existing entry in the mount table.
+   *
+   * @param request Fully populated request object.
+   * @return True if the mount table entry was successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the data store is not initialized.
+   */
+  UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException;
+
+  /**
+   * Remove an entry from the mount table.
+   *
+   * @param request Fully populated request object.
+   * @return True the mount table entry was removed from the data store.
+   * @throws IOException Throws exception if the data store is not initialized.
+   */
+  RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException;
+
+  /**
+   * List all mount table entries present at or below the path. Fetches from the
+   * state store.
+   *
+   * @param request Fully populated request object.
+   *
+   * @return List of all mount table entries under the path. Zero-length list if
+   *         none are found.
+   * @throws IOException Throws exception if the data store cannot be queried.
+   */
+  GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
new file mode 100644
index 0000000..13e3db3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
@@ -0,0 +1,544 @@
+/**
+ * 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.resolver;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreCache;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Mount table to map between global paths and remote locations. This allows the
+ * {@link org.apache.hadoop.hdfs.server.federation.router.Router Router} to map
+ * the global HDFS view to the remote namespaces. This is similar to
+ * {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
+ * This is implemented as a tree.
+ */
+public class MountTableResolver
+    implements FileSubclusterResolver, StateStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MountTableResolver.class);
+
+  /** Reference to Router. */
+  private final Router router;
+  /** Reference to the State Store. */
+  private final StateStoreService stateStore;
+  /** Interface to the mount table store. */
+  private MountTableStore mountTableStore;
+
+  /** If the tree has been initialized. */
+  private boolean init = false;
+  /** Path -> Remote HDFS location. */
+  private final TreeMap<String, MountTable> tree = new TreeMap<>();
+  /** Path -> Remote location. */
+  private final ConcurrentNavigableMap<String, PathLocation> locationCache =
+      new ConcurrentSkipListMap<>();
+
+  /** Default nameservice when no mount matches the math. */
+  private String defaultNameService = "";
+
+  /** Synchronization for both the tree and the cache. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+
+  @VisibleForTesting
+  public MountTableResolver(Configuration conf) {
+    this(conf, (StateStoreService)null);
+  }
+
+  public MountTableResolver(Configuration conf, Router routerService) {
+    this.router = routerService;
+    if (this.router != null) {
+      this.stateStore = this.router.getStateStore();
+    } else {
+      this.stateStore = null;
+    }
+
+    registerCacheExternal();
+    initDefaultNameService(conf);
+  }
+
+  public MountTableResolver(Configuration conf, StateStoreService store) {
+    this.router = null;
+    this.stateStore = store;
+
+    registerCacheExternal();
+    initDefaultNameService(conf);
+  }
+
+  /**
+   * Request cache updates from the State Store for this resolver.
+   */
+  private void registerCacheExternal() {
+    if (this.stateStore != null) {
+      this.stateStore.registerCacheExternal(this);
+    }
+  }
+
+  /**
+   * Nameservice for APIs that cannot be resolved to a specific one.
+   *
+   * @param conf Configuration for this resolver.
+   */
+  private void initDefaultNameService(Configuration conf) {
+    try {
+      this.defaultNameService = conf.get(
+          DFS_ROUTER_DEFAULT_NAMESERVICE,
+          DFSUtil.getNamenodeNameServiceId(conf));
+    } catch (HadoopIllegalArgumentException e) {
+      LOG.error("Cannot find default name service, setting it to the first");
+      Collection<String> nsIds = DFSUtilClient.getNameServiceIds(conf);
+      this.defaultNameService = nsIds.iterator().next();
+      LOG.info("Default name service: {}", this.defaultNameService);
+    }
+  }
+
+  /**
+   * Get a reference for the Router for this resolver.
+   *
+   * @return Router for this resolver.
+   */
+  protected Router getRouter() {
+    return this.router;
+  }
+
+  /**
+   * Get the mount table store for this resolver.
+   *
+   * @return Mount table store.
+   * @throws IOException If it cannot connect to the State Store.
+   */
+  protected MountTableStore getMountTableStore() throws IOException {
+    if (this.mountTableStore == null) {
+      this.mountTableStore = this.stateStore.getRegisteredRecordStore(
+          MountTableStore.class);
+      if (this.mountTableStore == null) {
+        throw new IOException("State Store does not have an interface for " +
+            MountTableStore.class);
+      }
+    }
+    return this.mountTableStore;
+  }
+
+  /**
+   * Add a mount entry to the table.
+   *
+   * @param entry The mount table record to add from the state store.
+   */
+  public void addEntry(final MountTable entry) {
+    writeLock.lock();
+    try {
+      String srcPath = entry.getSourcePath();
+      this.tree.put(srcPath, entry);
+      invalidateLocationCache(srcPath);
+    } finally {
+      writeLock.unlock();
+    }
+    this.init = true;
+  }
+
+  /**
+   * Remove a mount table entry.
+   *
+   * @param srcPath Source path for the entry to remove.
+   */
+  public void removeEntry(final String srcPath) {
+    writeLock.lock();
+    try {
+      this.tree.remove(srcPath);
+      invalidateLocationCache(srcPath);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Invalidates all cache entries below this path. It requires the write lock.
+   *
+   * @param src Source path.
+   */
+  private void invalidateLocationCache(final String path) {
+    if (locationCache.isEmpty()) {
+      return;
+    }
+    // Determine next lexicographic entry after source path
+    String nextSrc = path + Character.MAX_VALUE;
+    ConcurrentNavigableMap<String, PathLocation> subMap =
+        locationCache.subMap(path, nextSrc);
+    for (final String key : subMap.keySet()) {
+      locationCache.remove(key);
+    }
+  }
+
+  /**
+   * Updates the mount path tree with a new set of mount table entries. It also
+   * updates the needed caches.
+   *
+   * @param entries Full set of mount table entries to update.
+   */
+  @VisibleForTesting
+  public void refreshEntries(final Collection<MountTable> entries) {
+    // The tree read/write must be atomic
+    writeLock.lock();
+    try {
+      // New entries
+      Map<String, MountTable> newEntries = new ConcurrentHashMap<>();
+      for (MountTable entry : entries) {
+        String srcPath = entry.getSourcePath();
+        newEntries.put(srcPath, entry);
+      }
+
+      // Old entries (reversed to sort from the leaves to the root)
+      Set<String> oldEntries = new TreeSet<>(Collections.reverseOrder());
+      for (MountTable entry : getTreeValues("/")) {
+        String srcPath = entry.getSourcePath();
+        oldEntries.add(srcPath);
+      }
+
+      // Entries that need to be removed
+      for (String srcPath : oldEntries) {
+        if (!newEntries.containsKey(srcPath)) {
+          this.tree.remove(srcPath);
+          invalidateLocationCache(srcPath);
+          LOG.info("Removed stale mount point {} from resolver", srcPath);
+        }
+      }
+
+      // Entries that need to be added
+      for (MountTable entry : entries) {
+        String srcPath = entry.getSourcePath();
+        if (!oldEntries.contains(srcPath)) {
+          // Add node, it does not exist
+          this.tree.put(srcPath, entry);
+          LOG.info("Added new mount point {} to resolver", srcPath);
+        } else {
+          // Node exists, check for updates
+          MountTable existingEntry = this.tree.get(srcPath);
+          if (existingEntry != null && !existingEntry.equals(entry)) {
+            // Entry has changed
+            invalidateLocationCache(srcPath);
+            LOG.info("Updated mount point {} in resolver");
+          }
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+    this.init = true;
+  }
+
+  /**
+   * Replaces the current in-memory cached of the mount table with a new
+   * version fetched from the data store.
+   */
+  @Override
+  public boolean loadCache(boolean force) {
+    try {
+      // Our cache depends on the store, update it first
+      MountTableStore mountTable = this.getMountTableStore();
+      mountTable.loadCache(force);
+
+      GetMountTableEntriesRequest request =
+          GetMountTableEntriesRequest.newInstance("/");
+      GetMountTableEntriesResponse response =
+          mountTable.getMountTableEntries(request);
+      List<MountTable> records = response.getEntries();
+      refreshEntries(records);
+    } catch (IOException e) {
+      LOG.error("Cannot fetch mount table entries from State Store", e);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Clears all data.
+   */
+  public void clear() {
+    LOG.info("Clearing all mount location caches");
+    writeLock.lock();
+    try {
+      this.locationCache.clear();
+      this.tree.clear();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public PathLocation getDestinationForPath(final String path)
+      throws IOException {
+    verifyMountTable();
+    readLock.lock();
+    try {
+      return this.locationCache.computeIfAbsent(
+          path, this::lookupLocation);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Build the path location to insert into the cache atomically. It must hold
+   * the read lock.
+   * @param path Path to check/insert.
+   * @return New remote location.
+   */
+  public PathLocation lookupLocation(final String path) {
+    PathLocation ret = null;
+    MountTable entry = findDeepest(path);
+    if (entry != null) {
+      ret = buildLocation(path, entry);
+    } else {
+      // Not found, use default location
+      RemoteLocation remoteLocation =
+          new RemoteLocation(defaultNameService, path);
+      List<RemoteLocation> locations =
+          Collections.singletonList(remoteLocation);
+      ret = new PathLocation(null, locations);
+    }
+    return ret;
+  }
+
+  /**
+   * Get the mount table entry for a path.
+   *
+   * @param path Path to look for.
+   * @return Mount table entry the path belongs.
+   * @throws IOException If the State Store could not be reached.
+   */
+  public MountTable getMountPoint(final String path) throws IOException {
+    verifyMountTable();
+    return findDeepest(path);
+  }
+
+  @Override
+  public List<String> getMountPoints(final String path) throws IOException {
+    verifyMountTable();
+
+    Set<String> children = new TreeSet<>();
+    readLock.lock();
+    try {
+      String from = path;
+      String to = path + Character.MAX_VALUE;
+      SortedMap<String, MountTable> subMap = this.tree.subMap(from, to);
+
+      boolean exists = false;
+      for (String subPath : subMap.keySet()) {
+        String child = subPath;
+
+        // Special case for /
+        if (!path.equals(Path.SEPARATOR)) {
+          // Get the children
+          int ini = path.length();
+          child = subPath.substring(ini);
+        }
+
+        if (child.isEmpty()) {
+          // This is a mount point but without children
+          exists = true;
+        } else if (child.startsWith(Path.SEPARATOR)) {
+          // This is a mount point with children
+          exists = true;
+          child = child.substring(1);
+
+          // We only return immediate children
+          int fin = child.indexOf(Path.SEPARATOR);
+          if (fin > -1) {
+            child = child.substring(0, fin);
+          }
+          if (!child.isEmpty()) {
+            children.add(child);
+          }
+        }
+      }
+      if (!exists) {
+        return null;
+      }
+      return new LinkedList<>(children);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get all the mount records at or beneath a given path.
+   * @param path Path to get the mount points from.
+   * @return List of mount table records under the path or null if the path is
+   *         not found.
+   * @throws IOException If it's not connected to the State Store.
+   */
+  public List<MountTable> getMounts(final String path) throws IOException {
+    verifyMountTable();
+
+    return getTreeValues(path, false);
+  }
+
+  /**
+   * Check if the Mount Table is ready to be used.
+   * @throws StateStoreUnavailableException If it cannot connect to the store.
+   */
+  private void verifyMountTable() throws StateStoreUnavailableException {
+    if (!this.init) {
+      throw new StateStoreUnavailableException("Mount Table not initialized");
+    }
+  }
+
+  @Override
+  public String toString() {
+    readLock.lock();
+    try {
+      return this.tree.toString();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Build a location for this result beneath the discovered mount point.
+   *
+   * @param result Tree node search result.
+   * @return PathLocation containing the namespace, local path.
+   */
+  private static PathLocation buildLocation(
+      final String path, final MountTable entry) {
+
+    String srcPath = entry.getSourcePath();
+    if (!path.startsWith(srcPath)) {
+      LOG.error("Cannot build location, {} not a child of {}", path, srcPath);
+      return null;
+    }
+    String remainingPath = path.substring(srcPath.length());
+    if (remainingPath.startsWith(Path.SEPARATOR)) {
+      remainingPath = remainingPath.substring(1);
+    }
+
+    List<RemoteLocation> locations = new LinkedList<>();
+    for (RemoteLocation oneDst : entry.getDestinations()) {
+      String nsId = oneDst.getNameserviceId();
+      String dest = oneDst.getDest();
+      String newPath = dest;
+      if (!newPath.endsWith(Path.SEPARATOR)) {
+        newPath += Path.SEPARATOR;
+      }
+      newPath += remainingPath;
+      RemoteLocation remoteLocation = new RemoteLocation(nsId, newPath);
+      locations.add(remoteLocation);
+    }
+    DestinationOrder order = entry.getDestOrder();
+    return new PathLocation(srcPath, locations, order);
+  }
+
+  @Override
+  public String getDefaultNamespace() {
+    return this.defaultNameService;
+  }
+
+  /**
+   * Find the deepest mount point for a path.
+   * @param path Path to look for.
+   * @return Mount table entry.
+   */
+  private MountTable findDeepest(final String path) {
+    readLock.lock();
+    try {
+      Entry<String, MountTable> entry = this.tree.floorEntry(path);
+      while (entry != null && !path.startsWith(entry.getKey())) {
+        entry = this.tree.lowerEntry(entry.getKey());
+      }
+      if (entry == null) {
+        return null;
+      }
+      return entry.getValue();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get the mount table entries under a path.
+   * @param path Path to search from.
+   * @return Mount Table entries.
+   */
+  private List<MountTable> getTreeValues(final String path) {
+    return getTreeValues(path, false);
+  }
+
+  /**
+   * Get the mount table entries under a path.
+   * @param path Path to search from.
+   * @param reverse If the order should be reversed.
+   * @return Mount Table entries.
+   */
+  private List<MountTable> getTreeValues(final String path, boolean reverse) {
+    LinkedList<MountTable> ret = new LinkedList<>();
+    readLock.lock();
+    try {
+      String from = path;
+      String to = path + Character.MAX_VALUE;
+      SortedMap<String, MountTable> subMap = this.tree.subMap(from, to);
+      for (MountTable entry : subMap.values()) {
+        if (!reverse) {
+          ret.add(entry);
+        } else {
+          ret.addFirst(entry);
+        }
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
index d90565c..945d81d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
@@ -23,21 +23,27 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * A map of the properties and target destinations (name space + path) for
- * a path in the global/federated namespace.
+ * a path in the global/federated name space.
  * This data is generated from the @see MountTable records.
  */
 public class PathLocation {
 
+  private static final Logger LOG = LoggerFactory.getLogger(PathLocation.class);
+
+
   /** Source path in global namespace. */
   private final String sourcePath;
 
-  /** Remote paths in the target namespaces. */
+  /** Remote paths in the target name spaces. */
   private final List<RemoteLocation> destinations;
-
-  /** List of name spaces present. */
-  private final Set<String> namespaces;
+  /** Order for the destinations. */
+  private final DestinationOrder destOrder;
 
 
   /**
@@ -45,14 +51,23 @@ public class PathLocation {
    *
    * @param source Source path in the global name space.
    * @param dest Destinations of the mount table entry.
-   * @param namespaces Unique identifier representing the combination of
-   *          name spaces present in the destination list.
+   * @param order Order of the locations.
    */
   public PathLocation(
-      String source, List<RemoteLocation> dest, Set<String> nss) {
+      String source, List<RemoteLocation> dest, DestinationOrder order) {
     this.sourcePath = source;
-    this.destinations = dest;
-    this.namespaces = nss;
+    this.destinations = Collections.unmodifiableList(dest);
+    this.destOrder = order;
+  }
+
+  /**
+   * Create a new PathLocation with default HASH order.
+   *
+   * @param source Source path in the global name space.
+   * @param dest Destinations of the mount table entry.
+   */
+  public PathLocation(String source, List<RemoteLocation> dest) {
+    this(source, dest, DestinationOrder.HASH);
   }
 
   /**
@@ -60,10 +75,55 @@ public class PathLocation {
    *
    * @param other Other path location to copy from.
    */
-  public PathLocation(PathLocation other) {
+  public PathLocation(final PathLocation other) {
     this.sourcePath = other.sourcePath;
-    this.destinations = new LinkedList<RemoteLocation>(other.destinations);
-    this.namespaces = new HashSet<String>(other.namespaces);
+    this.destinations = Collections.unmodifiableList(other.destinations);
+    this.destOrder = other.destOrder;
+  }
+
+  /**
+   * Create a path location from another path with the destinations sorted.
+   *
+   * @param other Other path location to copy from.
+   * @param firstNsId Identifier of the namespace to place first.
+   */
+  public PathLocation(PathLocation other, String firstNsId) {
+    this.sourcePath = other.sourcePath;
+    this.destOrder = other.destOrder;
+    this.destinations = orderedNamespaces(other.destinations, firstNsId);
+  }
+
+  /**
+   * Prioritize a location/destination by its name space/nameserviceId.
+   * This destination might be used by other threads, so the source is not
+   * modifiable.
+   *
+   * @param original List of destinations to order.
+   * @param nsId The name space/nameserviceID to prioritize.
+   * @return Prioritized list of detinations that cannot be modified.
+   */
+  private static List<RemoteLocation> orderedNamespaces(
+      final List<RemoteLocation> original, final String nsId) {
+    if (original.size() <= 1) {
+      return original;
+    }
+
+    LinkedList<RemoteLocation> newDestinations = new LinkedList<>();
+    boolean found = false;
+    for (RemoteLocation dest : original) {
+      if (dest.getNameserviceId().equals(nsId)) {
+        found = true;
+        newDestinations.addFirst(dest);
+      } else {
+        newDestinations.add(dest);
+      }
+    }
+
+    if (!found) {
+      LOG.debug("Cannot find location with namespace {} in {}",
+          nsId, original);
+    }
+    return Collections.unmodifiableList(newDestinations);
   }
 
   /**
@@ -76,16 +136,37 @@ public class PathLocation {
   }
 
   /**
-   * Get the list of subclusters defined for the destinations.
+   * Get the subclusters defined for the destinations.
+   *
+   * @return Set containing the subclusters.
    */
   public Set<String> getNamespaces() {
-    return Collections.unmodifiableSet(this.namespaces);
+    Set<String> namespaces = new HashSet<>();
+    List<RemoteLocation> locations = this.getDestinations();
+    for (RemoteLocation location : locations) {
+      String nsId = location.getNameserviceId();
+      namespaces.add(nsId);
+    }
+    return namespaces;
   }
 
   @Override
   public String toString() {
-    RemoteLocation loc = getDefaultLocation();
-    return loc.getNameserviceId() + "->" + loc.getDest();
+    StringBuilder sb = new StringBuilder();
+    for (RemoteLocation destination : this.destinations) {
+      String nsId = destination.getNameserviceId();
+      String path = destination.getDest();
+      if (sb.length() > 0) {
+        sb.append(",");
+      }
+      sb.append(nsId + "->" + path);
+    }
+    if (this.destinations.size() > 1) {
+      sb.append(" [");
+      sb.append(this.destOrder.toString());
+      sb.append("]");
+    }
+    return sb.toString();
   }
 
   /**
@@ -108,6 +189,15 @@ public class PathLocation {
   }
 
   /**
+   * Get the order for the destinations.
+   *
+   * @return Order for the destinations.
+   */
+  public DestinationOrder getDestinationOrder() {
+    return this.destOrder;
+  }
+
+  /**
    * Get the default or highest priority location.
    *
    * @return The default location.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java
new file mode 100644
index 0000000..4bccf10
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.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.federation.resolver.order;
+
+/**
+ * Order of the destinations when we have multiple of them. When the resolver
+ * of files to subclusters (FileSubclusterResolver) has multiple destinations,
+ * this determines which location should be checked first.
+ */
+public enum DestinationOrder {
+  HASH, // Follow consistent hashing
+  LOCAL, // Local first
+  RANDOM // Random order
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/package-info.java
new file mode 100644
index 0000000..f90152f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/package-info.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.
+ */
+
+/**
+ * A federated location can be resolved to multiple subclusters. This package
+ * takes care of the order in which this multiple destinations should be used.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.resolver.order;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
index 78c473a..99af2d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
@@ -136,65 +136,19 @@ public final class FederationUtil {
   }
 
   /**
-   * Create an instance of an interface with a constructor using a state store
-   * constructor.
-   *
-   * @param conf Configuration
-   * @param context Context object to pass to the instance.
-   * @param contextType Type of the context passed to the constructor.
-   * @param configurationKeyName Configuration key to retrieve the class to load
-   * @param defaultClassName Default class to load if the configuration key is
-   *          not set
-   * @param clazz Class/interface that must be implemented by the instance.
-   * @return New instance of the specified class that implements the desired
-   *         interface and a single parameter constructor containing a
-   *         StateStore reference.
-   */
-  private static <T, R> T newInstance(final Configuration conf,
-      final R context, final Class<R> contextClass,
-      final String configKeyName, final String defaultClassName,
-      final Class<T> clazz) {
-
-    String className = conf.get(configKeyName, defaultClassName);
-    try {
-      Class<?> instance = conf.getClassByName(className);
-      if (clazz.isAssignableFrom(instance)) {
-        if (contextClass == null) {
-          // Default constructor if no context
-          @SuppressWarnings("unchecked")
-          Constructor<T> constructor =
-              (Constructor<T>) instance.getConstructor();
-          return constructor.newInstance();
-        } else {
-          // Constructor with context
-          @SuppressWarnings("unchecked")
-          Constructor<T> constructor = (Constructor<T>) instance.getConstructor(
-              Configuration.class, contextClass);
-          return constructor.newInstance(conf, context);
-        }
-      } else {
-        throw new RuntimeException("Class " + className + " not instance of "
-            + clazz.getCanonicalName());
-      }
-    } catch (ReflectiveOperationException e) {
-      LOG.error("Could not instantiate: " + className, e);
-      return null;
-    }
-  }
-
-  /**
    * Creates an instance of a FileSubclusterResolver from the configuration.
    *
    * @param conf Configuration that defines the file resolver class.
-   * @param obj Context object passed to class constructor.
-   * @return FileSubclusterResolver
+   * @param router Router service.
+   * @return New file subcluster resolver.
    */
   public static FileSubclusterResolver newFileSubclusterResolver(
-      Configuration conf, StateStoreService stateStore) {
-    return newInstance(conf, stateStore, StateStoreService.class,
+      Configuration conf, Router router) {
+    Class<? extends FileSubclusterResolver> clazz = conf.getClass(
         DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
         DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT,
         FileSubclusterResolver.class);
+    return newInstance(conf, router, Router.class, clazz);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index cfddf20..213a58f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -124,8 +124,7 @@ public class Router extends CompositeService {
     }
 
     // Lookup interface to map between the global and subcluster name spaces
-    this.subclusterResolver = newFileSubclusterResolver(
-        this.conf, this.stateStore);
+    this.subclusterResolver = newFileSubclusterResolver(this.conf, this);
     if (this.subclusterResolver == null) {
       throw new IOException("Cannot find subcluster resolver");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MountTableStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MountTableStore.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MountTableStore.java
new file mode 100644
index 0000000..b439659
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/MountTableStore.java
@@ -0,0 +1,49 @@
+/**
+ * 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.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+
+/**
+ * Management API for the HDFS mount table information stored in
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.MountTable
+ * MountTable} records. The mount table contains entries that map a particular
+ * global namespace path one or more HDFS nameservices (NN) + target path. It is
+ * possible to map mount locations for root folders, directories or individual
+ * files.
+ * <p>
+ * Once fetched from the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver}, MountTable records are cached in a tree for faster access.
+ * Each path in the global namespace is mapped to a nameserivce ID and local
+ * path upon request. The cache is periodically updated by the @{link
+ * StateStoreCacheUpdateService}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class MountTableStore extends CachedRecordStore<MountTable>
+    implements MountTableManager {
+
+  public MountTableStore(StateStoreDriver driver) {
+    super(MountTable.class, driver);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
index 73f607f..3aa3ffd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.impl.MembershipStoreImpl;
+import org.apache.hadoop.hdfs.server.federation.store.impl.MountTableStoreImpl;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
 import org.apache.hadoop.service.CompositeService;
@@ -136,6 +137,7 @@ public class StateStoreService extends CompositeService {
 
     // Add supported record stores
     addRecordStore(MembershipStoreImpl.class);
+    addRecordStore(MountTableStoreImpl.class);
 
     // Check the connection to the State Store periodically
     this.monitorService = new StateStoreConnectionMonitorService(this);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/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
new file mode 100644
index 0000000..e6affb2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.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.federation.store.impl;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Implementation of the {@link MountTableStore} state store API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MountTableStoreImpl extends MountTableStore {
+
+  public MountTableStoreImpl(StateStoreDriver driver) {
+    super(driver);
+  }
+
+  @Override
+  public AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException {
+    boolean status = getDriver().put(request.getEntry(), false, true);
+    AddMountTableEntryResponse response =
+        AddMountTableEntryResponse.newInstance();
+    response.setStatus(status);
+    return response;
+  }
+
+  @Override
+  public UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException {
+    MountTable entry = request.getEntry();
+    boolean status = getDriver().put(entry, true, true);
+    UpdateMountTableEntryResponse response =
+        UpdateMountTableEntryResponse.newInstance();
+    response.setStatus(status);
+    return response;
+  }
+
+  @Override
+  public RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException {
+    final String srcPath = request.getSrcPath();
+    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);
+    RemoveMountTableEntryResponse response =
+        RemoveMountTableEntryResponse.newInstance();
+    response.setStatus(status);
+    return response;
+  }
+
+  @Override
+  public GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException {
+
+    // Get all values from the cache
+    List<MountTable> records = getCachedRecords();
+
+    // Sort and filter
+    Collections.sort(records);
+    String reqSrcPath = request.getSrcPath();
+    if (reqSrcPath != null && !reqSrcPath.isEmpty()) {
+      // Return only entries beneath this path
+      Iterator<MountTable> it = records.iterator();
+      while (it.hasNext()) {
+        MountTable record = it.next();
+        String srcPath = record.getSourcePath();
+        if (!srcPath.startsWith(reqSrcPath)) {
+          it.remove();
+        }
+      }
+    }
+
+    GetMountTableEntriesResponse response =
+        GetMountTableEntriesResponse.newInstance();
+    response.setEntries(records);
+    response.setTimestamp(Time.now());
+    return response;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryRequest.java
new file mode 100644
index 0000000..2d9f102
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryRequest.java
@@ -0,0 +1,47 @@
+/**
+ * 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.store.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+
+/**
+ * API request for adding a mount table entry to the state store.
+ */
+public abstract class AddMountTableEntryRequest {
+
+  public static AddMountTableEntryRequest newInstance() {
+    return StateStoreSerializer.newRecord(AddMountTableEntryRequest.class);
+  }
+
+  public static AddMountTableEntryRequest newInstance(MountTable newEntry) {
+    AddMountTableEntryRequest request = newInstance();
+    request.setEntry(newEntry);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract MountTable getEntry();
+
+  @Public
+  @Unstable
+  public abstract void setEntry(MountTable mount);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryResponse.java
new file mode 100644
index 0000000..9bc7f92
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntryResponse.java
@@ -0,0 +1,42 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for adding a mount table entry to the state store.
+ */
+public abstract class AddMountTableEntryResponse {
+
+  public static AddMountTableEntryResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(AddMountTableEntryResponse.class);
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean getStatus();
+
+  @Public
+  @Unstable
+  public abstract void setStatus(boolean result);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesRequest.java
new file mode 100644
index 0000000..cd6c278
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesRequest.java
@@ -0,0 +1,49 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for listing mount table entries present in the state store.
+ */
+public abstract class GetMountTableEntriesRequest {
+
+  public static GetMountTableEntriesRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(GetMountTableEntriesRequest.class);
+  }
+
+  public static GetMountTableEntriesRequest newInstance(String srcPath)
+      throws IOException {
+    GetMountTableEntriesRequest request = newInstance();
+    request.setSrcPath(srcPath);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getSrcPath();
+
+  @Public
+  @Unstable
+  public abstract void setSrcPath(String path);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesResponse.java
new file mode 100644
index 0000000..cebc3f6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetMountTableEntriesResponse.java
@@ -0,0 +1,53 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+
+/**
+ * API response for listing mount table entries present in the state store.
+ */
+public abstract class GetMountTableEntriesResponse {
+
+  public static GetMountTableEntriesResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(GetMountTableEntriesResponse.class);
+  }
+
+  @Public
+  @Unstable
+  public abstract List<MountTable> getEntries() throws IOException;
+
+  @Public
+  @Unstable
+  public abstract void setEntries(List<MountTable> entries)
+      throws IOException;
+
+  @Public
+  @Unstable
+  public abstract long getTimestamp();
+
+  @Public
+  @Unstable
+  public abstract void setTimestamp(long time);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryRequest.java
new file mode 100644
index 0000000..642ee0d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryRequest.java
@@ -0,0 +1,49 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for removing a mount table path present in the state store.
+ */
+public abstract class RemoveMountTableEntryRequest {
+
+  public static RemoveMountTableEntryRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(RemoveMountTableEntryRequest.class);
+  }
+
+  public static RemoveMountTableEntryRequest newInstance(String path)
+      throws IOException {
+    RemoveMountTableEntryRequest request = newInstance();
+    request.setSrcPath(path);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getSrcPath();
+
+  @Public
+  @Unstable
+  public abstract void setSrcPath(String path);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryResponse.java
new file mode 100644
index 0000000..70f117d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RemoveMountTableEntryResponse.java
@@ -0,0 +1,42 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for removing a mount table path present in the state store.
+ */
+public abstract class RemoveMountTableEntryResponse {
+
+  public static RemoveMountTableEntryResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(RemoveMountTableEntryResponse.class);
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean getStatus();
+
+  @Public
+  @Unstable
+  public abstract void setStatus(boolean result);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryRequest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryRequest.java
new file mode 100644
index 0000000..afd5128
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryRequest.java
@@ -0,0 +1,51 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+
+/**
+ * API request for updating the destination of an existing mount point in the
+ * state store.
+ */
+public abstract class UpdateMountTableEntryRequest {
+
+  public static UpdateMountTableEntryRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(UpdateMountTableEntryRequest.class);
+  }
+
+  public static UpdateMountTableEntryRequest newInstance(MountTable entry)
+      throws IOException {
+    UpdateMountTableEntryRequest request = newInstance();
+    request.setEntry(entry);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract MountTable getEntry() throws IOException;
+
+  @Public
+  @Unstable
+  public abstract void setEntry(MountTable mount) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryResponse.java
new file mode 100644
index 0000000..7097e10
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/UpdateMountTableEntryResponse.java
@@ -0,0 +1,43 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for updating the destination of an existing mount point in the
+ * state store.
+ */
+public abstract class UpdateMountTableEntryResponse {
+
+  public static UpdateMountTableEntryResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(UpdateMountTableEntryResponse.class);
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean getStatus();
+
+  @Public
+  @Unstable
+  public abstract void setStatus(boolean result);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryRequestPBImpl.java
new file mode 100644
index 0000000..35455d2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryRequestPBImpl.java
@@ -0,0 +1,84 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.MountTablePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * AddMountTableEntryRequest.
+ */
+public class AddMountTableEntryRequestPBImpl
+    extends AddMountTableEntryRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<AddMountTableEntryRequestProto,
+      AddMountTableEntryRequestProto.Builder,
+      AddMountTableEntryRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<AddMountTableEntryRequestProto,
+              AddMountTableEntryRequestProto.Builder,
+              AddMountTableEntryRequestProtoOrBuilder>(
+                  AddMountTableEntryRequestProto.class);
+
+  public AddMountTableEntryRequestPBImpl() {
+  }
+
+  public AddMountTableEntryRequestPBImpl(AddMountTableEntryRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public AddMountTableEntryRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public MountTable getEntry() {
+    MountTableRecordProto entryProto =
+        this.translator.getProtoOrBuilder().getEntry();
+    return new MountTablePBImpl(entryProto);
+  }
+
+  @Override
+  public void setEntry(MountTable mount) {
+    if (mount instanceof MountTablePBImpl) {
+      MountTablePBImpl mountPB = (MountTablePBImpl)mount;
+      MountTableRecordProto mountProto = mountPB.getProto();
+      translator.getBuilder().setEntry(mountProto);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryResponsePBImpl.java
new file mode 100644
index 0000000..c1d9a65
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntryResponsePBImpl.java
@@ -0,0 +1,76 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * AddMountTableEntryResponse.
+ */
+public class AddMountTableEntryResponsePBImpl
+    extends AddMountTableEntryResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<AddMountTableEntryResponseProto,
+      AddMountTableEntryResponseProto.Builder,
+      AddMountTableEntryResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<AddMountTableEntryResponseProto,
+              AddMountTableEntryResponseProto.Builder,
+              AddMountTableEntryResponseProtoOrBuilder>(
+                  AddMountTableEntryResponseProto.class);
+
+  public AddMountTableEntryResponsePBImpl() {
+  }
+
+  public AddMountTableEntryResponsePBImpl(
+      AddMountTableEntryResponseProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public AddMountTableEntryResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getStatus() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setStatus(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesRequestPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesRequestPBImpl.java
new file mode 100644
index 0000000..3e0d1a6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesRequestPBImpl.java
@@ -0,0 +1,76 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetMountTableEntriesRequest.
+ */
+public class GetMountTableEntriesRequestPBImpl
+    extends GetMountTableEntriesRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetMountTableEntriesRequestProto,
+      GetMountTableEntriesRequestProto.Builder,
+      GetMountTableEntriesRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<GetMountTableEntriesRequestProto,
+              GetMountTableEntriesRequestProto.Builder,
+              GetMountTableEntriesRequestProtoOrBuilder>(
+                  GetMountTableEntriesRequestProto.class);
+
+  public GetMountTableEntriesRequestPBImpl() {
+  }
+
+  public GetMountTableEntriesRequestPBImpl(
+      GetMountTableEntriesRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public GetMountTableEntriesRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public String getSrcPath() {
+    return this.translator.getProtoOrBuilder().getSrcPath();
+  }
+
+  @Override
+  public void setSrcPath(String path) {
+    this.translator.getBuilder().setSrcPath(path);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28bd21f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesResponsePBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesResponsePBImpl.java
new file mode 100644
index 0000000..9d64bc9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetMountTableEntriesResponsePBImpl.java
@@ -0,0 +1,104 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto;
+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.server.federation.store.records.impl.pb.MountTablePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetMountTableEntriesResponse.
+ */
+public class GetMountTableEntriesResponsePBImpl
+    extends GetMountTableEntriesResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetMountTableEntriesResponseProto,
+      GetMountTableEntriesResponseProto.Builder,
+      GetMountTableEntriesResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<GetMountTableEntriesResponseProto,
+              GetMountTableEntriesResponseProto.Builder,
+              GetMountTableEntriesResponseProtoOrBuilder>(
+                  GetMountTableEntriesResponseProto.class);
+
+  public GetMountTableEntriesResponsePBImpl() {
+  }
+
+  public GetMountTableEntriesResponsePBImpl(
+      GetMountTableEntriesResponseProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public GetMountTableEntriesResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public List<MountTable> getEntries() throws IOException {
+    List<MountTableRecordProto> entries =
+        this.translator.getProtoOrBuilder().getEntriesList();
+    List<MountTable> ret = new ArrayList<MountTable>();
+    for (MountTableRecordProto entry : entries) {
+      MountTable record = new MountTablePBImpl(entry);
+      ret.add(record);
+    }
+    return ret;
+  }
+
+  @Override
+  public void setEntries(List<MountTable> records) throws IOException {
+    this.translator.getBuilder().clearEntries();
+    for (MountTable entry : records) {
+      if (entry instanceof MountTablePBImpl) {
+        MountTablePBImpl entryPB = (MountTablePBImpl)entry;
+        this.translator.getBuilder().addEntries(entryPB.getProto());
+      }
+    }
+  }
+
+  @Override
+  public long getTimestamp() {
+    return this.translator.getProtoOrBuilder().getTimestamp();
+  }
+
+  @Override
+  public void setTimestamp(long time) {
+    this.translator.getBuilder().setTimestamp(time);
+  }
+}
\ 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


[02/29] hadoop git commit: HADOOP-14688. Intern strings in KeyVersion and EncryptedKeyVersion. Contributed by Xiao Chen.

Posted by in...@apache.org.
HADOOP-14688. Intern strings in KeyVersion and EncryptedKeyVersion. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-10467
Commit: ad32759fd9f33e7bd18758ad1a5a464dab3bcbd9
Parents: ccd2ac6
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Sep 5 15:11:37 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Sep 5 15:11:37 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java | 4 ++--
 .../apache/hadoop/crypto/key/KeyProviderCryptoExtension.java    | 5 +++--
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad32759f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
index fb3dba4..c1c371b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
@@ -71,8 +71,8 @@ public abstract class KeyProvider {
 
     protected KeyVersion(String name, String versionName,
                          byte[] material) {
-      this.name = name;
-      this.versionName = versionName;
+      this.name = name == null ? null : name.intern();
+      this.versionName = versionName == null ? null : versionName.intern();
       this.material = material;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad32759f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 8c879b3..693c785 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -81,8 +81,9 @@ public class KeyProviderCryptoExtension extends
     protected EncryptedKeyVersion(String keyName,
         String encryptionKeyVersionName, byte[] encryptedKeyIv,
         KeyVersion encryptedKeyVersion) {
-      this.encryptionKeyName = keyName;
-      this.encryptionKeyVersionName = encryptionKeyVersionName;
+      this.encryptionKeyName = keyName == null ? null : keyName.intern();
+      this.encryptionKeyVersionName = encryptionKeyVersionName == null ?
+          null : encryptionKeyVersionName.intern();
       this.encryptedKeyIv = encryptedKeyIv;
       this.encryptedKeyVersion = encryptedKeyVersion;
     }


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


[03/29] hadoop git commit: HDFS-12377. Refactor TestReadStripedFileWithDecoding to avoid test timeouts.

Posted by in...@apache.org.
HDFS-12377. Refactor TestReadStripedFileWithDecoding to avoid test timeouts.


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

Branch: refs/heads/HDFS-10467
Commit: d4035d42f02507bc89adce3f0450c36b58b201c1
Parents: ad32759
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 5 16:33:29 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 5 16:33:29 2017 -0700

----------------------------------------------------------------------
 .../hdfs/ReadStripedFileWithDecodingHelper.java | 273 ++++++++++++++++
 .../apache/hadoop/hdfs/StripedFileTestUtil.java |   9 +-
 .../hdfs/TestReadStripedFileWithDNFailure.java  | 107 +++++++
 .../hdfs/TestReadStripedFileWithDecoding.java   | 309 +++----------------
 ...tReadStripedFileWithDecodingCorruptData.java |  87 ++++++
 ...tReadStripedFileWithDecodingDeletedData.java |  88 ++++++
 6 files changed, 596 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
new file mode 100644
index 0000000..4202969
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
@@ -0,0 +1,273 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Utility class for testing online recovery of striped files.
+ */
+abstract public class ReadStripedFileWithDecodingHelper {
+  static final Logger LOG =
+      LoggerFactory.getLogger(ReadStripedFileWithDecodingHelper.class);
+
+  static {
+    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
+        .getLogger().setLevel(org.apache.log4j.Level.ALL);
+    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.DEBUG);
+    GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.DEBUG);
+    GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.DEBUG);
+  }
+
+  protected static final ErasureCodingPolicy EC_POLICY =
+      StripedFileTestUtil.getDefaultECPolicy();
+  protected static final short NUM_DATA_UNITS =
+      (short) EC_POLICY.getNumDataUnits();
+  protected static final short NUM_PARITY_UNITS =
+      (short) EC_POLICY.getNumParityUnits();
+  protected static final int CELL_SIZE = EC_POLICY.getCellSize();
+  private static final int STRIPES_PER_BLOCK = 4;
+  protected static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
+  private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_UNITS;
+
+  private static final int NUM_DATANODES = NUM_DATA_UNITS + NUM_PARITY_UNITS;
+
+  protected static final int[] FILE_LENGTHS =
+      {BLOCK_GROUP_SIZE - 123, BLOCK_GROUP_SIZE + 123};
+
+  public static MiniDFSCluster initializeCluster() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
+        0);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
+        false);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
+    MiniDFSCluster myCluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(NUM_DATANODES)
+        .build();
+    myCluster.getFileSystem().getClient().setErasureCodingPolicy("/",
+        StripedFileTestUtil.getDefaultECPolicy().getName());
+    return myCluster;
+  }
+
+  public static void tearDownCluster(MiniDFSCluster cluster)
+      throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  public static int findFirstDataNode(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, Path file, long length) throws IOException {
+    BlockLocation[] locs = dfs.getFileBlockLocations(file, 0, length);
+    String name = (locs[0].getNames())[0];
+    int dnIndex = 0;
+    for (DataNode dn : cluster.getDataNodes()) {
+      int port = dn.getXferPort();
+      if (name.contains(Integer.toString(port))) {
+        return dnIndex;
+      }
+      dnIndex++;
+    }
+    return -1;
+  }
+
+  /**
+   * Cross product of FILE_LENGTHS, NUM_PARITY_UNITS+1, NUM_PARITY_UNITS.
+   * Input for parameterized tests classes.
+   *
+   * @return Test parameters.
+   */
+  public static Collection<Object[]> getParameters() {
+    ArrayList<Object[]> params = new ArrayList<>();
+    for (int fileLength : FILE_LENGTHS) {
+      for (int dataDelNum = 1; dataDelNum <= NUM_PARITY_UNITS; dataDelNum++) {
+        for (int parityDelNum = 0;
+             (dataDelNum + parityDelNum) <= NUM_PARITY_UNITS; parityDelNum++) {
+          params.add(new Object[] {fileLength, dataDelNum, parityDelNum});
+        }
+      }
+    }
+    return params;
+  }
+
+  public static void verifyRead(DistributedFileSystem dfs, Path testPath,
+      int length, byte[] expected) throws IOException {
+    LOG.info("verifyRead on path {}", testPath);
+    byte[] buffer = new byte[length + 100];
+    LOG.info("verifyRead verifyLength on path {}", testPath);
+    StripedFileTestUtil.verifyLength(dfs, testPath, length);
+    LOG.info("verifyRead verifyPread on path {}", testPath);
+    StripedFileTestUtil.verifyPread(dfs, testPath, length, expected, buffer);
+    LOG.info("verifyRead verifyStatefulRead on path {}", testPath);
+    StripedFileTestUtil.verifyStatefulRead(dfs, testPath, length, expected,
+        buffer);
+    LOG.info("verifyRead verifyStatefulRead2 on path {}", testPath);
+    StripedFileTestUtil.verifyStatefulRead(dfs, testPath, length, expected,
+        ByteBuffer.allocate(length + 100));
+    LOG.info("verifyRead verifySeek on path {}", testPath);
+    StripedFileTestUtil.verifySeek(dfs, testPath, length, EC_POLICY,
+        BLOCK_GROUP_SIZE);
+  }
+
+  public static void testReadWithDNFailure(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, int fileLength, int dnFailureNum)
+      throws Exception {
+    String fileType = fileLength < (BLOCK_SIZE * NUM_DATA_UNITS) ?
+        "smallFile" : "largeFile";
+    String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
+    LOG.info("testReadWithDNFailure: file = " + src
+        + ", fileSize = " + fileLength
+        + ", dnFailureNum = " + dnFailureNum);
+
+    Path testPath = new Path(src);
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(dfs, testPath, bytes);
+    StripedFileTestUtil.waitBlockGroupsReported(dfs, src);
+
+    // shut down the DN that holds an internal data block
+    BlockLocation[] locs = dfs.getFileBlockLocations(testPath, CELL_SIZE * 5,
+        CELL_SIZE);
+    for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) {
+      String name = (locs[0].getNames())[failedDnIdx];
+      for (DataNode dn : cluster.getDataNodes()) {
+        int port = dn.getXferPort();
+        if (name.contains(Integer.toString(port))) {
+          dn.shutdown();
+        }
+      }
+    }
+
+    // check file length, pread, stateful read and seek
+    verifyRead(dfs, testPath, fileLength, bytes);
+  }
+
+
+  /**
+   * Test reading a file with some blocks(data blocks or parity blocks or both)
+   * deleted or corrupted.
+   * @param src file path
+   * @param fileNumBytes file length
+   * @param dataBlkDelNum the deleted or corrupted number of data blocks.
+   * @param parityBlkDelNum the deleted or corrupted number of parity blocks.
+   * @param deleteBlockFile whether block file is deleted or corrupted.
+   *                        true is to delete the block file.
+   *                        false is to corrupt the content of the block file.
+   * @throws IOException
+   */
+  public static void testReadWithBlockCorrupted(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, String src, int fileNumBytes,
+      int dataBlkDelNum, int parityBlkDelNum,
+      boolean deleteBlockFile) throws IOException {
+    LOG.info("testReadWithBlockCorrupted: file = " + src
+        + ", dataBlkDelNum = " + dataBlkDelNum
+        + ", parityBlkDelNum = " + parityBlkDelNum
+        + ", deleteBlockFile? " + deleteBlockFile);
+    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
+    Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
+        dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
+    Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
+        "should be between 1 ~ " + NUM_PARITY_UNITS, recoverBlkNum <=
+        NUM_PARITY_UNITS);
+
+    // write a file with the length of writeLen
+    Path srcPath = new Path(src);
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileNumBytes);
+    DFSTestUtil.writeFile(dfs, srcPath, bytes);
+
+    // delete or corrupt some blocks
+    corruptBlocks(cluster, dfs, srcPath, dataBlkDelNum, parityBlkDelNum,
+        deleteBlockFile);
+
+    // check the file can be read after some blocks were deleted
+    verifyRead(dfs, srcPath, fileNumBytes, bytes);
+  }
+
+  public static void corruptBlocks(MiniDFSCluster cluster,
+      DistributedFileSystem dfs, Path srcPath,
+      int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile)
+      throws IOException {
+    LOG.info("corruptBlocks on path {}", srcPath);
+    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
+
+    LocatedBlocks locatedBlocks = getLocatedBlocks(dfs, srcPath);
+    LocatedStripedBlock lastBlock =
+        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+
+    int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, NUM_DATA_UNITS,
+        dataBlkDelNum);
+    Assert.assertNotNull(delDataBlkIndices);
+    int[] delParityBlkIndices = StripedFileTestUtil.randomArray(NUM_DATA_UNITS,
+        NUM_DATA_UNITS + NUM_PARITY_UNITS, parityBlkDelNum);
+    Assert.assertNotNull(delParityBlkIndices);
+
+    int[] delBlkIndices = new int[recoverBlkNum];
+    System.arraycopy(delDataBlkIndices, 0,
+        delBlkIndices, 0, delDataBlkIndices.length);
+    System.arraycopy(delParityBlkIndices, 0,
+        delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length);
+
+    ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum];
+    for (int i = 0; i < recoverBlkNum; i++) {
+      delBlocks[i] = StripedBlockUtil
+          .constructInternalBlock(lastBlock.getBlock(),
+              CELL_SIZE, NUM_DATA_UNITS, delBlkIndices[i]);
+      if (deleteBlockFile) {
+        // delete the block file
+        LOG.info("Deleting block file {}", delBlocks[i]);
+        cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]);
+      } else {
+        // corrupt the block file
+        LOG.info("Corrupting block file {}", delBlocks[i]);
+        cluster.corruptBlockOnDataNodes(delBlocks[i]);
+      }
+    }
+  }
+
+  public static LocatedBlocks getLocatedBlocks(DistributedFileSystem dfs,
+      Path filePath) throws IOException {
+    return dfs.getClient().getLocatedBlocks(filePath.toString(),
+        0, Long.MAX_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index 057e94a..1489e48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.hdfs;
 
 import com.google.common.base.Joiner;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -27,12 +25,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 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.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.WebHdfsInputStream;
 import org.apache.hadoop.io.IOUtils;
@@ -40,6 +38,8 @@ import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -57,7 +57,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static org.junit.Assert.assertEquals;
 
 public class StripedFileTestUtil {
-  public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(StripedFileTestUtil.class);
 
   public static byte[] generateBytes(int cnt) {
     byte[] bytes = new byte[cnt];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
new file mode 100644
index 0000000..40ac206
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
@@ -0,0 +1,107 @@
+/**
+ * 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;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.BLOCK_SIZE;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.FILE_LENGTHS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_DATA_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_PARITY_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
+
+/**
+ * Test online recovery with failed DNs. This test is parameterized.
+ */
+@RunWith(Parameterized.class)
+public class TestReadStripedFileWithDNFailure {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDNFailure.class);
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem dfs;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    tearDownCluster(cluster);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getParameters() {
+    ArrayList<Object[]> params = new ArrayList<>();
+    for (int fileLength : FILE_LENGTHS) {
+      for (int i = 0; i < NUM_PARITY_UNITS; i++) {
+        params.add(new Object[] {fileLength, i+1});
+      }
+    }
+    return params;
+  }
+
+  private int fileLength;
+  private int dnFailureNum;
+
+  public TestReadStripedFileWithDNFailure(int fileLength, int dnFailureNum) {
+    this.fileLength = fileLength;
+    this.dnFailureNum = dnFailureNum;
+  }
+
+  /**
+   * Shutdown tolerable number of Datanode before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test
+  public void testReadWithDNFailure() throws Exception {
+    try {
+      // setup a new cluster with no dead datanode
+      setup();
+      ReadStripedFileWithDecodingHelper.testReadWithDNFailure(cluster,
+          dfs, fileLength, dnFailureNum);
+    } catch (IOException ioe) {
+      String fileType = fileLength < (BLOCK_SIZE * NUM_DATA_UNITS) ?
+          "smallFile" : "largeFile";
+      LOG.error("Failed to read file with DN failure:"
+          + " fileType = " + fileType
+          + ", dnFailureNum = " + dnFailureNum);
+    } finally {
+      // tear down the cluster
+      tearDown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
index cb1640c..2fb9212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -17,222 +17,58 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
-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.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Test;
 import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-public class TestReadStripedFileWithDecoding {
-  static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.CELL_SIZE;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_DATA_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_PARITY_UNITS;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.findFirstDataNode;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
 
-  static {
-    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
-        .getLogger().setLevel(Level.ALL);
-    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.ALL);
-    GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.ALL);
-  }
+public class TestReadStripedFileWithDecoding {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDecoding.class);
 
   private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private final ErasureCodingPolicy ecPolicy =
-      StripedFileTestUtil.getDefaultECPolicy();
-  private final short dataBlocks = (short) ecPolicy.getNumDataUnits();
-  private final short parityBlocks =
-      (short) ecPolicy.getNumParityUnits();
-  private final int numDNs = dataBlocks + parityBlocks;
-  private final int cellSize = ecPolicy.getCellSize();
-  private final int stripPerBlock = 4;
-  private final int blockSize = cellSize * stripPerBlock;
-  private final int blockGroupSize = blockSize * dataBlocks;
-  private final int smallFileLength = blockGroupSize - 123;
-  private final int largeFileLength = blockGroupSize + 123;
-  private final int[] fileLengths = {smallFileLength, largeFileLength};
-  private final int[] dnFailureNums = getDnFailureNums();
-
-  private int[] getDnFailureNums() {
-    int[] dnFailureNums = new int[parityBlocks];
-    for (int i = 0; i < dnFailureNums.length; i++) {
-      dnFailureNums[i] = i + 1;
-    }
-    return dnFailureNums;
-  }
+  private DistributedFileSystem dfs;
 
   @Rule
   public Timeout globalTimeout = new Timeout(300000);
 
   @Before
   public void setup() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, 0);
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
-        false);
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
-        StripedFileTestUtil.getDefaultECPolicy().getName());
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().setErasureCodingPolicy("/",
-        StripedFileTestUtil.getDefaultECPolicy().getName());
-    fs = cluster.getFileSystem();
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
   }
 
   @After
   public void tearDown() throws IOException {
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  /**
-   * Shutdown tolerable number of Datanode before reading.
-   * Verify the decoding works correctly.
-   */
-  @Test(timeout=300000)
-  public void testReadWithDNFailure() throws Exception {
-    for (int fileLength : fileLengths) {
-      for (int dnFailureNum : dnFailureNums) {
-        try {
-          // setup a new cluster with no dead datanode
-          setup();
-          testReadWithDNFailure(fileLength, dnFailureNum);
-        } catch (IOException ioe) {
-          String fileType = fileLength < (blockSize * dataBlocks) ?
-              "smallFile" : "largeFile";
-          LOG.error("Failed to read file with DN failure:"
-              + " fileType = "+ fileType
-              + ", dnFailureNum = " + dnFailureNum);
-        } finally {
-          // tear down the cluster
-          tearDown();
-        }
-      }
-    }
-  }
-
-  /**
-   * Corrupt tolerable number of block before reading.
-   * Verify the decoding works correctly.
-   */
-  @Test(timeout=300000)
-  public void testReadCorruptedData() throws IOException {
-    for (int fileLength : fileLengths) {
-      for (int dataDelNum = 1; dataDelNum <= parityBlocks; dataDelNum++) {
-        for (int parityDelNum = 0; (dataDelNum + parityDelNum) <= parityBlocks;
-             parityDelNum++) {
-          String src = "/corrupted_" + dataDelNum + "_" + parityDelNum;
-          testReadWithBlockCorrupted(src, fileLength,
-              dataDelNum, parityDelNum, false);
-        }
-      }
-    }
-  }
-
-  /**
-   * Delete tolerable number of block before reading.
-   * Verify the decoding works correctly.
-   */
-  @Test(timeout=300000)
-  public void testReadCorruptedDataByDeleting() throws IOException {
-    for (int fileLength : fileLengths) {
-      for (int dataDelNum = 1; dataDelNum <= parityBlocks; dataDelNum++) {
-        for (int parityDelNum = 0; (dataDelNum + parityDelNum) <= parityBlocks;
-             parityDelNum++) {
-          String src = "/deleted_" + dataDelNum + "_" + parityDelNum;
-          testReadWithBlockCorrupted(src, fileLength,
-              dataDelNum, parityDelNum, true);
-        }
-      }
-    }
-  }
-
-  private int findFirstDataNode(Path file, long length) throws IOException {
-    BlockLocation[] locs = fs.getFileBlockLocations(file, 0, length);
-    String name = (locs[0].getNames())[0];
-    int dnIndex = 0;
-    for (DataNode dn : cluster.getDataNodes()) {
-      int port = dn.getXferPort();
-      if (name.contains(Integer.toString(port))) {
-        return dnIndex;
-      }
-      dnIndex++;
-    }
-    return -1;
-  }
-
-  private void verifyRead(Path testPath, int length, byte[] expected)
-      throws IOException {
-    byte[] buffer = new byte[length + 100];
-    StripedFileTestUtil.verifyLength(fs, testPath, length);
-    StripedFileTestUtil.verifyPread(fs, testPath, length, expected, buffer);
-    StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, buffer);
-    StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected,
-        ByteBuffer.allocate(length + 100));
-    StripedFileTestUtil.verifySeek(fs, testPath, length, ecPolicy,
-        blockGroupSize);
-  }
-
-  private void testReadWithDNFailure(int fileLength, int dnFailureNum)
-      throws Exception {
-    String fileType = fileLength < (blockSize * dataBlocks) ?
-        "smallFile" : "largeFile";
-    String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
-    LOG.info("testReadWithDNFailure: file = " + src
-        + ", fileSize = " + fileLength
-        + ", dnFailureNum = " + dnFailureNum);
-
-    Path testPath = new Path(src);
-    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
-    DFSTestUtil.writeFile(fs, testPath, bytes);
-    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
-
-    // shut down the DN that holds an internal data block
-    BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5,
-        cellSize);
-    for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) {
-      String name = (locs[0].getNames())[failedDnIdx];
-      for (DataNode dn : cluster.getDataNodes()) {
-        int port = dn.getXferPort();
-        if (name.contains(Integer.toString(port))) {
-          dn.shutdown();
-        }
-      }
-    }
-
-    // check file length, pread, stateful read and seek
-    verifyRead(testPath, fileLength, bytes);
+    tearDownCluster(cluster);
   }
 
   /**
@@ -245,15 +81,17 @@ public class TestReadStripedFileWithDecoding {
     final Path file = new Path("/corrupted");
     final int length = 10; // length of "corruption"
     final byte[] bytes = StripedFileTestUtil.generateBytes(length);
-    DFSTestUtil.writeFile(fs, file, bytes);
+    DFSTestUtil.writeFile(dfs, file, bytes);
 
     // corrupt the first data block
-    int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
+    int dnIndex = ReadStripedFileWithDecodingHelper.findFirstDataNode(
+        cluster, dfs, file, CELL_SIZE * NUM_DATA_UNITS);
     Assert.assertNotEquals(-1, dnIndex);
-    LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
-        .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+    LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
+        .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
+        .get(0);
     final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
-        cellSize, dataBlocks, parityBlocks);
+        CELL_SIZE, NUM_DATA_UNITS, NUM_PARITY_UNITS);
     // find the first block file
     File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
     File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
@@ -272,7 +110,7 @@ public class TestReadStripedFileWithDecoding {
 
     try {
       // do stateful read
-      StripedFileTestUtil.verifyStatefulRead(fs, file, length, bytes,
+      StripedFileTestUtil.verifyStatefulRead(dfs, file, length, bytes,
           ByteBuffer.allocate(1024));
 
       // check whether the corruption has been reported to the NameNode
@@ -293,110 +131,35 @@ public class TestReadStripedFileWithDecoding {
     final Path file = new Path("/invalidate");
     final int length = 10;
     final byte[] bytes = StripedFileTestUtil.generateBytes(length);
-    DFSTestUtil.writeFile(fs, file, bytes);
+    DFSTestUtil.writeFile(dfs, file, bytes);
 
-    int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
+    int dnIndex = findFirstDataNode(cluster, dfs, file,
+        CELL_SIZE * NUM_DATA_UNITS);
     Assert.assertNotEquals(-1, dnIndex);
-    LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
-        .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+    LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
+        .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
+        .get(0);
     final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
-        cellSize, dataBlocks, parityBlocks);
+        CELL_SIZE, NUM_DATA_UNITS, NUM_PARITY_UNITS);
     final Block b = blks[0].getBlock().getLocalBlock();
 
     DataNode dn = cluster.getDataNodes().get(dnIndex);
-    // disable the heartbeat from DN so that the invalidated block record is kept
-    // in NameNode until heartbeat expires and NN mark the dn as dead
+    // disable the heartbeat from DN so that the invalidated block record is
+    // kept in NameNode until heartbeat expires and NN mark the dn as dead
     DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
 
     try {
       // delete the file
-      fs.delete(file, true);
+      dfs.delete(file, true);
       // check the block is added to invalidateBlocks
       final FSNamesystem fsn = cluster.getNamesystem();
       final BlockManager bm = fsn.getBlockManager();
-      DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
+      DatanodeDescriptor dnd =
+          NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
       Assert.assertTrue(bm.containsInvalidateBlock(
           blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b));
     } finally {
       DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
     }
   }
-
-  /**
-   * Test reading a file with some blocks(data blocks or parity blocks or both)
-   * deleted or corrupted.
-   * @param src file path
-   * @param fileLength file length
-   * @param dataBlkDelNum the deleted or corrupted number of data blocks.
-   * @param parityBlkDelNum the deleted or corrupted number of parity blocks.
-   * @param deleteBlockFile whether block file is deleted or corrupted.
-   *                        true is to delete the block file.
-   *                        false is to corrupt the content of the block file.
-   * @throws IOException
-   */
-  private void testReadWithBlockCorrupted(String src, int fileLength,
-      int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile)
-      throws IOException {
-    LOG.info("testReadWithBlockCorrupted: file = " + src
-        + ", dataBlkDelNum = " + dataBlkDelNum
-        + ", parityBlkDelNum = " + parityBlkDelNum
-        + ", deleteBlockFile? " + deleteBlockFile);
-    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
-    Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
-        dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
-    Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
-        "should be between 1 ~ " + parityBlocks, recoverBlkNum <= parityBlocks);
-
-    // write a file with the length of writeLen
-    Path srcPath = new Path(src);
-    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
-    DFSTestUtil.writeFile(fs, srcPath, bytes);
-
-    // delete or corrupt some blocks
-    corruptBlocks(srcPath, dataBlkDelNum, parityBlkDelNum, deleteBlockFile);
-
-    // check the file can be read after some blocks were deleted
-    verifyRead(srcPath, fileLength, bytes);
-  }
-
-  private void corruptBlocks(Path srcPath, int dataBlkDelNum,
-      int parityBlkDelNum, boolean deleteBlockFile) throws IOException {
-    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
-
-    LocatedBlocks locatedBlocks = getLocatedBlocks(srcPath);
-    LocatedStripedBlock lastBlock =
-        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
-
-    int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
-        dataBlkDelNum);
-    Assert.assertNotNull(delDataBlkIndices);
-    int[] delParityBlkIndices = StripedFileTestUtil.randomArray(dataBlocks,
-        dataBlocks + parityBlocks, parityBlkDelNum);
-    Assert.assertNotNull(delParityBlkIndices);
-
-    int[] delBlkIndices = new int[recoverBlkNum];
-    System.arraycopy(delDataBlkIndices, 0,
-        delBlkIndices, 0, delDataBlkIndices.length);
-    System.arraycopy(delParityBlkIndices, 0,
-        delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length);
-
-    ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum];
-    for (int i = 0; i < recoverBlkNum; i++) {
-      delBlocks[i] = StripedBlockUtil
-          .constructInternalBlock(lastBlock.getBlock(),
-              cellSize, dataBlocks, delBlkIndices[i]);
-      if (deleteBlockFile) {
-        // delete the block file
-        cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]);
-      } else {
-        // corrupt the block file
-        cluster.corruptBlockOnDataNodes(delBlocks[i]);
-      }
-    }
-  }
-
-  private LocatedBlocks getLocatedBlocks(Path filePath) throws IOException {
-    return fs.getClient().getLocatedBlocks(filePath.toString(),
-        0, Long.MAX_VALUE);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.java
new file mode 100644
index 0000000..5a8fb4f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingCorruptData.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;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
+
+/**
+ * Test online recovery with corrupt files. This test is parameterized.
+ */
+@RunWith(Parameterized.class)
+public class TestReadStripedFileWithDecodingCorruptData {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDecodingCorruptData.class);
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem dfs;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    tearDownCluster(cluster);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getParameters() {
+    return ReadStripedFileWithDecodingHelper.getParameters();
+  }
+
+  private int fileLength;
+  private int dataDelNum;
+  private int parityDelNum;
+
+  public TestReadStripedFileWithDecodingCorruptData(int fileLength, int
+      dataDelNum, int parityDelNum) {
+    this.fileLength = fileLength;
+    this.dataDelNum = dataDelNum;
+    this.parityDelNum = parityDelNum;
+  }
+
+  /**
+   * Corrupt tolerable number of block before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test
+  public void testReadCorruptedData() throws IOException {
+    String src = "/corrupted_" + dataDelNum + "_" + parityDelNum;
+    ReadStripedFileWithDecodingHelper.testReadWithBlockCorrupted(cluster,
+        dfs, src, fileLength, dataDelNum, parityDelNum, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4035d42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.java
new file mode 100644
index 0000000..c267e84
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecodingDeletedData.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;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
+import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
+
+/**
+ * Test online recovery with files with deleted blocks. This test is
+ * parameterized.
+ */
+@RunWith(Parameterized.class)
+public class TestReadStripedFileWithDecodingDeletedData {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestReadStripedFileWithDecodingDeletedData.class);
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem dfs;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    cluster = initializeCluster();
+    dfs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    tearDownCluster(cluster);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getParameters() {
+    return ReadStripedFileWithDecodingHelper.getParameters();
+  }
+
+  private int fileLength;
+  private int dataDelNum;
+  private int parityDelNum;
+
+  public TestReadStripedFileWithDecodingDeletedData(int fileLength, int
+      dataDelNum, int parityDelNum) {
+    this.fileLength = fileLength;
+    this.dataDelNum = dataDelNum;
+    this.parityDelNum = parityDelNum;
+  }
+
+  /**
+   * Delete tolerable number of block before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test
+  public void testReadCorruptedDataByDeleting() throws IOException {
+    String src = "/deleted_" + dataDelNum + "_" + parityDelNum;
+    ReadStripedFileWithDecodingHelper.testReadWithBlockCorrupted(cluster,
+        dfs, src, fileLength, dataDelNum, parityDelNum, true);
+  }
+}


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


[20/29] hadoop git commit: HDFS-10646. Federation admin tool. Contributed by Inigo Goiri.

Posted by in...@apache.org.
HDFS-10646. Federation admin tool. Contributed by Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 400b8e98b583d0a8185055fe89ab01c19d5751b2
Parents: 28bd21f
Author: Inigo Goiri <in...@apache.org>
Authored: Tue Aug 8 14:44:43 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   1 +
 .../hadoop-hdfs/src/main/bin/hdfs               |   5 +
 .../hadoop-hdfs/src/main/bin/hdfs.cmd           |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  19 ++
 .../hdfs/protocolPB/RouterAdminProtocolPB.java  |  44 +++
 ...uterAdminProtocolServerSideTranslatorPB.java | 151 ++++++++
 .../RouterAdminProtocolTranslatorPB.java        | 150 ++++++++
 .../resolver/MembershipNamenodeResolver.java    |  34 +-
 .../hdfs/server/federation/router/Router.java   |  52 +++
 .../federation/router/RouterAdminServer.java    | 183 ++++++++++
 .../server/federation/router/RouterClient.java  |  76 +++++
 .../hdfs/tools/federation/RouterAdmin.java      | 341 +++++++++++++++++++
 .../hdfs/tools/federation/package-info.java     |  28 ++
 .../src/main/proto/RouterProtocol.proto         |  47 +++
 .../src/main/resources/hdfs-default.xml         |  46 +++
 .../server/federation/RouterConfigBuilder.java  |  26 ++
 .../server/federation/RouterDFSCluster.java     |  43 ++-
 .../server/federation/StateStoreDFSCluster.java | 148 ++++++++
 .../federation/router/TestRouterAdmin.java      | 261 ++++++++++++++
 19 files changed, 1644 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/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 cc7a975..93216db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -332,6 +332,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>editlog.proto</include>
                   <include>fsimage.proto</include>
                   <include>FederationProtocol.proto</include>
+                  <include>RouterProtocol.proto</include>
                 </includes>
               </source>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index b1f44a4..d51a8e2 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -31,6 +31,7 @@ function hadoop_usage
   hadoop_add_option "--hosts filename" "list of hosts to use in worker mode"
   hadoop_add_option "--workers" "turn on worker mode"
 
+<<<<<<< HEAD
   hadoop_add_subcommand "balancer" daemon "run a cluster balancing utility"
   hadoop_add_subcommand "cacheadmin" admin "configure the HDFS cache"
   hadoop_add_subcommand "classpath" client "prints the class path needed to get the hadoop jar and the required libraries"
@@ -42,6 +43,7 @@ function hadoop_usage
   hadoop_add_subcommand "diskbalancer" daemon "Distributes data evenly among disks on a given node"
   hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables"
   hadoop_add_subcommand "ec" admin "run a HDFS ErasureCoding CLI"
+  hadoop_add_subcommand "federation" admin "manage Router-based federation"
   hadoop_add_subcommand "fetchdt" client "fetch a delegation token from the NameNode"
   hadoop_add_subcommand "fsck" admin "run a DFS filesystem checking utility"
   hadoop_add_subcommand "getconf" client "get config values from configuration"
@@ -181,6 +183,9 @@ function hdfscmd_case
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.federation.router.Router'
     ;;
+    federation)
+      HADOOP_CLASSNAME='org.apache.hadoop.hdfs.tools.federation.RouterAdmin'
+    ;;
     secondarynamenode)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode'

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
index b9853d6..53bdf70 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
@@ -59,7 +59,7 @@ if "%1" == "--loglevel" (
     )
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto router debug
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto router federation debug
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -184,6 +184,11 @@ goto :eof
   set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ROUTER_OPTS%
   goto :eof
 
+:federation
+  set CLASS=org.apache.hadoop.hdfs.tools.federation.RouterAdmin
+  set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ROUTER_OPTS%
+  goto :eof
+
 :debug
   set CLASS=org.apache.hadoop.hdfs.tools.DebugAdmin
   goto :eof

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/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 5433df3..65bbc27 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
@@ -1190,6 +1190,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String FEDERATION_STORE_PREFIX =
       FEDERATION_ROUTER_PREFIX + "store.";
 
+  public static final String DFS_ROUTER_STORE_ENABLE =
+      FEDERATION_STORE_PREFIX + "enable";
+  public static final boolean DFS_ROUTER_STORE_ENABLE_DEFAULT = true;
+
   public static final String FEDERATION_STORE_SERIALIZER_CLASS =
       DFSConfigKeys.FEDERATION_STORE_PREFIX + "serializer";
   public static final Class<StateStoreSerializerPBImpl>
@@ -1216,6 +1220,21 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT =
       TimeUnit.MINUTES.toMillis(5);
 
+  // HDFS Router-based federation admin
+  public static final String DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "admin.handler.count";
+  public static final int DFS_ROUTER_ADMIN_HANDLER_COUNT_DEFAULT = 1;
+  public static final int    DFS_ROUTER_ADMIN_PORT_DEFAULT = 8111;
+  public static final String DFS_ROUTER_ADMIN_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "admin-address";
+  public static final String DFS_ROUTER_ADMIN_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_ADMIN_PORT_DEFAULT;
+  public static final String DFS_ROUTER_ADMIN_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "admin-bind-host";
+  public static final String DFS_ROUTER_ADMIN_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "admin.enable";
+  public static final boolean DFS_ROUTER_ADMIN_ENABLE_DEFAULT = true;
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolPB.java
new file mode 100644
index 0000000..96fa794
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolPB.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.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.RouterProtocolProtos.RouterAdminProtocolService;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+
+/**
+ * Protocol that a clients use to communicate with the NameNode.
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
+    protocolVersion = 1)
+public interface RouterAdminProtocolPB extends
+    RouterAdminProtocolService.BlockingInterface {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..415bbd9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java
@@ -0,0 +1,151 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryResponsePBImpl;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is used on the server side. Calls come across the wire for the for
+ * protocol {@link RouterAdminProtocolPB}. This class translates the PB data
+ * types to the native data types used inside the HDFS Router as specified in
+ * the generic RouterAdminProtocol.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class RouterAdminProtocolServerSideTranslatorPB implements
+    RouterAdminProtocolPB {
+
+  private final RouterAdminServer server;
+
+  /**
+   * Constructor.
+   * @param server The NN server.
+   * @throws IOException
+   */
+  public RouterAdminProtocolServerSideTranslatorPB(RouterAdminServer server)
+      throws IOException {
+    this.server = server;
+  }
+
+  @Override
+  public AddMountTableEntryResponseProto addMountTableEntry(
+      RpcController controller, AddMountTableEntryRequestProto request)
+      throws ServiceException {
+
+    try {
+      AddMountTableEntryRequest req =
+          new AddMountTableEntryRequestPBImpl(request);
+      AddMountTableEntryResponse response = server.addMountTableEntry(req);
+      AddMountTableEntryResponsePBImpl responsePB =
+          (AddMountTableEntryResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  /**
+   * Remove an entry from the mount table.
+   */
+  @Override
+  public RemoveMountTableEntryResponseProto removeMountTableEntry(
+      RpcController controller, RemoveMountTableEntryRequestProto request)
+      throws ServiceException {
+    try {
+      RemoveMountTableEntryRequest req =
+          new RemoveMountTableEntryRequestPBImpl(request);
+      RemoveMountTableEntryResponse response =
+          server.removeMountTableEntry(req);
+      RemoveMountTableEntryResponsePBImpl responsePB =
+          (RemoveMountTableEntryResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  /**
+   * Get matching mount table entries.
+   */
+  @Override
+  public GetMountTableEntriesResponseProto getMountTableEntries(
+      RpcController controller, GetMountTableEntriesRequestProto request)
+          throws ServiceException {
+    try {
+      GetMountTableEntriesRequest req =
+          new GetMountTableEntriesRequestPBImpl(request);
+      GetMountTableEntriesResponse response = server.getMountTableEntries(req);
+      GetMountTableEntriesResponsePBImpl responsePB =
+          (GetMountTableEntriesResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  /**
+   * Update a single mount table entry.
+   */
+  @Override
+  public UpdateMountTableEntryResponseProto updateMountTableEntry(
+      RpcController controller, UpdateMountTableEntryRequestProto request)
+          throws ServiceException {
+    try {
+      UpdateMountTableEntryRequest req =
+          new UpdateMountTableEntryRequestPBImpl(request);
+      UpdateMountTableEntryResponse response =
+          server.updateMountTableEntry(req);
+      UpdateMountTableEntryResponsePBImpl responsePB =
+          (UpdateMountTableEntryResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java
new file mode 100644
index 0000000..43663ac
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java
@@ -0,0 +1,150 @@
+/**
+ * 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 java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryResponsePBImpl;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.ProtocolTranslator;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
+ * while translating from the parameter types used in ClientProtocol to the
+ * new PB types.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class RouterAdminProtocolTranslatorPB
+    implements ProtocolMetaInterface, MountTableManager,
+    Closeable, ProtocolTranslator {
+  final private RouterAdminProtocolPB rpcProxy;
+
+  public RouterAdminProtocolTranslatorPB(RouterAdminProtocolPB proxy) {
+    rpcProxy = proxy;
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public Object getUnderlyingProxyObject() {
+    return rpcProxy;
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        RouterAdminProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(RouterAdminProtocolPB.class), methodName);
+  }
+
+  @Override
+  public AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException {
+    AddMountTableEntryRequestPBImpl requestPB =
+        (AddMountTableEntryRequestPBImpl)request;
+    AddMountTableEntryRequestProto proto = requestPB.getProto();
+    try {
+      AddMountTableEntryResponseProto response =
+          rpcProxy.addMountTableEntry(null, proto);
+      return new AddMountTableEntryResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException {
+    UpdateMountTableEntryRequestPBImpl requestPB =
+        (UpdateMountTableEntryRequestPBImpl)request;
+    UpdateMountTableEntryRequestProto proto = requestPB.getProto();
+    try {
+      UpdateMountTableEntryResponseProto response =
+          rpcProxy.updateMountTableEntry(null, proto);
+      return new UpdateMountTableEntryResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException {
+    RemoveMountTableEntryRequestPBImpl requestPB =
+        (RemoveMountTableEntryRequestPBImpl)request;
+    RemoveMountTableEntryRequestProto proto = requestPB.getProto();
+    try {
+      RemoveMountTableEntryResponseProto responseProto =
+          rpcProxy.removeMountTableEntry(null, proto);
+      return new RemoveMountTableEntryResponsePBImpl(responseProto);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException {
+    GetMountTableEntriesRequestPBImpl requestPB =
+        (GetMountTableEntriesRequestPBImpl)request;
+    GetMountTableEntriesRequestProto proto = requestPB.getProto();
+    try {
+      GetMountTableEntriesResponseProto response =
+          rpcProxy.getMountTableEntries(null, proto);
+      return new GetMountTableEntriesResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/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 b0ced24..d974c78 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
@@ -61,7 +61,7 @@ public class MembershipNamenodeResolver
   /** Reference to the State Store. */
   private final StateStoreService stateStore;
   /** Membership State Store interface. */
-  private final MembershipStore membershipInterface;
+  private MembershipStore membershipInterface;
 
   /** Parent router ID. */
   private String routerId;
@@ -82,25 +82,27 @@ public class MembershipNamenodeResolver
     if (this.stateStore != null) {
       // Request cache updates from the state store
       this.stateStore.registerCacheExternal(this);
-
-      // Initialize the interface to get the membership
-      this.membershipInterface = this.stateStore.getRegisteredRecordStore(
-          MembershipStore.class);
-    } else {
-      this.membershipInterface = null;
     }
+  }
 
+  private synchronized MembershipStore getMembershipStore() throws IOException {
     if (this.membershipInterface == null) {
-      throw new IOException("State Store does not have an interface for " +
-          MembershipStore.class.getSimpleName());
+      this.membershipInterface = this.stateStore.getRegisteredRecordStore(
+          MembershipStore.class);
+      if (this.membershipInterface == null) {
+        throw new IOException("State Store does not have an interface for " +
+            MembershipStore.class.getSimpleName());
+      }
     }
+    return this.membershipInterface;
   }
 
   @Override
   public boolean loadCache(boolean force) {
     // Our cache depends on the store, update it first
     try {
-      this.membershipInterface.loadCache(force);
+      MembershipStore membership = getMembershipStore();
+      membership.loadCache(force);
     } catch (IOException e) {
       LOG.error("Cannot update membership from the State Store", e);
     }
@@ -126,8 +128,9 @@ public class MembershipNamenodeResolver
       GetNamenodeRegistrationsRequest request =
           GetNamenodeRegistrationsRequest.newInstance(partial);
 
+      MembershipStore membership = getMembershipStore();
       GetNamenodeRegistrationsResponse response =
-          this.membershipInterface.getNamenodeRegistrations(request);
+          membership.getNamenodeRegistrations(request);
       List<MembershipState> records = response.getNamenodeMemberships();
 
       if (records != null && records.size() == 1) {
@@ -135,7 +138,7 @@ public class MembershipNamenodeResolver
         UpdateNamenodeRegistrationRequest updateRequest =
             UpdateNamenodeRegistrationRequest.newInstance(
                 record.getNameserviceId(), record.getNamenodeId(), ACTIVE);
-        this.membershipInterface.updateNamenodeRegistration(updateRequest);
+        membership.updateNamenodeRegistration(updateRequest);
       }
     } catch (StateStoreUnavailableException e) {
       LOG.error("Cannot update {} as active, State Store unavailable", address);
@@ -226,14 +229,14 @@ public class MembershipNamenodeResolver
 
     NamenodeHeartbeatRequest request = NamenodeHeartbeatRequest.newInstance();
     request.setNamenodeMembership(record);
-    return this.membershipInterface.namenodeHeartbeat(request).getResult();
+    return getMembershipStore().namenodeHeartbeat(request).getResult();
   }
 
   @Override
   public Set<FederationNamespaceInfo> getNamespaces() throws IOException {
     GetNamespaceInfoRequest request = GetNamespaceInfoRequest.newInstance();
     GetNamespaceInfoResponse response =
-        this.membershipInterface.getNamespaceInfo(request);
+        getMembershipStore().getNamespaceInfo(request);
     return response.getNamespaceInfo();
   }
 
@@ -259,8 +262,9 @@ public class MembershipNamenodeResolver
     // Retrieve a list of all registrations that match this query.
     // This may include all NN records for a namespace/blockpool, including
     // duplicate records for the same NN from different routers.
+    MembershipStore membershipStore = getMembershipStore();
     GetNamenodeRegistrationsResponse response =
-        this.membershipInterface.getNamenodeRegistrations(request);
+        membershipStore.getNamenodeRegistrations(request);
 
     List<MembershipState> memberships = response.getNamenodeMemberships();
     if (!addExpired || !addUnavailable) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index 213a58f..fcbd2eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -81,6 +81,10 @@ public class Router extends CompositeService {
   private RouterRpcServer rpcServer;
   private InetSocketAddress rpcAddress;
 
+  /** RPC interface for the admin. */
+  private RouterAdminServer adminServer;
+  private InetSocketAddress adminAddress;
+
   /** Interface with the State Store. */
   private StateStoreService stateStore;
 
@@ -116,6 +120,14 @@ public class Router extends CompositeService {
   protected void serviceInit(Configuration configuration) throws Exception {
     this.conf = configuration;
 
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_STORE_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_STORE_ENABLE_DEFAULT)) {
+      // Service that maintains the State Store connection
+      this.stateStore = new StateStoreService();
+      addService(this.stateStore);
+    }
+
     // Resolver to track active NNs
     this.namenodeResolver = newActiveNamenodeResolver(
         this.conf, this.stateStore);
@@ -139,6 +151,14 @@ public class Router extends CompositeService {
     }
 
     if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ENABLE_DEFAULT)) {
+      // Create admin server
+      this.adminServer = createAdminServer();
+      addService(this.adminServer);
+    }
+
+    if (conf.getBoolean(
         DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE,
         DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT)) {
 
@@ -264,6 +284,38 @@ public class Router extends CompositeService {
   }
 
   /////////////////////////////////////////////////////////
+  // Admin server
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create a new router admin server to handle the router admin interface.
+   *
+   * @return RouterAdminServer
+   * @throws IOException If the admin server was not successfully started.
+   */
+  protected RouterAdminServer createAdminServer() throws IOException {
+    return new RouterAdminServer(this.conf, this);
+  }
+
+  /**
+   * Set the current Admin socket for the router.
+   *
+   * @param adminAddress Admin RPC address.
+   */
+  protected void setAdminServerAddress(InetSocketAddress address) {
+    this.adminAddress = address;
+  }
+
+  /**
+   * Get the current Admin socket address for the router.
+   *
+   * @return InetSocketAddress Admin address.
+   */
+  public InetSocketAddress getAdminServerAddress() {
+    return adminAddress;
+  }
+
+  /////////////////////////////////////////////////////////
   // Namenode heartbeat monitors
   /////////////////////////////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/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
new file mode 100644
index 0000000..7687216
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.RouterProtocolProtos.RouterAdminProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.service.AbstractService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+
+/**
+ * This class is responsible for handling all of the Admin calls to the HDFS
+ * router. It is created, started, and stopped by {@link Router}.
+ */
+public class RouterAdminServer extends AbstractService
+    implements MountTableManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterAdminServer.class);
+
+  private Configuration conf;
+
+  private final Router router;
+
+  private MountTableStore mountTableStore;
+
+  /** The Admin server that listens to requests from clients. */
+  private final Server adminServer;
+  private final InetSocketAddress adminAddress;
+
+  public RouterAdminServer(Configuration conf, Router router)
+      throws IOException {
+    super(RouterAdminServer.class.getName());
+
+    this.conf = conf;
+    this.router = router;
+
+    int handlerCount = this.conf.getInt(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_HANDLER_COUNT_DEFAULT);
+
+    RPC.setProtocolEngine(this.conf, RouterAdminProtocolPB.class,
+        ProtobufRpcEngine.class);
+
+    RouterAdminProtocolServerSideTranslatorPB routerAdminProtocolTranslator =
+        new RouterAdminProtocolServerSideTranslatorPB(this);
+    BlockingService clientNNPbService = RouterAdminProtocolService.
+        newReflectiveBlockingService(routerAdminProtocolTranslator);
+
+    InetSocketAddress confRpcAddress = conf.getSocketAddr(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_DEFAULT,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_PORT_DEFAULT);
+
+    String bindHost = conf.get(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY,
+        confRpcAddress.getHostName());
+    LOG.info("Admin server binding to {}:{}",
+        bindHost, confRpcAddress.getPort());
+
+    this.adminServer = new RPC.Builder(this.conf)
+        .setProtocol(RouterAdminProtocolPB.class)
+        .setInstance(clientNNPbService)
+        .setBindAddress(bindHost)
+        .setPort(confRpcAddress.getPort())
+        .setNumHandlers(handlerCount)
+        .setVerbose(false)
+        .build();
+
+    // The RPC-server port can be ephemeral... ensure we have the correct info
+    InetSocketAddress listenAddress = this.adminServer.getListenerAddress();
+    this.adminAddress = new InetSocketAddress(
+        confRpcAddress.getHostName(), listenAddress.getPort());
+    router.setAdminServerAddress(this.adminAddress);
+  }
+
+  /** Allow access to the client RPC server for testing. */
+  @VisibleForTesting
+  Server getAdminServer() {
+    return this.adminServer;
+  }
+
+  private MountTableStore getMountTableStore() throws IOException {
+    if (this.mountTableStore == null) {
+      this.mountTableStore = router.getStateStore().getRegisteredRecordStore(
+          MountTableStore.class);
+      if (this.mountTableStore == null) {
+        throw new IOException("Mount table state store is not available.");
+      }
+    }
+    return this.mountTableStore;
+  }
+
+  /**
+   * Get the RPC address of the admin service.
+   * @return Administration service RPC address.
+   */
+  public InetSocketAddress getRpcAddress() {
+    return this.adminAddress;
+  }
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    this.adminServer.start();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (this.adminServer != null) {
+      this.adminServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  @Override
+  public AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException {
+    return getMountTableStore().addMountTableEntry(request);
+  }
+
+  @Override
+  public UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException {
+    return getMountTableStore().updateMountTableEntry(request);
+  }
+
+  @Override
+  public RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException {
+    return getMountTableStore().removeMountTableEntry(request);
+  }
+
+  @Override
+  public GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException {
+    return getMountTableStore().getMountTableEntries(request);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClient.java
new file mode 100644
index 0000000..1f76b98
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClient.java
@@ -0,0 +1,76 @@
+/**
+ * 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 java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Client to connect to the {@link Router} via the admin protocol.
+ */
+@Private
+public class RouterClient implements Closeable {
+
+  private final RouterAdminProtocolTranslatorPB proxy;
+  private final UserGroupInformation ugi;
+
+  private static RouterAdminProtocolTranslatorPB createRouterProxy(
+      InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
+          throws IOException {
+
+    RPC.setProtocolEngine(
+        conf, RouterAdminProtocolPB.class, ProtobufRpcEngine.class);
+
+    AtomicBoolean fallbackToSimpleAuth = new AtomicBoolean(false);
+    final long version = RPC.getProtocolVersion(RouterAdminProtocolPB.class);
+    RouterAdminProtocolPB proxy = RPC.getProtocolProxy(
+        RouterAdminProtocolPB.class, version, address, ugi, conf,
+        NetUtils.getDefaultSocketFactory(conf),
+        RPC.getRpcTimeout(conf), null,
+        fallbackToSimpleAuth).getProxy();
+
+    return new RouterAdminProtocolTranslatorPB(proxy);
+  }
+
+  public RouterClient(InetSocketAddress address, Configuration conf)
+      throws IOException {
+    this.ugi = UserGroupInformation.getCurrentUser();
+    this.proxy = createRouterProxy(address, conf, ugi);
+  }
+
+  public MountTableManager getMountTableManager() {
+    return proxy;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    RPC.stopProxy(proxy);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/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
new file mode 100644
index 0000000..0786419
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.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.tools.federation;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.LinkedHashMap;
+import java.util.List;
+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.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+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.RouterClient;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+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.protocol.RemoveMountTableEntryRequest;
+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.federation.store.records.MountTable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class provides some Federation administrative access shell commands.
+ */
+@Private
+public class RouterAdmin extends Configured implements Tool {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RouterAdmin.class);
+
+  private RouterClient client;
+
+  public static void main(String[] argv) throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    RouterAdmin admin = new RouterAdmin(conf);
+
+    int res = ToolRunner.run(admin, argv);
+    System.exit(res);
+  }
+
+  public RouterAdmin(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * Print the usage message.
+   */
+  public void printUsage() {
+    String usage = "Federation Admin Tools:\n"
+        + "\t[-add <source> <nameservice> <destination> "
+        + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL]]\n"
+        + "\t[-rm <source>]\n"
+        + "\t[-ls <path>]\n";
+    System.out.println(usage);
+  }
+
+  @Override
+  public int run(String[] argv) throws Exception {
+    if (argv.length < 1) {
+      System.err.println("Not enough parameters specificed");
+      printUsage();
+      return -1;
+    }
+
+    int exitCode = -1;
+    int i = 0;
+    String cmd = argv[i++];
+
+    // Verify that we have enough command line parameters
+    if ("-add".equals(cmd)) {
+      if (argv.length < 4) {
+        System.err.println("Not enough parameters specificed for cmd " + cmd);
+        printUsage();
+        return exitCode;
+      }
+    } else if ("-rm".equalsIgnoreCase(cmd)) {
+      if (argv.length < 2) {
+        System.err.println("Not enough parameters specificed for cmd " + cmd);
+        printUsage();
+        return exitCode;
+      }
+    }
+
+    // Initialize RouterClient
+    try {
+      String address = getConf().getTrimmed(
+          DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY,
+          DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_DEFAULT);
+      InetSocketAddress routerSocket = NetUtils.createSocketAddr(address);
+      client = new RouterClient(routerSocket, getConf());
+    } catch (RPC.VersionMismatch v) {
+      System.err.println(
+          "Version mismatch between client and server... command aborted");
+      return exitCode;
+    } catch (IOException e) {
+      System.err.println("Bad connection to Router... command aborted");
+      return exitCode;
+    }
+
+    Exception debugException = null;
+    exitCode = 0;
+    try {
+      if ("-add".equals(cmd)) {
+        if (addMount(argv, i)) {
+          System.err.println("Successfuly added mount point " + argv[i]);
+        }
+      } else if ("-rm".equals(cmd)) {
+        if (removeMount(argv[i])) {
+          System.err.println("Successfully removed mount point " + argv[i]);
+        }
+      } else if ("-ls".equals(cmd)) {
+        if (argv.length > 1) {
+          listMounts(argv[i]);
+        } else {
+          listMounts("/");
+        }
+      } else {
+        printUsage();
+        return exitCode;
+      }
+    } catch (IllegalArgumentException arge) {
+      debugException = arge;
+      exitCode = -1;
+      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
+      printUsage();
+    } catch (RemoteException e) {
+      // This is a error returned by the server.
+      // Print out the first line of the error message, ignore the stack trace.
+      exitCode = -1;
+      debugException = e;
+      try {
+        String[] content;
+        content = e.getLocalizedMessage().split("\n");
+        System.err.println(cmd.substring(1) + ": " + content[0]);
+        e.printStackTrace();
+      } catch (Exception ex) {
+        System.err.println(cmd.substring(1) + ": " + ex.getLocalizedMessage());
+        e.printStackTrace();
+        debugException = ex;
+      }
+    } catch (Exception e) {
+      exitCode = -1;
+      debugException = e;
+      System.err.println(cmd.substring(1) + ": " + e.getLocalizedMessage());
+      e.printStackTrace();
+    }
+    if (debugException != null) {
+      LOG.debug("Exception encountered", debugException);
+    }
+    return exitCode;
+  }
+
+  /**
+   * Add a mount table entry or update if it exists.
+   *
+   * @param parameters Parameters for the mount point.
+   * @param i Index in the parameters.
+   */
+  public boolean addMount(String[] parameters, int i) throws IOException {
+    // Mandatory parameters
+    String mount = parameters[i++];
+    String[] nss = parameters[i++].split(",");
+    String dest = parameters[i++];
+
+    // Optional parameters
+    boolean readOnly = false;
+    DestinationOrder order = DestinationOrder.HASH;
+    while (i < parameters.length) {
+      if (parameters[i].equals("-readonly")) {
+        readOnly = true;
+      } else if (parameters[i].equals("-order")) {
+        i++;
+        try {
+          order = DestinationOrder.valueOf(parameters[i]);
+        } catch(Exception e) {
+          System.err.println("Cannot parse order: " + parameters[i]);
+        }
+      }
+      i++;
+    }
+
+    return addMount(mount, nss, dest, readOnly, order);
+  }
+
+  /**
+   * Add a mount table entry or update if it exists.
+   *
+   * @param mount Mount point.
+   * @param nss Namespaces where this is mounted to.
+   * @param dest Destination path.
+   * @param readonly If the mount point is read only.
+   * @param order Order of the destination locations.
+   * @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 {
+    // Get the existing entry
+    MountTableManager mountTable = client.getMountTableManager();
+    GetMountTableEntriesRequest getRequest =
+        GetMountTableEntriesRequest.newInstance(mount);
+    GetMountTableEntriesResponse getResponse =
+        mountTable.getMountTableEntries(getRequest);
+    List<MountTable> results = getResponse.getEntries();
+    MountTable existingEntry = null;
+    for (MountTable result : results) {
+      if (mount.equals(result.getSourcePath())) {
+        existingEntry = result;
+      }
+    }
+
+    if (existingEntry == null) {
+      // Create and add the entry if it doesn't exist
+      Map<String, String> destMap = new LinkedHashMap<>();
+      for (String ns : nss) {
+        destMap.put(ns, dest);
+      }
+      MountTable newEntry = MountTable.newInstance(mount, destMap);
+      if (readonly) {
+        newEntry.setReadOnly(true);
+      }
+      if (order != null) {
+        newEntry.setDestOrder(order);
+      }
+      AddMountTableEntryRequest request =
+          AddMountTableEntryRequest.newInstance(newEntry);
+      AddMountTableEntryResponse addResponse =
+          mountTable.addMountTableEntry(request);
+      boolean added = addResponse.getStatus();
+      if (!added) {
+        System.err.println("Cannot add mount point " + mount);
+      }
+      return added;
+    } else {
+      // Update the existing entry if it exists
+      for (String nsId : nss) {
+        if (!existingEntry.addDestination(nsId, dest)) {
+          System.err.println("Cannot add destination at " + nsId + " " + dest);
+        }
+      }
+      if (readonly) {
+        existingEntry.setReadOnly(true);
+      }
+      if (order != null) {
+        existingEntry.setDestOrder(order);
+      }
+      UpdateMountTableEntryRequest updateRequest =
+          UpdateMountTableEntryRequest.newInstance(existingEntry);
+      UpdateMountTableEntryResponse updateResponse =
+          mountTable.updateMountTableEntry(updateRequest);
+      boolean updated = updateResponse.getStatus();
+      if (!updated) {
+        System.err.println("Cannot update mount point " + mount);
+      }
+      return updated;
+    }
+  }
+
+  /**
+   * Remove mount point.
+   *
+   * @param path Path to remove.
+   * @throws IOException If it cannot be removed.
+   */
+  public boolean removeMount(String path) throws IOException {
+    MountTableManager mountTable = client.getMountTableManager();
+    RemoveMountTableEntryRequest request =
+        RemoveMountTableEntryRequest.newInstance(path);
+    RemoveMountTableEntryResponse response =
+        mountTable.removeMountTableEntry(request);
+    boolean removed = response.getStatus();
+    if (!removed) {
+      System.out.println("Cannot remove mount point " + path);
+    }
+    return removed;
+  }
+
+  /**
+   * List mount points.
+   *
+   * @param path Path to list.
+   * @throws IOException If it cannot be listed.
+   */
+  public void listMounts(String path) throws IOException {
+    MountTableManager mountTable = client.getMountTableManager();
+    GetMountTableEntriesRequest request =
+        GetMountTableEntriesRequest.newInstance(path);
+    GetMountTableEntriesResponse response =
+        mountTable.getMountTableEntries(request);
+    List<MountTable> entries = response.getEntries();
+    printMounts(entries);
+  }
+
+  private static void printMounts(List<MountTable> entries) {
+    System.out.println("Mount Table Entries:");
+    System.out.println(String.format(
+        "%-25s %-25s",
+        "Source", "Destinations"));
+    for (MountTable entry : entries) {
+      StringBuilder destBuilder = new StringBuilder();
+      for (RemoteLocation location : entry.getDestinations()) {
+        if (destBuilder.length() > 0) {
+          destBuilder.append(",");
+        }
+        destBuilder.append(String.format("%s->%s", location.getNameserviceId(),
+            location.getDest()));
+      }
+      System.out.println(String.format("%-25s %-25s", entry.getSourcePath(),
+          destBuilder.toString()));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/package-info.java
new file mode 100644
index 0000000..466c3d3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+/**
+ * It includes the tools to manage the Router-based federation. Includes the
+ * utilities to add and remove mount table entries.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.tools.federation;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RouterProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RouterProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RouterProtocol.proto
new file mode 100644
index 0000000..3f43040
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RouterProtocol.proto
@@ -0,0 +1,47 @@
+/**
+ * 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 = "RouterProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs.router;
+
+import "FederationProtocol.proto";
+
+service RouterAdminProtocolService {
+  /**
+   * Add a mount table entry.
+   */
+  rpc addMountTableEntry(AddMountTableEntryRequestProto) returns(AddMountTableEntryResponseProto);
+
+  /**
+   * Update an existing mount table entry without copying files.
+   */
+  rpc updateMountTableEntry(UpdateMountTableEntryRequestProto) returns(UpdateMountTableEntryResponseProto);
+
+  /**
+   * Remove a mount table entry.
+   */
+  rpc removeMountTableEntry(RemoveMountTableEntryRequestProto) returns(RemoveMountTableEntryResponseProto);
+
+  /**
+   * Get matching mount entries
+   */
+  rpc getMountTableEntries(GetMountTableEntriesRequestProto) returns(GetMountTableEntriesResponseProto);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/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 6f00261..faa40fa 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
@@ -4714,6 +4714,44 @@
     </description>
   </property>
 
+    <property>
+    <name>dfs.federation.router.admin.enable</name>
+    <value>true</value>
+    <description>
+      If the RPC admin service to handle client requests in the router is
+      enabled.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.admin-address</name>
+    <value>0.0.0.0:8111</value>
+    <description>
+      RPC address that handles the admin requests.
+      The value of this property will take the form of router-host1:rpc-port.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.admin-bind-host</name>
+    <value></value>
+    <description>
+      The actual address the RPC admin server will bind to. If this optional
+      address is set, it overrides only the hostname portion of
+      dfs.federation.router.admin-address. This is useful for making the name
+      node listen on all interfaces by setting it to 0.0.0.0.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.admin.handler.count</name>
+    <value>1</value>
+    <description>
+      The number of server threads for the router to handle RPC requests from
+      admin.
+    </description>
+  </property>
+
   <property>
     <name>dfs.federation.router.file.resolver.client.class</name>
     <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
@@ -4731,6 +4769,14 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.store.enable</name>
+    <value>true</value>
+    <description>
+      If the Router connects to the State Store.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.store.serializer</name>
     <value>org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
index 21555c5..cac5e6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
@@ -28,8 +28,10 @@ public class RouterConfigBuilder {
   private Configuration conf;
 
   private boolean enableRpcServer = false;
+  private boolean enableAdminServer = false;
   private boolean enableHeartbeat = false;
   private boolean enableLocalHeartbeat = false;
+  private boolean enableStateStore = false;
 
   public RouterConfigBuilder(Configuration configuration) {
     this.conf = configuration;
@@ -41,8 +43,10 @@ public class RouterConfigBuilder {
 
   public RouterConfigBuilder all() {
     this.enableRpcServer = true;
+    this.enableAdminServer = true;
     this.enableHeartbeat = true;
     this.enableLocalHeartbeat = true;
+    this.enableStateStore = true;
     return this;
   }
 
@@ -56,21 +60,43 @@ public class RouterConfigBuilder {
     return this;
   }
 
+  public RouterConfigBuilder admin(boolean enable) {
+    this.enableAdminServer = enable;
+    return this;
+  }
+
   public RouterConfigBuilder heartbeat(boolean enable) {
     this.enableHeartbeat = enable;
     return this;
   }
 
+  public RouterConfigBuilder stateStore(boolean enable) {
+    this.enableStateStore = enable;
+    return this;
+  }
+
   public RouterConfigBuilder rpc() {
     return this.rpc(true);
   }
 
+  public RouterConfigBuilder admin() {
+    return this.admin(true);
+  }
+
   public RouterConfigBuilder heartbeat() {
     return this.heartbeat(true);
   }
 
+  public RouterConfigBuilder stateStore() {
+    return this.stateStore(true);
+  }
+
   public Configuration build() {
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_STORE_ENABLE,
+        this.enableStateStore);
     conf.setBoolean(DFSConfigKeys.DFS_ROUTER_RPC_ENABLE, this.enableRpcServer);
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_ADMIN_ENABLE,
+        this.enableAdminServer);
     conf.setBoolean(DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE,
         this.enableHeartbeat);
     conf.setBoolean(DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
index 0830c19..1ee49d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -25,8 +25,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE_MS;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HEARTBEAT_INTERVAL_MS;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS;
@@ -46,6 +50,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
@@ -67,6 +72,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServi
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.router.RouterClient;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -101,6 +107,15 @@ public class RouterDFSCluster {
   /** Mini cluster. */
   private MiniDFSCluster cluster;
 
+  protected static final long DEFAULT_HEARTBEAT_INTERVAL_MS =
+      TimeUnit.SECONDS.toMillis(5);
+  protected static final long DEFAULT_CACHE_INTERVAL_MS =
+      TimeUnit.SECONDS.toMillis(5);
+  /** Heartbeat interval in milliseconds. */
+  private long heartbeatInterval;
+  /** Cache flush interval in milliseconds. */
+  private long cacheFlushInterval;
+
   /** Router configuration overrides. */
   private Configuration routerOverrides;
   /** Namenode configuration overrides. */
@@ -118,6 +133,7 @@ public class RouterDFSCluster {
     private int rpcPort;
     private DFSClient client;
     private Configuration conf;
+    private RouterClient adminClient;
     private URI fileSystemUri;
 
     public RouterContext(Configuration conf, String nsId, String nnId)
@@ -183,6 +199,15 @@ public class RouterDFSCluster {
       });
     }
 
+    public RouterClient getAdminClient() throws IOException {
+      if (adminClient == null) {
+        InetSocketAddress routerSocket = router.getAdminServerAddress();
+        LOG.info("Connecting to router admin at {}", routerSocket);
+        adminClient = new RouterClient(routerSocket, conf);
+      }
+      return adminClient;
+    }
+
     public DFSClient getClient() throws IOException, URISyntaxException {
       if (client == null) {
         LOG.info("Connecting to router at {}", fileSystemUri);
@@ -304,13 +329,22 @@ public class RouterDFSCluster {
     }
   }
 
-  public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
+  public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes,
+      long heartbeatInterval, long cacheFlushInterval) {
     this.highAvailability = ha;
+    this.heartbeatInterval = heartbeatInterval;
+    this.cacheFlushInterval = cacheFlushInterval;
     configureNameservices(numNameservices, numNamenodes);
   }
 
   public RouterDFSCluster(boolean ha, int numNameservices) {
-    this(ha, numNameservices, 2);
+    this(ha, numNameservices, 2,
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
+  }
+
+  public RouterDFSCluster(boolean ha, int numNameservices, int numNamnodes) {
+    this(ha, numNameservices, numNamnodes,
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
   }
 
   /**
@@ -404,7 +438,12 @@ public class RouterDFSCluster {
     conf.set(DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0");
     conf.set(DFS_ROUTER_RPC_BIND_HOST_KEY, "0.0.0.0");
 
+    conf.set(DFS_ROUTER_ADMIN_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(DFS_ROUTER_ADMIN_BIND_HOST_KEY, "0.0.0.0");
+
     conf.set(DFS_ROUTER_DEFAULT_NAMESERVICE, nameservices.get(0));
+    conf.setLong(DFS_ROUTER_HEARTBEAT_INTERVAL_MS, heartbeatInterval);
+    conf.setLong(DFS_ROUTER_CACHE_TIME_TO_LIVE_MS, cacheFlushInterval);
 
     // Use mock resolver classes
     conf.setClass(FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/400b8e98/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java
new file mode 100644
index 0000000..e42ab50
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.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.federation;
+
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.createMockRegistrationForNamenode;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.synchronizeRecords;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+
+/**
+ * Test utility to mimic a federated HDFS cluster with a router and a state
+ * store.
+ */
+public class StateStoreDFSCluster extends RouterDFSCluster {
+
+  private static final Class<?> DEFAULT_FILE_RESOLVER =
+      MountTableResolver.class;
+  private static final Class<?> DEFAULT_NAMENODE_RESOLVER =
+      MembershipNamenodeResolver.class;
+
+  public StateStoreDFSCluster(boolean ha, int numNameservices, int numNamenodes,
+      long heartbeatInterval, long cacheFlushInterval)
+          throws IOException, InterruptedException {
+    this(ha, numNameservices, numNamenodes, heartbeatInterval,
+        cacheFlushInterval, DEFAULT_FILE_RESOLVER);
+  }
+
+  public StateStoreDFSCluster(boolean ha, int numNameservices, int numNamenodes,
+      long heartbeatInterval, long cacheFlushInterval, Class<?> fileResolver)
+          throws IOException, InterruptedException {
+    super(ha, numNameservices, numNamenodes, heartbeatInterval,
+        cacheFlushInterval);
+
+    // Attach state store and resolvers to router
+    Configuration stateStoreConfig = getStateStoreConfiguration();
+    // Use state store backed resolvers
+    stateStoreConfig.setClass(
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        DEFAULT_NAMENODE_RESOLVER, ActiveNamenodeResolver.class);
+    stateStoreConfig.setClass(
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        fileResolver, FileSubclusterResolver.class);
+    this.addRouterOverrides(stateStoreConfig);
+  }
+
+  public StateStoreDFSCluster(boolean ha, int numNameservices,
+      Class<?> fileResolver) throws IOException, InterruptedException {
+    this(ha, numNameservices, 2,
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS, fileResolver);
+  }
+
+  public StateStoreDFSCluster(boolean ha, int numNameservices)
+      throws IOException, InterruptedException {
+    this(ha, numNameservices, 2,
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
+  }
+
+  public StateStoreDFSCluster(boolean ha, int numNameservices,
+      int numNamnodes) throws IOException, InterruptedException {
+    this(ha, numNameservices, numNamnodes,
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // State Store Test Fixtures
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Adds test fixtures for NN registation for each NN nameservice -> NS
+   * namenode -> NN rpcAddress -> 0.0.0.0:0 webAddress -> 0.0.0.0:0 state ->
+   * STANDBY safeMode -> false blockPool -> test.
+   *
+   * @param stateStore State Store.
+   * @throws IOException If it cannot register.
+   */
+  public void createTestRegistration(StateStoreService stateStore)
+      throws IOException {
+    List<MembershipState> entries = new ArrayList<MembershipState>();
+    for (NamenodeContext nn : this.getNamenodes()) {
+      MembershipState entry = createMockRegistrationForNamenode(
+          nn.getNameserviceId(), nn.getNamenodeId(),
+          FederationNamenodeServiceState.STANDBY);
+      entries.add(entry);
+    }
+    synchronizeRecords(
+        stateStore, entries, MembershipState.class);
+  }
+
+  public void createTestMountTable(StateStoreService stateStore)
+      throws IOException {
+    List<MountTable> mounts = generateMockMountTable();
+    synchronizeRecords(stateStore, mounts, MountTable.class);
+    stateStore.refreshCaches();
+  }
+
+  public List<MountTable> generateMockMountTable() throws IOException {
+    // create table entries
+    List<MountTable> entries = new ArrayList<>();
+    for (String ns : this.getNameservices()) {
+      Map<String, String> destMap = new HashMap<>();
+      destMap.put(ns, getNamenodePathForNS(ns));
+
+      // Direct path
+      String fedPath = getFederatedPathForNS(ns);
+      MountTable entry = MountTable.newInstance(fedPath, destMap);
+      entries.add(entry);
+    }
+
+    // Root path goes to nameservice 1
+    Map<String, String> destMap = new HashMap<>();
+    String ns0 = this.getNameservices().get(0);
+    destMap.put(ns0, "/");
+    MountTable entry = MountTable.newInstance("/", destMap);
+    entries.add(entry);
+    return entries;
+  }
+}


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


[09/29] hadoop git commit: HDFS-10629. Federation Roter. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
new file mode 100644
index 0000000..ee6f57d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -0,0 +1,290 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodePriorityComparator;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.util.Time;
+
+/**
+ * In-memory cache/mock of a namenode and file resolver. Stores the most
+ * recently updated NN information for each nameservice and block pool. Also
+ * stores a virtual mount table for resolving global namespace paths to local NN
+ * paths.
+ */
+public class MockResolver
+    implements ActiveNamenodeResolver, FileSubclusterResolver {
+
+  private Map<String, List<? extends FederationNamenodeContext>> resolver =
+      new HashMap<String, List<? extends FederationNamenodeContext>>();
+  private Map<String, List<RemoteLocation>> locations =
+      new HashMap<String, List<RemoteLocation>>();
+  private Set<FederationNamespaceInfo> namespaces =
+      new HashSet<FederationNamespaceInfo>();
+  private String defaultNamespace = null;
+
+  public MockResolver(Configuration conf, StateStoreService store) {
+    this.cleanRegistrations();
+  }
+
+  public void addLocation(String mount, String nameservice, String location) {
+    RemoteLocation remoteLocation = new RemoteLocation(nameservice, location);
+    List<RemoteLocation> locationsList = locations.get(mount);
+    if (locationsList == null) {
+      locationsList = new LinkedList<RemoteLocation>();
+      locations.put(mount, locationsList);
+    }
+    if (!locationsList.contains(remoteLocation)) {
+      locationsList.add(remoteLocation);
+    }
+
+    if (this.defaultNamespace == null) {
+      this.defaultNamespace = nameservice;
+    }
+  }
+
+  public synchronized void cleanRegistrations() {
+    this.resolver =
+        new HashMap<String, List<? extends FederationNamenodeContext>>();
+    this.namespaces = new HashSet<FederationNamespaceInfo>();
+  }
+
+  @Override
+  public void updateActiveNamenode(
+      String ns, InetSocketAddress successfulAddress) {
+
+    String address = successfulAddress.getHostName() + ":" +
+        successfulAddress.getPort();
+    String key = ns;
+    if (key != null) {
+      // Update the active entry
+      @SuppressWarnings("unchecked")
+      List<FederationNamenodeContext> iterator =
+          (List<FederationNamenodeContext>) resolver.get(key);
+      for (FederationNamenodeContext namenode : iterator) {
+        if (namenode.getRpcAddress().equals(address)) {
+          MockNamenodeContext nn = (MockNamenodeContext) namenode;
+          nn.setState(FederationNamenodeServiceState.ACTIVE);
+          break;
+        }
+      }
+      Collections.sort(iterator, new NamenodePriorityComparator());
+    }
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext>
+      getNamenodesForNameserviceId(String nameserviceId) {
+    return resolver.get(nameserviceId);
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
+      String blockPoolId) {
+    return resolver.get(blockPoolId);
+  }
+
+  private static class MockNamenodeContext
+      implements FederationNamenodeContext {
+    private String webAddress;
+    private String rpcAddress;
+    private String serviceAddress;
+    private String lifelineAddress;
+    private String namenodeId;
+    private String nameserviceId;
+    private FederationNamenodeServiceState state;
+    private long dateModified;
+
+    MockNamenodeContext(
+        String rpc, String service, String lifeline, String web,
+        String ns, String nn, FederationNamenodeServiceState state) {
+      this.rpcAddress = rpc;
+      this.serviceAddress = service;
+      this.lifelineAddress = lifeline;
+      this.webAddress = web;
+      this.namenodeId = nn;
+      this.nameserviceId = ns;
+      this.state = state;
+      this.dateModified = Time.now();
+    }
+
+    public void setState(FederationNamenodeServiceState newState) {
+      this.state = newState;
+      this.dateModified = Time.now();
+    }
+
+    @Override
+    public String getRpcAddress() {
+      return rpcAddress;
+    }
+
+    @Override
+    public String getServiceAddress() {
+      return serviceAddress;
+    }
+
+    @Override
+    public String getLifelineAddress() {
+      return lifelineAddress;
+    }
+
+    @Override
+    public String getWebAddress() {
+      return webAddress;
+    }
+
+    @Override
+    public String getNamenodeKey() {
+      return nameserviceId + " " + namenodeId + " " + rpcAddress;
+    }
+
+    @Override
+    public String getNameserviceId() {
+      return nameserviceId;
+    }
+
+    @Override
+    public String getNamenodeId() {
+      return namenodeId;
+    }
+
+    @Override
+    public FederationNamenodeServiceState getState() {
+      return state;
+    }
+
+    @Override
+    public long getDateModified() {
+      return dateModified;
+    }
+  }
+
+  @Override
+  public synchronized boolean registerNamenode(NamenodeStatusReport report)
+      throws IOException {
+    MockNamenodeContext context = new MockNamenodeContext(
+        report.getRpcAddress(), report.getServiceAddress(),
+        report.getLifelineAddress(), report.getWebAddress(),
+        report.getNameserviceId(), report.getNamenodeId(), report.getState());
+
+    String nsId = report.getNameserviceId();
+    String bpId = report.getBlockPoolId();
+    String cId = report.getClusterId();
+    @SuppressWarnings("unchecked")
+    List<MockNamenodeContext> existingItems =
+        (List<MockNamenodeContext>) resolver.get(nsId);
+    if (existingItems == null) {
+      existingItems = new ArrayList<MockNamenodeContext>();
+      resolver.put(bpId, existingItems);
+      resolver.put(nsId, existingItems);
+    }
+    boolean added = false;
+    for (int i=0; i<existingItems.size() && !added; i++) {
+      MockNamenodeContext existing = existingItems.get(i);
+      if (existing.getNamenodeKey().equals(context.getNamenodeKey())) {
+        existingItems.set(i, context);
+        added = true;
+      }
+    }
+    if (!added) {
+      existingItems.add(context);
+    }
+    Collections.sort(existingItems, new NamenodePriorityComparator());
+
+    FederationNamespaceInfo info = new FederationNamespaceInfo(bpId, cId, nsId);
+    namespaces.add(info);
+    return true;
+  }
+
+  @Override
+  public Set<FederationNamespaceInfo> getNamespaces() throws IOException {
+    return this.namespaces;
+  }
+
+  @Override
+  public PathLocation getDestinationForPath(String path) throws IOException {
+    String finalPath = null;
+    String nameservice = null;
+    Set<String> namespaceSet = new HashSet<String>();
+    LinkedList<RemoteLocation> remoteLocations =
+        new LinkedList<RemoteLocation>();
+    for(String key : this.locations.keySet()) {
+      if(path.startsWith(key)) {
+        for (RemoteLocation location : this.locations.get(key)) {
+          finalPath = location.getDest() + path.substring(key.length());
+          nameservice = location.getNameserviceId();
+          RemoteLocation remoteLocation =
+              new RemoteLocation(nameservice, finalPath);
+          remoteLocations.add(remoteLocation);
+          namespaceSet.add(nameservice);
+        }
+        break;
+      }
+    }
+    if (remoteLocations.isEmpty()) {
+      // Path isn't supported, mimic resolver behavior.
+      return null;
+    }
+    return new PathLocation(path, remoteLocations, namespaceSet);
+  }
+
+  @Override
+  public List<String> getMountPoints(String path) throws IOException {
+    List<String> mounts = new ArrayList<String>();
+    if (path.equals("/")) {
+      // Mounts only supported under root level
+      for (String mount : this.locations.keySet()) {
+        if (mount.length() > 1) {
+          // Remove leading slash, this is the behavior of the mount tree,
+          // return only names.
+          mounts.add(mount.replace("/", ""));
+        }
+      }
+    }
+    return mounts;
+  }
+
+  @Override
+  public void setRouterId(String router) {
+  }
+
+  @Override
+  public String getDefaultNamespace() {
+    return defaultNamespace;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
new file mode 100644
index 0000000..16d624c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
@@ -0,0 +1,40 @@
+/**
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Constructs a router configuration with individual features enabled/disabled.
+ */
+public class RouterConfigBuilder {
+
+  private Configuration conf;
+
+  public RouterConfigBuilder(Configuration configuration) {
+    this.conf = configuration;
+  }
+
+  public RouterConfigBuilder() {
+    this.conf = new Configuration();
+  }
+
+  public Configuration build() {
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
new file mode 100644
index 0000000..55d04ad
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -0,0 +1,767 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.Service.STATE;
+
+/**
+ * Test utility to mimic a federated HDFS cluster with a router.
+ */
+public class RouterDFSCluster {
+  /**
+   * Router context.
+   */
+  public class RouterContext {
+    private Router router;
+    private FileContext fileContext;
+    private String nameserviceId;
+    private String namenodeId;
+    private int rpcPort;
+    private DFSClient client;
+    private Configuration conf;
+    private URI fileSystemUri;
+
+    public RouterContext(Configuration conf, String ns, String nn)
+        throws URISyntaxException {
+      this.namenodeId = nn;
+      this.nameserviceId = ns;
+      this.conf = conf;
+      router = new Router();
+      router.init(conf);
+    }
+
+    public Router getRouter() {
+      return this.router;
+    }
+
+    public String getNameserviceId() {
+      return this.nameserviceId;
+    }
+
+    public String getNamenodeId() {
+      return this.namenodeId;
+    }
+
+    public int getRpcPort() {
+      return this.rpcPort;
+    }
+
+    public FileContext getFileContext() {
+      return this.fileContext;
+    }
+
+    public void initRouter() throws URISyntaxException {
+    }
+
+    public DistributedFileSystem getFileSystem() throws IOException {
+      DistributedFileSystem fs =
+          (DistributedFileSystem) DistributedFileSystem.get(conf);
+      return fs;
+    }
+
+    public DFSClient getClient(UserGroupInformation user)
+        throws IOException, URISyntaxException, InterruptedException {
+
+      LOG.info("Connecting to router at " + fileSystemUri);
+      return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
+        @Override
+        public DFSClient run() throws IOException {
+          return new DFSClient(fileSystemUri, conf);
+        }
+      });
+    }
+
+    public DFSClient getClient() throws IOException, URISyntaxException {
+
+      if (client == null) {
+        LOG.info("Connecting to router at " + fileSystemUri);
+        client = new DFSClient(fileSystemUri, conf);
+      }
+      return client;
+    }
+  }
+
+  /**
+   * Namenode context.
+   */
+  public class NamenodeContext {
+    private NameNode namenode;
+    private String nameserviceId;
+    private String namenodeId;
+    private FileContext fileContext;
+    private int rpcPort;
+    private int servicePort;
+    private int lifelinePort;
+    private int httpPort;
+    private URI fileSystemUri;
+    private int index;
+    private Configuration conf;
+    private DFSClient client;
+
+    public NamenodeContext(Configuration conf, String ns, String nn,
+        int index) {
+      this.conf = conf;
+      this.namenodeId = nn;
+      this.nameserviceId = ns;
+      this.index = index;
+    }
+
+    public NameNode getNamenode() {
+      return this.namenode;
+    }
+
+    public String getNameserviceId() {
+      return this.nameserviceId;
+    }
+
+    public String getNamenodeId() {
+      return this.namenodeId;
+    }
+
+    public FileContext getFileContext() {
+      return this.fileContext;
+    }
+
+    public void setNamenode(NameNode n) throws URISyntaxException {
+      namenode = n;
+
+      // Store the bound ports and override the default FS with the local NN's
+      // RPC
+      rpcPort = n.getNameNodeAddress().getPort();
+      servicePort = n.getServiceRpcAddress().getPort();
+      lifelinePort = n.getServiceRpcAddress().getPort();
+      httpPort = n.getHttpAddress().getPort();
+      fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
+      DistributedFileSystem.setDefaultUri(conf, fileSystemUri);
+
+      try {
+        this.fileContext = FileContext.getFileContext(conf);
+      } catch (UnsupportedFileSystemException e) {
+        this.fileContext = null;
+      }
+    }
+
+    public String getRpcAddress() {
+      return namenode.getNameNodeAddress().getHostName() + ":" + rpcPort;
+    }
+
+    public String getServiceAddress() {
+      return namenode.getServiceRpcAddress().getHostName() + ":" + servicePort;
+    }
+
+    public String getLifelineAddress() {
+      return namenode.getServiceRpcAddress().getHostName() + ":" + lifelinePort;
+    }
+
+    public String getHttpAddress() {
+      return namenode.getHttpAddress().getHostName() + ":" + httpPort;
+    }
+
+    public DistributedFileSystem getFileSystem() throws IOException {
+      DistributedFileSystem fs =
+          (DistributedFileSystem) DistributedFileSystem.get(conf);
+      return fs;
+    }
+
+    public void resetClient() {
+      client = null;
+    }
+
+    public DFSClient getClient(UserGroupInformation user)
+        throws IOException, URISyntaxException, InterruptedException {
+
+      LOG.info("Connecting to namenode at " + fileSystemUri);
+      return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
+        @Override
+        public DFSClient run() throws IOException {
+          return new DFSClient(fileSystemUri, conf);
+        }
+      });
+    }
+
+    public DFSClient getClient() throws IOException, URISyntaxException {
+      if (client == null) {
+        LOG.info("Connecting to namenode at " + fileSystemUri);
+        client = new DFSClient(fileSystemUri, conf);
+      }
+      return client;
+    }
+
+    public String getConfSuffix() {
+      String suffix = nameserviceId;
+      if (highAvailability) {
+        suffix += "." + namenodeId;
+      }
+      return suffix;
+    }
+  }
+
+  public static final String NAMENODE1 = "nn0";
+  public static final String NAMENODE2 = "nn1";
+  public static final String NAMENODE3 = "nn2";
+  public static final String TEST_STRING = "teststring";
+  public static final String TEST_DIR = "testdir";
+  public static final String TEST_FILE = "testfile";
+
+  private List<String> nameservices;
+  private List<RouterContext> routers;
+  private List<NamenodeContext> namenodes;
+  private static final Log LOG = LogFactory.getLog(RouterDFSCluster.class);
+  private MiniDFSCluster cluster;
+  private boolean highAvailability;
+
+  protected static final int DEFAULT_HEARTBEAT_INTERVAL = 5;
+  protected static final int DEFAULT_CACHE_INTERVAL_SEC = 5;
+  private Configuration routerOverrides;
+  private Configuration namenodeOverrides;
+
+  private static final String NAMENODES = NAMENODE1 + "," + NAMENODE2;
+
+  public RouterDFSCluster(boolean ha, int numNameservices) {
+    this(ha, numNameservices, 2);
+  }
+
+  public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
+    this.highAvailability = ha;
+    configureNameservices(numNameservices, numNamenodes);
+  }
+
+  public void addRouterOverrides(Configuration conf) {
+    if (this.routerOverrides == null) {
+      this.routerOverrides = conf;
+    } else {
+      this.routerOverrides.addResource(conf);
+    }
+  }
+
+  public void addNamenodeOverrides(Configuration conf) {
+    if (this.namenodeOverrides == null) {
+      this.namenodeOverrides = conf;
+    } else {
+      this.namenodeOverrides.addResource(conf);
+    }
+  }
+
+  public Configuration generateNamenodeConfiguration(
+      String defaultNameserviceId) {
+    Configuration c = new HdfsConfiguration();
+
+    c.set(DFSConfigKeys.DFS_NAMESERVICES, getNameservicesKey());
+    c.set("fs.defaultFS", "hdfs://" + defaultNameserviceId);
+
+    for (String ns : nameservices) {
+      if (highAvailability) {
+        c.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns, NAMENODES);
+      }
+
+      for (NamenodeContext context : getNamenodes(ns)) {
+        String suffix = context.getConfSuffix();
+
+        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix,
+            "127.0.0.1:" + context.rpcPort);
+        c.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix,
+            "127.0.0.1:" + context.httpPort);
+        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
+            "0.0.0.0");
+      }
+    }
+
+    if (namenodeOverrides != null) {
+      c.addResource(namenodeOverrides);
+    }
+    return c;
+  }
+
+  public Configuration generateClientConfiguration() {
+    Configuration conf = new HdfsConfiguration();
+    conf.addResource(generateNamenodeConfiguration(getNameservices().get(0)));
+    return conf;
+  }
+
+  public Configuration generateRouterConfiguration(String localNameserviceId,
+      String localNamenodeId) throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.addResource(generateNamenodeConfiguration(localNameserviceId));
+
+    // Use mock resolver classes
+    conf.set(DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+    conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+
+    // Set the nameservice ID for the default NN monitor
+    conf.set(DFSConfigKeys.DFS_NAMESERVICE_ID, localNameserviceId);
+
+    if (localNamenodeId != null) {
+      conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, localNamenodeId);
+    }
+
+    StringBuilder routerBuilder = new StringBuilder();
+    for (String ns : nameservices) {
+      for (NamenodeContext context : getNamenodes(ns)) {
+        String suffix = context.getConfSuffix();
+
+        if (routerBuilder.length() != 0) {
+          routerBuilder.append(",");
+        }
+        routerBuilder.append(suffix);
+      }
+    }
+
+    return conf;
+  }
+
+  public void configureNameservices(int numNameservices, int numNamenodes) {
+    nameservices = new ArrayList<String>();
+    for (int i = 0; i < numNameservices; i++) {
+      nameservices.add("ns" + i);
+    }
+    namenodes = new ArrayList<NamenodeContext>();
+    int index = 0;
+    for (String ns : nameservices) {
+      Configuration nnConf = generateNamenodeConfiguration(ns);
+      if (highAvailability) {
+        NamenodeContext context =
+            new NamenodeContext(nnConf, ns, NAMENODE1, index);
+        namenodes.add(context);
+        index++;
+
+        if (numNamenodes > 1) {
+          context = new NamenodeContext(nnConf, ns, NAMENODE2, index + 1);
+          namenodes.add(context);
+          index++;
+        }
+
+        if (numNamenodes > 2) {
+          context = new NamenodeContext(nnConf, ns, NAMENODE3, index + 1);
+          namenodes.add(context);
+          index++;
+        }
+
+      } else {
+        NamenodeContext context = new NamenodeContext(nnConf, ns, null, index);
+        namenodes.add(context);
+        index++;
+      }
+    }
+  }
+
+  public String getNameservicesKey() {
+    StringBuilder ns = new StringBuilder();
+    for (int i = 0; i < nameservices.size(); i++) {
+      if (i > 0) {
+        ns.append(",");
+      }
+      ns.append(nameservices.get(i));
+    }
+    return ns.toString();
+  }
+
+  public String getRandomNameservice() {
+    Random r = new Random();
+    return nameservices.get(r.nextInt(nameservices.size()));
+  }
+
+  public List<String> getNameservices() {
+    return nameservices;
+  }
+
+  public List<NamenodeContext> getNamenodes(String nameservice) {
+    ArrayList<NamenodeContext> nns = new ArrayList<NamenodeContext>();
+    for (NamenodeContext c : namenodes) {
+      if (c.nameserviceId.equals(nameservice)) {
+        nns.add(c);
+      }
+    }
+    return nns;
+  }
+
+  public NamenodeContext getRandomNamenode() {
+    Random rand = new Random();
+    return namenodes.get(rand.nextInt(namenodes.size()));
+  }
+
+  public List<NamenodeContext> getNamenodes() {
+    return namenodes;
+  }
+
+  public boolean isHighAvailability() {
+    return highAvailability;
+  }
+
+  public NamenodeContext getNamenode(String nameservice,
+      String namenode) {
+    for (NamenodeContext c : namenodes) {
+      if (c.nameserviceId.equals(nameservice)) {
+        if (namenode == null || c.namenodeId == null || namenode.isEmpty()
+            || c.namenodeId.isEmpty()) {
+          return c;
+        } else if (c.namenodeId.equals(namenode)) {
+          return c;
+        }
+      }
+    }
+    return null;
+  }
+
+  public List<RouterContext> getRouters(String nameservice) {
+    ArrayList<RouterContext> nns = new ArrayList<RouterContext>();
+    for (RouterContext c : routers) {
+      if (c.nameserviceId.equals(nameservice)) {
+        nns.add(c);
+      }
+    }
+    return nns;
+  }
+
+  public RouterContext getRouterContext(String nameservice,
+      String namenode) {
+    for (RouterContext c : routers) {
+      if (namenode == null) {
+        return c;
+      }
+      if (c.namenodeId.equals(namenode)
+          && c.nameserviceId.equals(nameservice)) {
+        return c;
+      }
+    }
+    return null;
+  }
+
+  public RouterContext getRandomRouter() {
+    Random rand = new Random();
+    return routers.get(rand.nextInt(routers.size()));
+  }
+
+  public List<RouterContext> getRouters() {
+    return routers;
+  }
+
+  public RouterContext buildRouter(String nameservice, String namenode)
+      throws URISyntaxException, IOException {
+    Configuration config = generateRouterConfiguration(nameservice, namenode);
+    RouterContext rc = new RouterContext(config, nameservice, namenode);
+    return rc;
+  }
+
+  public void startCluster() {
+    startCluster(null);
+  }
+
+  public void startCluster(Configuration overrideConf) {
+    try {
+      MiniDFSNNTopology topology = new MiniDFSNNTopology();
+      for (String ns : nameservices) {
+
+        NSConf conf = new MiniDFSNNTopology.NSConf(ns);
+        if (highAvailability) {
+          for(int i = 0; i < namenodes.size()/nameservices.size(); i++) {
+            NNConf nnConf = new MiniDFSNNTopology.NNConf("nn" + i);
+            conf.addNN(nnConf);
+          }
+        } else {
+          NNConf nnConf = new MiniDFSNNTopology.NNConf(null);
+          conf.addNN(nnConf);
+        }
+        topology.addNameservice(conf);
+      }
+      topology.setFederation(true);
+
+      // Start mini DFS cluster
+      Configuration nnConf = generateNamenodeConfiguration(nameservices.get(0));
+      if (overrideConf != null) {
+        nnConf.addResource(overrideConf);
+      }
+      cluster = new MiniDFSCluster.Builder(nnConf).nnTopology(topology).build();
+      cluster.waitActive();
+
+      // Store NN pointers
+      for (int i = 0; i < namenodes.size(); i++) {
+        NameNode nn = cluster.getNameNode(i);
+        namenodes.get(i).setNamenode(nn);
+      }
+
+    } catch (Exception e) {
+      LOG.error("Cannot start Router DFS cluster: " + e.getMessage(), e);
+      cluster.shutdown();
+    }
+  }
+
+  public void startRouters()
+      throws InterruptedException, URISyntaxException, IOException {
+    // Create routers
+    routers = new ArrayList<RouterContext>();
+    for (String ns : nameservices) {
+
+      for (NamenodeContext context : getNamenodes(ns)) {
+        routers.add(buildRouter(ns, context.namenodeId));
+      }
+    }
+
+    // Start all routers
+    for (RouterContext router : routers) {
+      router.router.start();
+    }
+    // Wait until all routers are active and record their ports
+    for (RouterContext router : routers) {
+      waitActive(router);
+      router.initRouter();
+    }
+  }
+
+  public void waitActive(NamenodeContext nn) throws IOException {
+    cluster.waitActive(nn.index);
+  }
+
+  public void waitActive(RouterContext router)
+      throws InterruptedException {
+    for (int loopCount = 0; loopCount < 20; loopCount++) {
+      // Validate connection of routers to NNs
+      if (router.router.getServiceState() == STATE.STARTED) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    assertFalse(
+        "Timeout waiting for " + router.router.toString() + " to activate.",
+        true);
+  }
+
+
+  public void registerNamenodes() throws IOException {
+    for (RouterContext r : routers) {
+      ActiveNamenodeResolver resolver = r.router.getNamenodeResolver();
+      for (NamenodeContext nn : namenodes) {
+        // Generate a report
+        NamenodeStatusReport report = new NamenodeStatusReport(nn.nameserviceId,
+            nn.namenodeId, nn.getRpcAddress(), nn.getServiceAddress(),
+            nn.getLifelineAddress(), nn.getHttpAddress());
+        report.setNamespaceInfo(nn.namenode.getNamesystem().getFSImage()
+            .getStorage().getNamespaceInfo());
+
+        // Determine HA state from nn public state string
+        String nnState = nn.namenode.getState();
+        HAServiceState haState = HAServiceState.ACTIVE;
+        for (HAServiceState state : HAServiceState.values()) {
+          if (nnState.equalsIgnoreCase(state.name())) {
+            haState = state;
+            break;
+          }
+        }
+        report.setHAServiceState(haState);
+
+        // Register with the resolver
+        resolver.registerNamenode(report);
+      }
+    }
+  }
+
+  public void waitNamenodeRegistration()
+      throws InterruptedException, IllegalStateException, IOException {
+    for (RouterContext r : routers) {
+      for (NamenodeContext nn : namenodes) {
+        FederationTestUtils.waitNamenodeRegistered(
+            r.router.getNamenodeResolver(), nn.nameserviceId, nn.namenodeId,
+            null);
+      }
+    }
+  }
+
+  public void waitRouterRegistrationQuorum(RouterContext router,
+      FederationNamenodeServiceState state, String nameservice, String namenode)
+          throws InterruptedException, IOException {
+    LOG.info("Waiting for NN - " + nameservice + ":" + namenode
+        + " to transition to state - " + state);
+    FederationTestUtils.waitNamenodeRegistered(
+        router.router.getNamenodeResolver(), nameservice, namenode, state);
+  }
+
+  public String getFederatedPathForNameservice(String ns) {
+    return "/" + ns;
+  }
+
+  public String getNamenodePathForNameservice(String ns) {
+    return "/target-" + ns;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/ns0/testdir which maps to ns0->/target-ns0/testdir
+   *         </ul>
+   */
+  public String getFederatedTestDirectoryForNameservice(String ns) {
+    return getFederatedPathForNameservice(ns) + "/" + TEST_DIR;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/target-ns0/testdir
+   *         </ul>
+   */
+  public String getNamenodeTestDirectoryForNameservice(String ns) {
+    return getNamenodePathForNameservice(ns) + "/" + TEST_DIR;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/ns0/testfile which maps to ns0->/target-ns0/testfile
+   *         </ul>
+   */
+  public String getFederatedTestFileForNameservice(String ns) {
+    return getFederatedPathForNameservice(ns) + "/" + TEST_FILE;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/target-ns0/testfile
+   *         </ul>
+   */
+  public String getNamenodeTestFileForNameservice(String ns) {
+    return getNamenodePathForNameservice(ns) + "/" + TEST_FILE;
+  }
+
+  public void shutdown() {
+    cluster.shutdown();
+    if (routers != null) {
+      for (RouterContext context : routers) {
+        stopRouter(context);
+      }
+    }
+  }
+
+  public void stopRouter(RouterContext router) {
+    try {
+
+      router.router.shutDown();
+
+      int loopCount = 0;
+      while (router.router.getServiceState() != STATE.STOPPED) {
+        loopCount++;
+        Thread.sleep(1000);
+        if (loopCount > 20) {
+          LOG.error("Unable to shutdown router - " + router.rpcPort);
+          break;
+        }
+      }
+    } catch (InterruptedException e) {
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // Namespace Test Fixtures
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Creates test directories via the namenode.
+   * 1) /target-ns0/testfile
+   * 2) /target-ns1/testfile
+   * @throws IOException
+   */
+  public void createTestDirectoriesNamenode() throws IOException {
+    // Add a test dir to each NS and verify
+    for (String ns : getNameservices()) {
+      NamenodeContext context = getNamenode(ns, null);
+      if (!createTestDirectoriesNamenode(context)) {
+        throw new IOException("Unable to create test directory for ns - " + ns);
+      }
+    }
+  }
+
+  public boolean createTestDirectoriesNamenode(NamenodeContext nn)
+      throws IOException {
+    return FederationTestUtils.addDirectory(nn.getFileSystem(),
+        getNamenodeTestDirectoryForNameservice(nn.nameserviceId));
+  }
+
+  public void deleteAllFiles() throws IOException {
+    // Delete all files via the NNs and verify
+    for (NamenodeContext context : getNamenodes()) {
+      FileStatus[] status = context.getFileSystem().listStatus(new Path("/"));
+      for(int i = 0; i <status.length; i++) {
+        Path p = status[i].getPath();
+        context.getFileSystem().delete(p, true);
+      }
+      status = context.getFileSystem().listStatus(new Path("/"));
+      assertEquals(status.length, 0);
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // MockRouterResolver Test Fixtures
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * <ul>
+   * <li>/ -> [ns0->/].
+   * <li>/nso -> ns0->/target-ns0.
+   * <li>/ns1 -> ns1->/target-ns1.
+   * </ul>
+   */
+  public void installMockLocations() {
+    for (RouterContext r : routers) {
+      MockResolver resolver =
+          (MockResolver) r.router.getSubclusterResolver();
+      // create table entries
+      for (String ns : nameservices) {
+        // Direct path
+        resolver.addLocation(getFederatedPathForNameservice(ns), ns,
+            getNamenodePathForNameservice(ns));
+      }
+
+      // Root path goes to both NS1
+      resolver.addLocation("/", nameservices.get(0), "/");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7ffbf3f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
new file mode 100644
index 0000000..8c720c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
@@ -0,0 +1,96 @@
+/**
+ * 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.IOException;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.MockResolver;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.service.Service.STATE;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The the safe mode for the {@link Router} controlled by
+ * {@link SafeModeTimer}.
+ */
+public class TestRouter {
+
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void create() throws IOException {
+    // Basic configuration without the state store
+    conf = new Configuration();
+    // Mock resolver classes
+    conf.set(DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+    conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+
+    // Simulate a co-located NN
+    conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns0");
+    conf.set("fs.defaultFS", "hdfs://" + "ns0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + "ns0",
+            "127.0.0.1:0" + 0);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + "ns0",
+            "127.0.0.1:" + 0);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + "ns0",
+            "0.0.0.0");
+  }
+
+  @AfterClass
+  public static void destroy() {
+  }
+
+  @Before
+  public void setup() throws IOException, URISyntaxException {
+  }
+
+  @After
+  public void cleanup() {
+  }
+
+  private static void testRouterStartup(Configuration routerConfig)
+      throws InterruptedException, IOException {
+    Router router = new Router();
+    assertEquals(STATE.NOTINITED, router.getServiceState());
+    router.init(routerConfig);
+    assertEquals(STATE.INITED, router.getServiceState());
+    router.start();
+    assertEquals(STATE.STARTED, router.getServiceState());
+    router.stop();
+    assertEquals(STATE.STOPPED, router.getServiceState());
+    router.close();
+  }
+
+  @Test
+  public void testRouterService() throws InterruptedException, IOException {
+
+    // Run with all services
+    testRouterStartup((new RouterConfigBuilder(conf)).build());
+  }
+}


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


[04/29] hadoop git commit: HDFS-12392. Writing striped file failed due to different cell size. Contributed by Sammi Chen

Posted by in...@apache.org.
HDFS-12392. Writing striped file failed due to different cell size. Contributed by Sammi Chen


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

Branch: refs/heads/HDFS-10467
Commit: d7f27043ce034d69003b054dfe4f9e8e317ec543
Parents: d4035d4
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 6 13:29:52 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Sep 6 13:29:52 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  2 +
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 50 +++++++++++++++++++-
 2 files changed, 50 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7f27043/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 408b325..7f05338 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -215,6 +215,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       buffers = new ByteBuffer[numAllBlocks];
       for (int i = 0; i < buffers.length; i++) {
         buffers[i] = BUFFER_POOL.getBuffer(useDirectBuffer(), cellSize);
+        buffers[i].limit(cellSize);
       }
     }
 
@@ -237,6 +238,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
     private void clear() {
       for (int i = 0; i< numAllBlocks; i++) {
         buffers[i].clear();
+        buffers[i].limit(cellSize);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7f27043/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 0c545be..e095602 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -60,7 +61,7 @@ public class TestErasureCodingPolicies {
   private Configuration conf;
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
+  private static final int BLOCK_SIZE = 16 * 1024;
   private ErasureCodingPolicy ecPolicy;
   private FSNamesystem namesystem;
 
@@ -78,7 +79,8 @@ public class TestErasureCodingPolicies {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     DFSTestUtil.enableAllECPolicies(conf);
     cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(1).build();
+        numDataNodes(ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()).
+        build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     namesystem = cluster.getNamesystem();
@@ -836,4 +838,48 @@ public class TestErasureCodingPolicies {
         ecPolicy, fs.getErasureCodingPolicy(subReplicaFile));
     fs.delete(subReplicaFile, false);
   }
+
+  @Test
+  public void testDifferentErasureCodingPolicyCellSize() throws Exception {
+    // add policy with cell size 8K
+    ErasureCodingPolicy newPolicy1 =
+        new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
+    ErasureCodingPolicy[] policyArray =
+        new ErasureCodingPolicy[] {newPolicy1};
+    AddECPolicyResponse[] responses = fs.addErasureCodingPolicies(policyArray);
+    assertEquals(1, responses.length);
+    assertTrue(responses[0].isSucceed());
+    newPolicy1 = responses[0].getPolicy();
+
+    // add policy with cell size 4K
+    ErasureCodingPolicy newPolicy2 =
+        new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 4 * 1024);
+    policyArray = new ErasureCodingPolicy[] {newPolicy2};
+    responses = fs.addErasureCodingPolicies(policyArray);
+    assertEquals(1, responses.length);
+    assertTrue(responses[0].isSucceed());
+    newPolicy2 = responses[0].getPolicy();
+
+    // enable policies
+    fs.enableErasureCodingPolicy(newPolicy1.getName());
+    fs.enableErasureCodingPolicy(newPolicy2.getName());
+
+    final Path stripedDir1 = new Path("/striped1");
+    final Path stripedDir2 = new Path("/striped2");
+    final Path file1 = new Path(stripedDir1, "file");
+    final Path file2 = new Path(stripedDir2, "file");
+
+    fs.mkdirs(stripedDir1);
+    fs.setErasureCodingPolicy(stripedDir1, newPolicy1.getName());
+    fs.mkdirs(stripedDir2);
+    fs.setErasureCodingPolicy(stripedDir2, newPolicy2.getName());
+
+    final int fileLength = BLOCK_SIZE * newPolicy1.getNumDataUnits();
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(fs, file1, bytes);
+    DFSTestUtil.writeFile(fs, file2, bytes);
+
+    fs.delete(stripedDir1, true);
+    fs.delete(stripedDir2, true);
+  }
 }


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


[24/29] hadoop git commit: HDFS-10630. Federation State Store FS Implementation. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10630. Federation State Store FS Implementation. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 2096bce77263bc37f84ee05e059d66d9194f7181
Parents: 98efaf6
Author: Inigo Goiri <in...@apache.org>
Authored: Tue May 2 15:49:53 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed Sep 6 09:49:44 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  14 +
 .../federation/router/PeriodicService.java      | 198 ++++++++
 .../StateStoreConnectionMonitorService.java     |  67 +++
 .../federation/store/StateStoreService.java     | 152 +++++-
 .../federation/store/StateStoreUtils.java       |  51 +-
 .../store/driver/StateStoreDriver.java          |  31 +-
 .../driver/StateStoreRecordOperations.java      |  17 +-
 .../store/driver/impl/StateStoreBaseImpl.java   |  31 +-
 .../driver/impl/StateStoreFileBaseImpl.java     | 429 ++++++++++++++++
 .../store/driver/impl/StateStoreFileImpl.java   | 161 +++++++
 .../driver/impl/StateStoreFileSystemImpl.java   | 178 +++++++
 .../driver/impl/StateStoreSerializableImpl.java |  77 +++
 .../federation/store/records/BaseRecord.java    |  20 +-
 .../server/federation/store/records/Query.java  |  66 +++
 .../src/main/resources/hdfs-default.xml         |  16 +
 .../store/FederationStateStoreTestUtils.java    | 232 +++++++++
 .../store/driver/TestStateStoreDriverBase.java  | 483 +++++++++++++++++++
 .../store/driver/TestStateStoreFile.java        |  64 +++
 .../store/driver/TestStateStoreFileSystem.java  |  88 ++++
 19 files changed, 2329 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/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 7623839..8cdd450 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
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -25,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
 import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl;
 import org.apache.hadoop.http.HttpConfig;
 
@@ -1134,6 +1138,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT =
           StateStoreSerializerPBImpl.class;
 
+  public static final String FEDERATION_STORE_DRIVER_CLASS =
+      FEDERATION_STORE_PREFIX + "driver.class";
+  public static final Class<? extends StateStoreDriver>
+      FEDERATION_STORE_DRIVER_CLASS_DEFAULT = StateStoreFileImpl.class;
+
+  public static final String FEDERATION_STORE_CONNECTION_TEST_MS =
+      FEDERATION_STORE_PREFIX + "connection.test";
+  public static final long FEDERATION_STORE_CONNECTION_TEST_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java
new file mode 100644
index 0000000..5e12222
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java
@@ -0,0 +1,198 @@
+/**
+ * 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 java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Service to periodically execute a runnable.
+ */
+public abstract class PeriodicService extends AbstractService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicService.class);
+
+  /** Default interval in milliseconds for the periodic service. */
+  private static final long DEFAULT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(1);
+
+
+  /** Interval for running the periodic service in milliseconds. */
+  private long intervalMs;
+  /** Name of the service. */
+  private final String serviceName;
+
+  /** Scheduler for the periodic service. */
+  private final ScheduledExecutorService scheduler;
+
+  /** If the service is running. */
+  private volatile boolean isRunning = false;
+
+  /** How many times we run. */
+  private long runCount;
+  /** How many errors we got. */
+  private long errorCount;
+  /** When was the last time we executed this service successfully. */
+  private long lastRun;
+
+  /**
+   * Create a new periodic update service.
+   *
+   * @param name Name of the service.
+   */
+  public PeriodicService(String name) {
+    this(name, DEFAULT_INTERVAL_MS);
+  }
+
+  /**
+   * Create a new periodic update service.
+   *
+   * @param name Name of the service.
+   * @param interval Interval for the periodic service in milliseconds.
+   */
+  public PeriodicService(String name, long interval) {
+    super(name);
+    this.serviceName = name;
+    this.intervalMs = interval;
+
+    ThreadFactory threadFactory = new ThreadFactoryBuilder()
+        .setNameFormat(this.getName() + "-%d")
+        .build();
+    this.scheduler = Executors.newScheduledThreadPool(1, threadFactory);
+  }
+
+  /**
+   * Set the interval for the periodic service.
+   *
+   * @param interval Interval in milliseconds.
+   */
+  protected void setIntervalMs(long interval) {
+    if (getServiceState() == STATE.STARTED) {
+      throw new ServiceStateException("Periodic service already started");
+    } else {
+      this.intervalMs = interval;
+    }
+  }
+
+  /**
+   * Get the interval for the periodic service.
+   *
+   * @return Interval in milliseconds.
+   */
+  protected long getIntervalMs() {
+    return this.intervalMs;
+  }
+
+  /**
+   * Get how many times we failed to run the periodic service.
+   *
+   * @return Times we failed to run the periodic service.
+   */
+  protected long getErrorCount() {
+    return this.errorCount;
+  }
+
+  /**
+   * Get how many times we run the periodic service.
+   *
+   * @return Times we run the periodic service.
+   */
+  protected long getRunCount() {
+    return this.runCount;
+  }
+
+  /**
+   * Get the last time the periodic service was executed.
+   *
+   * @return Last time the periodic service was executed.
+   */
+  protected long getLastUpdate() {
+    return this.lastRun;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    LOG.info("Starting periodic service {}", this.serviceName);
+    startPeriodic();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    stopPeriodic();
+    LOG.info("Stopping periodic service {}", this.serviceName);
+    super.serviceStop();
+  }
+
+  /**
+   * Stop the periodic task.
+   */
+  protected synchronized void stopPeriodic() {
+    if (this.isRunning) {
+      LOG.info("{} is shutting down", this.serviceName);
+      this.isRunning = false;
+      this.scheduler.shutdownNow();
+    }
+  }
+
+  /**
+   * Start the periodic execution.
+   */
+  protected synchronized void startPeriodic() {
+    stopPeriodic();
+
+    // Create the runnable service
+    Runnable updateRunnable = new Runnable() {
+      @Override
+      public void run() {
+        LOG.debug("Running {} update task", serviceName);
+        try {
+          if (!isRunning) {
+            return;
+          }
+          periodicInvoke();
+          runCount++;
+          lastRun = Time.now();
+        } catch (Exception ex) {
+          errorCount++;
+          LOG.warn(serviceName + " service threw an exception", ex);
+        }
+      }
+    };
+
+    // Start the execution of the periodic service
+    this.isRunning = true;
+    this.scheduler.scheduleWithFixedDelay(
+        updateRunnable, 0, this.intervalMs, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Method that the service will run periodically.
+   */
+  protected abstract void periodicInvoke();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java
new file mode 100644
index 0000000..4d279c5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java
@@ -0,0 +1,67 @@
+/**
+ * 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.store;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.router.PeriodicService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service to periodically monitor the connection of the StateStore
+ * {@link StateStoreService} data store and to re-open the connection
+ * to the data store if required.
+ */
+public class StateStoreConnectionMonitorService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreConnectionMonitorService.class);
+
+  /** Service that maintains the State Store connection. */
+  private final StateStoreService stateStore;
+
+
+  /**
+   * Create a new service to monitor the connectivity of the state store driver.
+   *
+   * @param store Instance of the state store to be monitored.
+   */
+  public StateStoreConnectionMonitorService(StateStoreService store) {
+    super(StateStoreConnectionMonitorService.class.getSimpleName());
+    this.stateStore = store;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    this.setIntervalMs(conf.getLong(
+        DFSConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS,
+        DFSConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS_DEFAULT));
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    LOG.debug("Checking state store connection");
+    if (!stateStore.isDriverReady()) {
+      LOG.info("Attempting to open state store driver.");
+      stateStore.loadDriver();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
index 866daa3..df207e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
@@ -15,45 +15,168 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hdfs.server.federation.store;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * A service to initialize a
  * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
- * StateStoreDriver} and maintain the connection to the data store. There
- * are multiple state store driver connections supported:
+ * StateStoreDriver} and maintain the connection to the data store. There are
+ * multiple state store driver connections supported:
  * <ul>
- * <li>File {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
+ * <li>File
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
  * StateStoreFileImpl StateStoreFileImpl}
- * <li>ZooKeeper {@link org.apache.hadoop.hdfs.server.federation.store.driver.
- * impl.StateStoreZooKeeperImpl StateStoreZooKeeperImpl}
+ * <li>ZooKeeper
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
+ * StateStoreZooKeeperImpl StateStoreZooKeeperImpl}
  * </ul>
  * <p>
- * The service also supports the dynamic registration of data interfaces such as
- * the following:
+ * The service also supports the dynamic registration of record stores like:
  * <ul>
- * <li>{@link MembershipStateStore}: state of the Namenodes in the
+ * <li>{@link MembershipStore}: state of the Namenodes in the
  * federation.
  * <li>{@link MountTableStore}: Mount table between to subclusters.
  * See {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
- * <li>{@link RouterStateStore}: State of the routers in the federation.
  * </ul>
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class StateStoreService extends CompositeService {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreService.class);
+
+
+  /** State Store configuration. */
+  private Configuration conf;
+
   /** Identifier for the service. */
   private String identifier;
 
-  // Stub class
-  public StateStoreService(String name) {
-    super(name);
+  /** Driver for the back end connection. */
+  private StateStoreDriver driver;
+
+  /** Service to maintain data store connection. */
+  private StateStoreConnectionMonitorService monitorService;
+
+
+  public StateStoreService() {
+    super(StateStoreService.class.getName());
+  }
+
+  /**
+   * Initialize the State Store and the connection to the backend.
+   *
+   * @param config Configuration for the State Store.
+   * @throws IOException
+   */
+  @Override
+  protected void serviceInit(Configuration config) throws Exception {
+    this.conf = config;
+
+    // Create implementation of State Store
+    Class<? extends StateStoreDriver> driverClass = this.conf.getClass(
+        DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS,
+        DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS_DEFAULT,
+        StateStoreDriver.class);
+    this.driver = ReflectionUtils.newInstance(driverClass, this.conf);
+
+    if (this.driver == null) {
+      throw new IOException("Cannot create driver for the State Store");
+    }
+
+    // Check the connection to the State Store periodically
+    this.monitorService = new StateStoreConnectionMonitorService(this);
+    this.addService(monitorService);
+
+    super.serviceInit(this.conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    loadDriver();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    closeDriver();
+
+    super.serviceStop();
+  }
+
+  /**
+   * List of records supported by this State Store.
+   *
+   * @return List of supported record classes.
+   */
+  public Collection<Class<? extends BaseRecord>> getSupportedRecords() {
+    // TODO add list of records
+    return new LinkedList<>();
+  }
+
+  /**
+   * Load the State Store driver. If successful, refresh cached data tables.
+   */
+  public void loadDriver() {
+    synchronized (this.driver) {
+      if (!isDriverReady()) {
+        String driverName = this.driver.getClass().getSimpleName();
+        if (this.driver.init(conf, getIdentifier(), getSupportedRecords())) {
+          LOG.info("Connection to the State Store driver {} is open and ready",
+              driverName);
+        } else {
+          LOG.error("Cannot initialize State Store driver {}", driverName);
+        }
+      }
+    }
+  }
+
+  /**
+   * Check if the driver is ready to be used.
+   *
+   * @return If the driver is ready.
+   */
+  public boolean isDriverReady() {
+    return this.driver.isDriverReady();
+  }
+
+  /**
+   * Manually shuts down the driver.
+   *
+   * @throws Exception If the driver cannot be closed.
+   */
+  @VisibleForTesting
+  public void closeDriver() throws Exception {
+    if (this.driver != null) {
+      this.driver.close();
+    }
+  }
+
+  /**
+   * Get the state store driver.
+   *
+   * @return State store driver.
+   */
+  public StateStoreDriver getDriver() {
+    return this.driver;
   }
 
   /**
@@ -74,4 +197,5 @@ public class StateStoreService extends CompositeService {
   public void setIdentifier(String id) {
     this.identifier = id;
   }
-}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
index 8c681df..0a36619 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
@@ -17,17 +17,22 @@
  */
 package org.apache.hadoop.hdfs.server.federation.store;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Set of utility functions used to query, create, update and delete data
- * records in the state store.
+ * Set of utility functions used to work with the State Store.
  */
 public final class StateStoreUtils {
 
-  private static final Log LOG = LogFactory.getLog(StateStoreUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreUtils.class);
+
 
   private StateStoreUtils() {
     // Utility class
@@ -52,7 +57,7 @@ public final class StateStoreUtils {
 
     // Check if we went too far
     if (actualClazz.equals(BaseRecord.class)) {
-      LOG.error("We went too far (" + actualClazz + ") with " + clazz);
+      LOG.error("We went too far ({}) with {}", actualClazz, clazz);
       actualClazz = clazz;
     }
     return actualClazz;
@@ -69,4 +74,36 @@ public final class StateStoreUtils {
       Class<? extends BaseRecord> getRecordClass(final T record) {
     return getRecordClass(record.getClass());
   }
-}
+
+  /**
+   * Get the base class name for a record. If we get an implementation of a
+   * record we will return the real parent record class.
+   *
+   * @param clazz Class of the data record to check.
+   * @return Name of the base class for the record.
+   */
+  public static <T extends BaseRecord> String getRecordName(
+      final Class<T> clazz) {
+    return getRecordClass(clazz).getSimpleName();
+  }
+
+  /**
+   * Filters a list of records to find all records matching the query.
+   *
+   * @param query Map of field names and objects to use to filter results.
+   * @param records List of data records to filter.
+   * @return List of all records matching the query (or empty list if none
+   *         match), null if the data set could not be filtered.
+   */
+  public static <T extends BaseRecord> List<T> filterMultiple(
+      final Query<T> query, final Iterable<T> records) {
+
+    List<T> matchingList = new ArrayList<>();
+    for (T record : records) {
+      if (query.matches(record)) {
+        matchingList.add(record);
+      }
+    }
+    return matchingList;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
index a1527df..90111bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
@@ -18,15 +18,16 @@
 package org.apache.hadoop.hdfs.server.federation.store.driver;
 
 import java.net.InetAddress;
-import java.util.List;
+import java.util.Collection;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Driver class for an implementation of a {@link StateStoreService}
@@ -35,7 +36,8 @@ import org.apache.hadoop.util.Time;
  */
 public abstract class StateStoreDriver implements StateStoreRecordOperations {
 
-  private static final Log LOG = LogFactory.getLog(StateStoreDriver.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreDriver.class);
 
 
   /** State Store configuration. */
@@ -47,13 +49,14 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
 
   /**
    * Initialize the state store connection.
+   *
    * @param config Configuration for the driver.
    * @param id Identifier for the driver.
    * @param records Records that are supported.
    * @return If initialized and ready, false if failed to initialize driver.
    */
   public boolean init(final Configuration config, final String id,
-      final List<Class<? extends BaseRecord>> records) {
+      final Collection<Class<? extends BaseRecord>> records) {
 
     this.conf = config;
     this.identifier = id;
@@ -62,8 +65,20 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
       LOG.warn("The identifier for the State Store connection is not set");
     }
 
-    // TODO stub
-    return false;
+    boolean success = initDriver();
+    if (!success) {
+      LOG.error("Cannot intialize driver for {}", getDriverName());
+      return false;
+    }
+
+    for (Class<? extends BaseRecord> cls : records) {
+      String recordString = StateStoreUtils.getRecordName(cls);
+      if (!initRecordStorage(recordString, cls)) {
+        LOG.error("Cannot initialize record store for {}", cls.getSimpleName());
+        return false;
+      }
+    }
+    return true;
   }
 
   /**
@@ -169,4 +184,4 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
     }
     return hostname;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
index 739eeba..e76a733 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hdfs.server.federation.store.driver;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
 import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
@@ -67,14 +67,14 @@ public interface StateStoreRecordOperations {
    * Get a single record from the store that matches the query.
    *
    * @param clazz Class of record to fetch.
-   * @param query Map of field names and objects to filter results.
+   * @param query Query to filter results.
    * @return A single record matching the query. Null if there are no matching
    *         records or more than one matching record in the store.
    * @throws IOException If multiple records match or if the data store cannot
    *           be queried.
    */
   @Idempotent
-  <T extends BaseRecord> T get(Class<T> clazz, Map<String, String> query)
+  <T extends BaseRecord> T get(Class<T> clazz, Query<T> query)
       throws IOException;
 
   /**
@@ -83,14 +83,14 @@ public interface StateStoreRecordOperations {
    * supports filtering it should overwrite this method.
    *
    * @param clazz Class of record to fetch.
-   * @param query Map of field names and objects to filter results.
+   * @param query Query to filter results.
    * @return Records of type clazz that match the query or empty list if none
    *         are found.
    * @throws IOException Throws exception if unable to query the data store.
    */
   @Idempotent
   <T extends BaseRecord> List<T> getMultiple(
-      Class<T> clazz, Map<String, String> query) throws IOException;
+      Class<T> clazz, Query<T> query) throws IOException;
 
   /**
    * Creates a single record. Optionally updates an existing record with same
@@ -152,13 +152,12 @@ public interface StateStoreRecordOperations {
    * Remove multiple records of a specific class that match a query. Requires
    * the getAll implementation to fetch fresh records on each call.
    *
-   * @param clazz Class of record to remove.
-   * @param filter matching filter to remove.
+   * @param query Query to filter what to remove.
    * @return The number of records removed.
    * @throws IOException Throws exception if unable to query the data store.
    */
   @AtMostOnce
-  <T extends BaseRecord> int remove(Class<T> clazz, Map<String, String> filter)
+  <T extends BaseRecord> int remove(Class<T> clazz, Query<T> query)
       throws IOException;
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
index b711fa9..1bd35f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
@@ -17,14 +17,17 @@
  */
 package org.apache.hadoop.hdfs.server.federation.store.driver.impl;
 
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.filterMultiple;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
 
 /**
  * Base implementation of a State Store driver. It contains default
@@ -41,7 +44,7 @@ public abstract class StateStoreBaseImpl extends StateStoreDriver {
 
   @Override
   public <T extends BaseRecord> T get(
-      Class<T> clazz, Map<String, String> query) throws IOException {
+      Class<T> clazz, Query<T> query) throws IOException {
     List<T> records = getMultiple(clazz, query);
     if (records.size() > 1) {
       throw new IOException("Found more than one object in collection");
@@ -53,17 +56,31 @@ public abstract class StateStoreBaseImpl extends StateStoreDriver {
   }
 
   @Override
+  public <T extends BaseRecord> List<T> getMultiple(
+      Class<T> clazz, Query<T> query) throws IOException  {
+    QueryResult<T> result = get(clazz);
+    List<T> records = result.getRecords();
+    List<T> ret = filterMultiple(query, records);
+    if (ret == null) {
+      throw new IOException("Cannot fetch records from the store");
+    }
+    return ret;
+  }
+
+  @Override
   public <T extends BaseRecord> boolean put(
       T record, boolean allowUpdate, boolean errorIfExists) throws IOException {
-    List<T> singletonList = new ArrayList<T>();
+    List<T> singletonList = new ArrayList<>();
     singletonList.add(record);
     return putAll(singletonList, allowUpdate, errorIfExists);
   }
 
   @Override
   public <T extends BaseRecord> boolean remove(T record) throws IOException {
-    Map<String, String> primaryKeys = record.getPrimaryKeys();
-    Class<? extends BaseRecord> clazz = StateStoreUtils.getRecordClass(record);
-    return remove(clazz, primaryKeys) == 1;
+    final Query<T> query = new Query<T>(record);
+    Class<? extends BaseRecord> clazz = record.getClass();
+    @SuppressWarnings("unchecked")
+    Class<T> recordClass = (Class<T>)StateStoreUtils.getRecordClass(clazz);
+    return remove(recordClass, query) == 1;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
new file mode 100644
index 0000000..d7c00ff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
@@ -0,0 +1,429 @@
+/**
+ * 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.store.driver.impl;
+
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.filterMultiple;
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.getRecordClass;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link StateStoreDriver} implementation based on a local file.
+ */
+public abstract class StateStoreFileBaseImpl
+    extends StateStoreSerializableImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreFileBaseImpl.class);
+
+  /** If it is initialized. */
+  private boolean initialized = false;
+
+  /** Name of the file containing the data. */
+  private static final String DATA_FILE_NAME = "records.data";
+
+
+  /**
+   * Lock reading records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void lockRecordRead(Class<T> clazz);
+
+  /**
+   * Unlock reading records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void unlockRecordRead(
+      Class<T> clazz);
+
+  /**
+   * Lock writing records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void lockRecordWrite(
+      Class<T> clazz);
+
+  /**
+   * Unlock writing records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void unlockRecordWrite(
+      Class<T> clazz);
+
+  /**
+   * Get the reader for the file system.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> BufferedReader getReader(
+      Class<T> clazz, String sub);
+
+  /**
+   * Get the writer for the file system.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> BufferedWriter getWriter(
+      Class<T> clazz, String sub);
+
+  /**
+   * Check if a path exists.
+   *
+   * @param path Path to check.
+   * @return If the path exists.
+   */
+  protected abstract boolean exists(String path);
+
+  /**
+   * Make a directory.
+   *
+   * @param path Path of the directory to create.
+   * @return If the directory was created.
+   */
+  protected abstract boolean mkdir(String path);
+
+  /**
+   * Get root directory.
+   *
+   * @return Root directory.
+   */
+  protected abstract String getRootDir();
+
+  /**
+   * Set the driver as initialized.
+   *
+   * @param ini If the driver is initialized.
+   */
+  public void setInitialized(boolean ini) {
+    this.initialized = ini;
+  }
+
+  @Override
+  public boolean initDriver() {
+    String rootDir = getRootDir();
+    try {
+      if (rootDir == null) {
+        LOG.error("Invalid root directory, unable to initialize driver.");
+        return false;
+      }
+
+      // Check root path
+      if (!exists(rootDir)) {
+        if (!mkdir(rootDir)) {
+          LOG.error("Cannot create State Store root directory {}", rootDir);
+          return false;
+        }
+      }
+    } catch (Exception ex) {
+      LOG.error(
+          "Cannot initialize filesystem using root directory {}", rootDir, ex);
+      return false;
+    }
+    setInitialized(true);
+    return true;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean initRecordStorage(
+      String className, Class<T> recordClass) {
+
+    String dataDirPath = getRootDir() + "/" + className;
+    try {
+      // Create data directories for files
+      if (!exists(dataDirPath)) {
+        LOG.info("{} data directory doesn't exist, creating it", dataDirPath);
+        if (!mkdir(dataDirPath)) {
+          LOG.error("Cannot create data directory {}", dataDirPath);
+          return false;
+        }
+        String dataFilePath = dataDirPath + "/" + DATA_FILE_NAME;
+        if (!exists(dataFilePath)) {
+          // Create empty file
+          List<T> emtpyList = new ArrayList<>();
+          if(!writeAll(emtpyList, recordClass)) {
+            LOG.error("Cannot create data file {}", dataFilePath);
+            return false;
+          }
+        }
+      }
+    } catch (Exception ex) {
+      LOG.error("Cannot create data directory {}", dataDirPath, ex);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Read all lines from a file and deserialize into the desired record type.
+   *
+   * @param reader Open handle for the file.
+   * @param recordClass Record class to create.
+   * @param includeDates True if dateModified/dateCreated are serialized.
+   * @return List of records.
+   * @throws IOException
+   */
+  private <T extends BaseRecord> List<T> getAllFile(
+      BufferedReader reader, Class<T> clazz, boolean includeDates)
+          throws IOException {
+
+    List<T> ret = new ArrayList<T>();
+    String line;
+    while ((line = reader.readLine()) != null) {
+      if (!line.startsWith("#") && line.length() > 0) {
+        try {
+          T record = newRecord(line, clazz, includeDates);
+          ret.add(record);
+        } catch (Exception ex) {
+          LOG.error("Cannot parse line in data source file: {}", line, ex);
+        }
+      }
+    }
+    return ret;
+  }
+
+  @Override
+  public <T extends BaseRecord> QueryResult<T> get(Class<T> clazz)
+      throws IOException {
+    return get(clazz, (String)null);
+  }
+
+  @Override
+  public <T extends BaseRecord> QueryResult<T> get(Class<T> clazz, String sub)
+      throws IOException {
+    verifyDriverReady();
+    BufferedReader reader = null;
+    lockRecordRead(clazz);
+    try {
+      reader = getReader(clazz, sub);
+      List<T> data = getAllFile(reader, clazz, true);
+      return new QueryResult<T>(data, getTime());
+    } catch (Exception ex) {
+      LOG.error("Cannot fetch records {}", clazz.getSimpleName());
+      throw new IOException("Cannot read from data store " + ex.getMessage());
+    } finally {
+      if (reader != null) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Failed closing file", e);
+        }
+      }
+      unlockRecordRead(clazz);
+    }
+  }
+
+  /**
+   * Overwrite the existing data with a new data set.
+   *
+   * @param list List of records to write.
+   * @param writer BufferedWriter stream to write to.
+   * @return If the records were succesfully written.
+   */
+  private <T extends BaseRecord> boolean writeAllFile(
+      Collection<T> records, BufferedWriter writer) {
+
+    try {
+      for (BaseRecord record : records) {
+        try {
+          String data = serializeString(record);
+          writer.write(data);
+          writer.newLine();
+        } catch (IllegalArgumentException ex) {
+          LOG.error("Cannot write record {} to file", record, ex);
+        }
+      }
+      writer.flush();
+      return true;
+    } catch (IOException e) {
+      LOG.error("Cannot commit records to file", e);
+      return false;
+    }
+  }
+
+  /**
+   * Overwrite the existing data with a new data set. Replaces all records in
+   * the data store for this record class. If all records in the data store are
+   * not successfully committed, this function must return false and leave the
+   * data store unchanged.
+   *
+   * @param records List of records to write. All records must be of type
+   *                recordClass.
+   * @param recordClass Class of record to replace.
+   * @return true if all operations were successful, false otherwise.
+   * @throws StateStoreUnavailableException
+   */
+  public <T extends BaseRecord> boolean writeAll(
+      Collection<T> records, Class<T> recordClass)
+          throws StateStoreUnavailableException {
+    verifyDriverReady();
+    lockRecordWrite(recordClass);
+    BufferedWriter writer = null;
+    try {
+      writer = getWriter(recordClass, null);
+      return writeAllFile(records, writer);
+    } catch (Exception e) {
+      LOG.error(
+          "Cannot add records to file for {}", recordClass.getSimpleName(), e);
+      return false;
+    } finally {
+      if (writer != null) {
+        try {
+          writer.close();
+        } catch (IOException e) {
+          LOG.error(
+              "Cannot close writer for {}", recordClass.getSimpleName(), e);
+        }
+      }
+      unlockRecordWrite(recordClass);
+    }
+  }
+
+  /**
+   * Get the data file name.
+   *
+   * @return Data file name.
+   */
+  protected String getDataFileName() {
+    return DATA_FILE_NAME;
+  }
+
+  @Override
+  public boolean isDriverReady() {
+    return this.initialized;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean putAll(
+      List<T> records, boolean allowUpdate, boolean errorIfExists)
+          throws StateStoreUnavailableException {
+    verifyDriverReady();
+
+    if (records.isEmpty()) {
+      return true;
+    }
+
+    @SuppressWarnings("unchecked")
+    Class<T> clazz = (Class<T>) getRecordClass(records.get(0).getClass());
+    QueryResult<T> result;
+    try {
+      result = get(clazz);
+    } catch (IOException e) {
+      return false;
+    }
+    Map<Object, T> writeList = new HashMap<>();
+
+    // Write all of the existing records
+    for (T existingRecord : result.getRecords()) {
+      String key = existingRecord.getPrimaryKey();
+      writeList.put(key, existingRecord);
+    }
+
+    // Add inserts and updates, overwrite any existing values
+    for (T updatedRecord : records) {
+      try {
+        updatedRecord.validate();
+        String key = updatedRecord.getPrimaryKey();
+        if (writeList.containsKey(key) && allowUpdate) {
+          // Update
+          writeList.put(key, updatedRecord);
+          // Update the mod time stamp. Many backends will use their
+          // own timestamp for the mod time.
+          updatedRecord.setDateModified(this.getTime());
+        } else if (!writeList.containsKey(key)) {
+          // Insert
+          // Create/Mod timestamps are already initialized
+          writeList.put(key, updatedRecord);
+        } else if (errorIfExists) {
+          LOG.error("Attempt to insert record {} that already exists",
+              updatedRecord);
+          return false;
+        }
+      } catch (IllegalArgumentException ex) {
+        LOG.error("Cannot write invalid record to State Store", ex);
+        return false;
+      }
+    }
+
+    // Write all
+    boolean status = writeAll(writeList.values(), clazz);
+    return status;
+  }
+
+  @Override
+  public <T extends BaseRecord> int remove(Class<T> clazz, Query<T> query)
+      throws StateStoreUnavailableException {
+    verifyDriverReady();
+
+    if (query == null) {
+      return 0;
+    }
+
+    int removed = 0;
+    // Get the current records
+    try {
+      final QueryResult<T> result = get(clazz);
+      final List<T> existingRecords = result.getRecords();
+      // Write all of the existing records except those to be removed
+      final List<T> recordsToRemove = filterMultiple(query, existingRecords);
+      removed = recordsToRemove.size();
+      final List<T> newRecords = new LinkedList<>();
+      for (T record : existingRecords) {
+        if (!recordsToRemove.contains(record)) {
+          newRecords.add(record);
+        }
+      }
+      if (!writeAll(newRecords, clazz)) {
+        throw new IOException(
+            "Cannot remove record " + clazz + " query " + query);
+      }
+    } catch (IOException e) {
+      LOG.error("Cannot remove records {} query {}", clazz, query, e);
+    }
+
+    return removed;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean removeAll(Class<T> clazz)
+      throws StateStoreUnavailableException {
+    verifyDriverReady();
+    List<T> emptyList = new ArrayList<>();
+    boolean status = writeAll(emptyList, clazz);
+    return status;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java
new file mode 100644
index 0000000..24e9660
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.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.federation.store.driver.impl;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.io.Files;
+
+/**
+ * StateStoreDriver implementation based on a local file.
+ */
+public class StateStoreFileImpl extends StateStoreFileBaseImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreFileImpl.class);
+
+  /** Configuration keys. */
+  public static final String FEDERATION_STORE_FILE_DIRECTORY =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "driver.file.directory";
+
+  /** Synchronization. */
+  private static final ReadWriteLock READ_WRITE_LOCK =
+      new ReentrantReadWriteLock();
+
+  /** Root directory for the state store. */
+  private String rootDirectory;
+
+
+  @Override
+  protected boolean exists(String path) {
+    File test = new File(path);
+    return test.exists();
+  }
+
+  @Override
+  protected boolean mkdir(String path) {
+    File dir = new File(path);
+    return dir.mkdirs();
+  }
+
+  @Override
+  protected String getRootDir() {
+    if (this.rootDirectory == null) {
+      String dir = getConf().get(FEDERATION_STORE_FILE_DIRECTORY);
+      if (dir == null) {
+        File tempDir = Files.createTempDir();
+        dir = tempDir.getAbsolutePath();
+      }
+      this.rootDirectory = dir;
+    }
+    return this.rootDirectory;
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordWrite(Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.writeLock().lock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordWrite(
+      Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.writeLock().unlock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordRead(Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.readLock().lock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordRead(Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.readLock().unlock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedReader getReader(
+      Class<T> clazz, String sub) {
+    String filename = StateStoreUtils.getRecordName(clazz);
+    if (sub != null && sub.length() > 0) {
+      filename += "/" + sub;
+    }
+    filename += "/" + getDataFileName();
+
+    try {
+      LOG.debug("Loading file: {}", filename);
+      File file = new File(getRootDir(), filename);
+      FileInputStream fis = new FileInputStream(file);
+      InputStreamReader isr =
+          new InputStreamReader(fis, StandardCharsets.UTF_8);
+      BufferedReader reader = new BufferedReader(isr);
+      return reader;
+    } catch (Exception ex) {
+      LOG.error(
+          "Cannot open read stream for record {}", clazz.getSimpleName(), ex);
+      return null;
+    }
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedWriter getWriter(
+      Class<T> clazz, String sub) {
+    String filename = StateStoreUtils.getRecordName(clazz);
+    if (sub != null && sub.length() > 0) {
+      filename += "/" + sub;
+    }
+    filename += "/" + getDataFileName();
+
+    try {
+      File file = new File(getRootDir(), filename);
+      FileOutputStream fos = new FileOutputStream(file, false);
+      OutputStreamWriter osw =
+          new OutputStreamWriter(fos, StandardCharsets.UTF_8);
+      BufferedWriter writer = new BufferedWriter(osw);
+      return writer;
+    } catch (IOException ex) {
+      LOG.error(
+          "Cannot open read stream for record {}", clazz.getSimpleName(), ex);
+      return null;
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    setInitialized(false);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
new file mode 100644
index 0000000..5968421
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
@@ -0,0 +1,178 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StateStoreDriver} implementation based on a filesystem. The most common uses
+ * HDFS as a backend.
+ */
+public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreFileSystemImpl.class);
+
+
+  /** Configuration keys. */
+  public static final String FEDERATION_STORE_FS_PATH =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "driver.fs.path";
+
+  /** File system to back the State Store. */
+  private FileSystem fs;
+  /** Working path in the filesystem. */
+  private String workPath;
+
+  @Override
+  protected boolean exists(String path) {
+    try {
+      return fs.exists(new Path(path));
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  @Override
+  protected boolean mkdir(String path) {
+    try {
+      return fs.mkdirs(new Path(path));
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  @Override
+  protected String getRootDir() {
+    if (this.workPath == null) {
+      String rootPath = getConf().get(FEDERATION_STORE_FS_PATH);
+      URI workUri;
+      try {
+        workUri = new URI(rootPath);
+        fs = FileSystem.get(workUri, getConf());
+      } catch (Exception ex) {
+        return null;
+      }
+      this.workPath = rootPath;
+    }
+    return this.workPath;
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+  }
+
+  /**
+   * Get the folder path for the record class' data.
+   *
+   * @param cls Data record class.
+   * @return Path of the folder containing the record class' data files.
+   */
+  private Path getPathForClass(Class<? extends BaseRecord> clazz) {
+    if (clazz == null) {
+      return null;
+    }
+    // TODO extract table name from class: entry.getTableName()
+    String className = StateStoreUtils.getRecordName(clazz);
+    return new Path(workPath, className);
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordRead(Class<T> clazz) {
+    // Not required, synced with HDFS leasing
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordRead(Class<T> clazz) {
+    // Not required, synced with HDFS leasing
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordWrite(Class<T> clazz) {
+    // TODO -> wait for lease to be available
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordWrite(Class<T> clazz) {
+    // TODO -> ensure lease is closed for the file
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedReader getReader(
+      Class<T> clazz, String sub) {
+
+    Path path = getPathForClass(clazz);
+    if (sub != null && sub.length() > 0) {
+      path = Path.mergePaths(path, new Path("/" + sub));
+    }
+    path = Path.mergePaths(path, new Path("/" + getDataFileName()));
+
+    try {
+      FSDataInputStream fdis = fs.open(path);
+      InputStreamReader isr =
+          new InputStreamReader(fdis, StandardCharsets.UTF_8);
+      BufferedReader reader = new BufferedReader(isr);
+      return reader;
+    } catch (IOException ex) {
+      LOG.error("Cannot open write stream for {}  to {}",
+          clazz.getSimpleName(), path);
+      return null;
+    }
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedWriter getWriter(
+      Class<T> clazz, String sub) {
+
+    Path path = getPathForClass(clazz);
+    if (sub != null && sub.length() > 0) {
+      path = Path.mergePaths(path, new Path("/" + sub));
+    }
+    path = Path.mergePaths(path, new Path("/" + getDataFileName()));
+
+    try {
+      FSDataOutputStream fdos = fs.create(path, true);
+      OutputStreamWriter osw =
+          new OutputStreamWriter(fdos, StandardCharsets.UTF_8);
+      BufferedWriter writer = new BufferedWriter(osw);
+      return writer;
+    } catch (IOException ex) {
+      LOG.error("Cannot open write stream for {} to {}",
+          clazz.getSimpleName(), path);
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
new file mode 100644
index 0000000..e9b3fdf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
@@ -0,0 +1,77 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * State Store driver that stores a serialization of the records. The serializer
+ * is pluggable.
+ */
+public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
+
+  /** Default serializer for this driver. */
+  private StateStoreSerializer serializer;
+
+
+  @Override
+  public boolean init(final Configuration config, final String id,
+      final Collection<Class<? extends BaseRecord>> records) {
+    boolean ret = super.init(config, id, records);
+
+    this.serializer = StateStoreSerializer.getSerializer(config);
+
+    return ret;
+  }
+
+  /**
+   * Serialize a record using the serializer.
+   * @param record Record to serialize.
+   * @return Byte array with the serialization of the record.
+   */
+  protected <T extends BaseRecord> byte[] serialize(T record) {
+    return serializer.serialize(record);
+  }
+
+  /**
+   * Serialize a record using the serializer.
+   * @param record Record to serialize.
+   * @return String with the serialization of the record.
+   */
+  protected <T extends BaseRecord> String serializeString(T record) {
+    return serializer.serializeString(record);
+  }
+
+  /**
+   * Creates a record from an input data string.
+   * @param data Serialized text of the record.
+   * @param clazz Record class.
+   * @param includeDates If dateModified and dateCreated are serialized.
+   * @return The created record.
+   * @throws IOException
+   */
+  protected <T extends BaseRecord> T newRecord(
+      String data, Class<T> clazz, boolean includeDates) throws IOException {
+    return serializer.deserialize(data, clazz);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
index 4192a3d..79f99c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
@@ -123,6 +123,24 @@ public abstract class BaseRecord implements Comparable<BaseRecord> {
   }
 
   /**
+   * Check if this record matches a partial record.
+   *
+   * @param other Partial record.
+   * @return If this record matches.
+   */
+  public boolean like(BaseRecord other) {
+    if (other == null) {
+      return false;
+    }
+    Map<String, String> thisKeys = this.getPrimaryKeys();
+    Map<String, String> otherKeys = other.getPrimaryKeys();
+    if (thisKeys == null) {
+      return otherKeys == null;
+    }
+    return thisKeys.equals(otherKeys);
+  }
+
+  /**
    * Override equals check to use primary key(s) for comparison.
    */
   @Override
@@ -186,4 +204,4 @@ public abstract class BaseRecord implements Comparable<BaseRecord> {
   public String toString() {
     return getPrimaryKey();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java
new file mode 100644
index 0000000..3c59abf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java
@@ -0,0 +1,66 @@
+/**
+ * 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.store.records;
+
+/**
+ * Check if a record matches a query. The query is usually a partial record.
+ *
+ * @param <T> Type of the record to query.
+ */
+public class Query<T extends BaseRecord> {
+
+  /** Partial object to compare against. */
+  private final T partial;
+
+
+  /**
+   * Create a query to search for a partial record.
+   *
+   * @param partial It defines the attributes to search.
+   */
+  public Query(final T part) {
+    this.partial = part;
+  }
+
+  /**
+   * Get the partial record used to query.
+   *
+   * @return The partial record used for the query.
+   */
+  public T getPartial() {
+    return this.partial;
+  }
+
+  /**
+   * Check if a record matches the primary keys or the partial record.
+   *
+   * @param other Record to check.
+   * @return If the record matches. Don't match if there is no partial.
+   */
+  public boolean matches(T other) {
+    if (this.partial == null) {
+      return false;
+    }
+    return this.partial.like(other);
+  }
+
+  @Override
+  public String toString() {
+    return "Checking: " + this.partial;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/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 f1dce6b..a8410b2 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,4 +4643,20 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.store.driver.class</name>
+    <value>org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl</value>
+    <description>
+      Class to implement the State Store. By default it uses the local disk.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.store.connection.test</name>
+    <value>60000</value>
+    <description>
+      How often to check for the connection to the State Store in milliseconds.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2096bce7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
new file mode 100644
index 0000000..fc5aebd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
@@ -0,0 +1,232 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS;
+import static org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl.FEDERATION_STORE_FILE_DIRECTORY;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileBaseImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Utilities to test the State Store.
+ */
+public final class FederationStateStoreTestUtils {
+
+  private FederationStateStoreTestUtils() {
+    // Utility Class
+  }
+
+  /**
+   * Get the default State Store driver implementation.
+   *
+   * @return Class of the default State Store driver implementation.
+   */
+  public static Class<? extends StateStoreDriver> getDefaultDriver() {
+    return DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS_DEFAULT;
+  }
+
+  /**
+   * Create a default State Store configuration.
+   *
+   * @return State Store configuration.
+   */
+  public static Configuration getStateStoreConfiguration() {
+    Class<? extends StateStoreDriver> clazz = getDefaultDriver();
+    return getStateStoreConfiguration(clazz);
+  }
+
+  /**
+   * Create a new State Store configuration for a particular driver.
+   *
+   * @param clazz Class of the driver to create.
+   * @return State Store configuration.
+   */
+  public static Configuration getStateStoreConfiguration(
+      Class<? extends StateStoreDriver> clazz) {
+    Configuration conf = new HdfsConfiguration(false);
+
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "hdfs://test");
+
+    conf.setClass(FEDERATION_STORE_DRIVER_CLASS, clazz, StateStoreDriver.class);
+
+    if (clazz.isAssignableFrom(StateStoreFileBaseImpl.class)) {
+      setFileConfiguration(conf);
+    }
+    return conf;
+  }
+
+  /**
+   * Create a new State Store based on a configuration.
+   *
+   * @param configuration Configuration for the State Store.
+   * @return New State Store service.
+   * @throws IOException If it cannot create the State Store.
+   * @throws InterruptedException If we cannot wait for the store to start.
+   */
+  public static StateStoreService getStateStore(
+      Configuration configuration) throws IOException, InterruptedException {
+
+    StateStoreService stateStore = new StateStoreService();
+    assertNotNull(stateStore);
+
+    // Set unique identifier, this is normally the router address
+    String identifier = UUID.randomUUID().toString();
+    stateStore.setIdentifier(identifier);
+
+    stateStore.init(configuration);
+    stateStore.start();
+
+    // Wait for state store to connect
+    waitStateStore(stateStore, TimeUnit.SECONDS.toMillis(10));
+
+    return stateStore;
+  }
+
+  /**
+   * Wait for the State Store to initialize its driver.
+   *
+   * @param stateStore State Store.
+   * @param timeoutMs Time out in milliseconds.
+   * @throws IOException If the State Store cannot be reached.
+   * @throws InterruptedException If the sleep is interrupted.
+   */
+  public static void waitStateStore(StateStoreService stateStore,
+      long timeoutMs) throws IOException, InterruptedException {
+    long startingTime = Time.monotonicNow();
+    while (!stateStore.isDriverReady()) {
+      Thread.sleep(100);
+      if (Time.monotonicNow() - startingTime > timeoutMs) {
+        throw new IOException("Timeout waiting for State Store to connect");
+      }
+    }
+  }
+
+  /**
+   * Delete the default State Store.
+   *
+   * @throws IOException
+   */
+  public static void deleteStateStore() throws IOException {
+    Class<? extends StateStoreDriver> driverClass = getDefaultDriver();
+    deleteStateStore(driverClass);
+  }
+
+  /**
+   * Delete the State Store.
+   * @param driverClass Class of the State Store driver implementation.
+   * @throws IOException If it cannot be deleted.
+   */
+  public static void deleteStateStore(
+      Class<? extends StateStoreDriver> driverClass) throws IOException {
+
+    if (StateStoreFileBaseImpl.class.isAssignableFrom(driverClass)) {
+      String workingDirectory = System.getProperty("user.dir");
+      File dir = new File(workingDirectory + "/statestore");
+      if (dir.exists()) {
+        FileUtils.cleanDirectory(dir);
+      }
+    }
+  }
+
+  /**
+   * Set the default configuration for drivers based on files.
+   *
+   * @param conf Configuration to extend.
+   */
+  public static void setFileConfiguration(Configuration conf) {
+    String workingPath = System.getProperty("user.dir");
+    String stateStorePath = workingPath + "/statestore";
+    conf.set(FEDERATION_STORE_FILE_DIRECTORY, stateStorePath);
+  }
+
+  /**
+   * Clear all the records from the State Store.
+   *
+   * @param store State Store to remove records from.
+   * @return If the State Store was cleared.
+   * @throws IOException If it cannot clear the State Store.
+   */
+  public static boolean clearAllRecords(StateStoreService store)
+      throws IOException {
+    Collection<Class<? extends BaseRecord>> allRecords =
+        store.getSupportedRecords();
+    for (Class<? extends BaseRecord> recordType : allRecords) {
+      if (!clearRecords(store, recordType)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Clear records from a certain type from the State Store.
+   *
+   * @param store State Store to remove records from.
+   * @param recordClass Class of the records to remove.
+   * @return If the State Store was cleared.
+   * @throws IOException If it cannot clear the State Store.
+   */
+  public static <T extends BaseRecord> boolean clearRecords(
+      StateStoreService store, Class<T> recordClass) throws IOException {
+    List<T> emptyList = new ArrayList<>();
+    if (!synchronizeRecords(store, emptyList, recordClass)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Synchronize a set of records. Remove all and keep the ones specified.
+   *
+   * @param stateStore State Store service managing the driver.
+   * @param records Records to add.
+   * @param clazz Class of the record to synchronize.
+   * @return If the synchronization succeeded.
+   * @throws IOException If it cannot connect to the State Store.
+   */
+  public static <T extends BaseRecord> boolean synchronizeRecords(
+      StateStoreService stateStore, List<T> records, Class<T> clazz)
+          throws IOException {
+    StateStoreDriver driver = stateStore.getDriver();
+    driver.verifyDriverReady();
+    if (driver.removeAll(clazz)) {
+      if (driver.putAll(records, true, false)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
\ 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