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/02 21:20:45 UTC

[01/48] hadoop git commit: YARN-6721. container-executor should have stack checking [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10467 da654226d -> 8c2b7aeaf (forced update)


YARN-6721. container-executor should have stack checking

Signed-off-by: Chris Douglas <cd...@apache.org>


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

Branch: refs/heads/HDFS-10467
Commit: 0adc3a0533e90c8a42c5924be4847753e7f8d281
Parents: 1904100
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jun 23 11:39:37 2017 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Thu Aug 31 19:39:31 2017 -0700

----------------------------------------------------------------------
 .../hadoop-common/HadoopCommon.cmake            |  7 ++-
 .../src/CMakeLists.txt                          | 45 ++++++++++++++++++++
 2 files changed, 48 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc3a05/hadoop-common-project/hadoop-common/HadoopCommon.cmake
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/HadoopCommon.cmake b/hadoop-common-project/hadoop-common/HadoopCommon.cmake
index faabeed..63de1de 100644
--- a/hadoop-common-project/hadoop-common/HadoopCommon.cmake
+++ b/hadoop-common-project/hadoop-common/HadoopCommon.cmake
@@ -121,7 +121,9 @@ endmacro()
 # set the shared compiler flags
 # support for GNU C/C++, add other compilers as necessary
 
-if (CMAKE_C_COMPILER_ID STREQUAL "GNU")
+if (CMAKE_C_COMPILER_ID STREQUAL "GNU" OR
+    CMAKE_C_COMPILER_ID STREQUAL "Clang" OR
+    CMAKE_C_COMPILER_ID STREQUAL "AppleClang")
   if(NOT DEFINED GCC_SHARED_FLAGS)
     find_package(Threads REQUIRED)
     if(CMAKE_USE_PTHREADS_INIT)
@@ -130,9 +132,6 @@ if (CMAKE_C_COMPILER_ID STREQUAL "GNU")
       set(GCC_SHARED_FLAGS "-g -O2 -Wall -D_FILE_OFFSET_BITS=64")
     endif()
   endif()
-elseif (CMAKE_C_COMPILER_ID STREQUAL "Clang" OR
-        CMAKE_C_COMPILER_ID STREQUAL "AppleClang")
-  set(GCC_SHARED_FLAGS "-g -O2 -Wall -D_FILE_OFFSET_BITS=64")
 endif()
 
 # Set the shared linker flags.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc3a05/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index 7f2b00d..3d5b506 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -53,6 +53,51 @@ if(APPLE)
   set(EXTRA_LIBS ${COCOA_LIBRARY})
 endif(APPLE)
 
+include(CheckCCompilerFlag)
+
+# Building setuid = attempt to enable stack protection.
+# assumption here is that the C compiler and the C++
+# compiler match.  need both so that gtest gets same
+# stack treatment that the real c-e does
+IF(CMAKE_C_COMPILER_ID STREQUAL "GNU")
+    CHECK_C_COMPILER_FLAG("-fstack-check" STACKRESULT)
+    IF(STACKRESULT)
+      SET (CMAKE_C_FLAGS "-fstack-check ${CMAKE_C_FLAGS}")
+      SET (CMAKE_CXX_FLAGS "-fstack-check ${CMAKE_CXX_FLAGS}")
+    ENDIF()
+ELSEIF(CMAKE_C_COMPILER_ID STREQUAL "Clang" OR
+       CMAKE_C_COMPILER_ID STREQUAL "AppleClang")
+
+  # clang is a bit difficult here:
+  # - some versions don't support the flag
+  # - some versions support the flag, despite not having
+  #   the library that is actually required (!)
+  # Notably, Xcode is a problem here.
+  # In the end, this is needlessly complex. :(
+
+  SET(PRE_SANITIZE ${CMAKE_REQUIRED_FLAGS})
+  SET(CMAKE_REQUIRED_FLAGS "-fsanitize=safe-stack ${CMAKE_REQUIRED_FLAGS}")
+  CHECK_C_COMPILER_FLAG("" STACKRESULT)
+  SET(CMAKE_REQUIRED_FLAGS ${PRE_SANITIZE})
+  IF(STACKRESULT)
+     SET(CMAKE_C_FLAGS "-fsanitize=safe-stack ${CMAKE_C_FLAGS}")
+     SET(CMAKE_CXX_FLAGS "-fsanitize=safe-stack ${CMAKE_CXX_FLAGS}")
+  ENDIF()
+ELSEIF(CMAKE_C_COMPILER_ID STREQUAL "SunPro")
+
+  # this appears to only be supported on SPARC, for some reason
+
+  CHECK_C_COMPILER_FLAG("-xcheck=stkovf" STACKRESULT)
+  IF(STACKRESULT)
+    SET (CMAKE_C_FLAGS "-xcheck=stkovf ${CMAKE_C_FLAGS}")
+    SET (CMAKE_CXX_FLAGS "-xcheck=stkovf ${CMAKE_CXX_FLAGS}")
+  ENDIF()
+ENDIF()
+
+IF(NOT STACKRESULT)
+   MESSAGE(WARNING "Stack Clash security protection is not suported.")
+ENDIF()
+
 function(output_directory TGT DIR)
     set_target_properties(${TGT} PROPERTIES
         RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")


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


[30/48] 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/c0a06922
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c0a06922
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c0a06922

Branch: refs/heads/HDFS-10467
Commit: c0a069224a6eff85d041b92503a981b6a93d8478
Parents: 5da7d3f
Author: Inigo Goiri <in...@apache.org>
Authored: Fri Jul 28 15:55:10 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:08 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0a06922/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


[29/48] 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/9176c5a8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9176c5a8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9176c5a8

Branch: refs/heads/HDFS-10467
Commit: 9176c5a82896b309cfeb33398c0b8b9a52d55a24
Parents: cd09291
Author: Inigo <in...@apache.org>
Authored: Wed Mar 29 19:35:06 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:07 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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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/9176c5a8/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


[14/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
new file mode 100644
index 0000000..66ada49
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
@@ -0,0 +1,132 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
+
+/**
+ * Interface to create a DynamoDB client.
+ *
+ * Implementation should be configured for setting and getting configuration.
+ */
+@InterfaceAudience.Private
+public interface DynamoDBClientFactory extends Configurable {
+  Logger LOG = LoggerFactory.getLogger(DynamoDBClientFactory.class);
+
+  /**
+   * Create a DynamoDB client object from configuration.
+   *
+   * The DynamoDB client to create does not have to relate to any S3 buckets.
+   * All information needed to create a DynamoDB client is from the hadoop
+   * configuration. Specially, if the region is not configured, it will use the
+   * provided region parameter. If region is neither configured nor provided,
+   * it will indicate an error.
+   *
+   * @param defaultRegion the default region of the AmazonDynamoDB client
+   * @return a new DynamoDB client
+   * @throws IOException if any IO error happens
+   */
+  AmazonDynamoDB createDynamoDBClient(String defaultRegion) throws IOException;
+
+  /**
+   * The default implementation for creating an AmazonDynamoDB.
+   */
+  class DefaultDynamoDBClientFactory extends Configured
+      implements DynamoDBClientFactory {
+    @Override
+    public AmazonDynamoDB createDynamoDBClient(String defaultRegion)
+        throws IOException {
+      Preconditions.checkNotNull(getConf(),
+          "Should have been configured before usage");
+
+      final Configuration conf = getConf();
+      final AWSCredentialsProvider credentials =
+          createAWSCredentialProviderSet(null, conf);
+      final ClientConfiguration awsConf =
+          DefaultS3ClientFactory.createAwsConf(conf);
+
+      final String region = getRegion(conf, defaultRegion);
+      LOG.debug("Creating DynamoDB client in region {}", region);
+
+      return AmazonDynamoDBClientBuilder.standard()
+          .withCredentials(credentials)
+          .withClientConfiguration(awsConf)
+          .withRegion(region)
+          .build();
+    }
+
+    /**
+     * Helper method to get and validate the AWS region for DynamoDBClient.
+     *
+     * @param conf configuration
+     * @param defaultRegion the default region
+     * @return configured region or else the provided default region
+     * @throws IOException if the region is not valid
+     */
+    static String getRegion(Configuration conf, String defaultRegion)
+        throws IOException {
+      String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+      if (StringUtils.isEmpty(region)) {
+        region = defaultRegion;
+      }
+      try {
+        Regions.fromName(region);
+      } catch (IllegalArgumentException | NullPointerException e) {
+        throw new IOException("Invalid region specified: " + region + "; " +
+            "Region can be configured with " + S3GUARD_DDB_REGION_KEY + ": " +
+            validRegionsString());
+      }
+      return region;
+    }
+
+    private static String validRegionsString() {
+      final String delimiter = ", ";
+      Regions[] regions = Regions.values();
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < regions.length; i++) {
+        if (i > 0) {
+          sb.append(delimiter);
+        }
+        sb.append(regions[i].getName());
+      }
+      return sb.toString();
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
new file mode 100644
index 0000000..1bed03d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -0,0 +1,1010 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.ItemCollection;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.document.PutItemOutcome;
+import com.amazonaws.services.dynamodbv2.document.QueryOutcome;
+import com.amazonaws.services.dynamodbv2.document.ScanOutcome;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
+import com.amazonaws.services.dynamodbv2.document.spec.GetItemSpec;
+import com.amazonaws.services.dynamodbv2.document.spec.QuerySpec;
+import com.amazonaws.services.dynamodbv2.document.utils.ValueMap;
+import com.amazonaws.services.dynamodbv2.model.CreateTableRequest;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
+import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.WriteRequest;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
+
+/**
+ * DynamoDBMetadataStore is a {@link MetadataStore} that persists
+ * file system metadata to DynamoDB.
+ *
+ * The current implementation uses a schema consisting of a single table.  The
+ * name of the table can be configured by config key
+ * {@link org.apache.hadoop.fs.s3a.Constants#S3GUARD_DDB_TABLE_NAME_KEY}.
+ * By default, it matches the name of the S3 bucket.  Each item in the table
+ * represents a single directory or file.  Its path is split into separate table
+ * attributes:
+ * <ul>
+ * <li> parent (absolute path of the parent, with bucket name inserted as
+ * first path component). </li>
+ * <li> child (path of that specific child, relative to parent). </li>
+ * <li> optional boolean attribute tracking whether the path is a directory.
+ *      Absence or a false value indicates the path is a file. </li>
+ * <li> optional long attribute revealing modification time of file.
+ *      This attribute is meaningful only to file items.</li>
+ * <li> optional long attribute revealing file length.
+ *      This attribute is meaningful only to file items.</li>
+ * <li> optional long attribute revealing block size of the file.
+ *      This attribute is meaningful only to file items.</li>
+ * </ul>
+ *
+ * The DynamoDB partition key is the parent, and the range key is the child.
+ *
+ * To allow multiple buckets to share the same DynamoDB table, the bucket
+ * name is treated as the root directory.
+ *
+ * For example, assume the consistent store contains metadata representing this
+ * file system structure:
+ *
+ * <pre>
+ * s3a://bucket/dir1
+ * |-- dir2
+ * |   |-- file1
+ * |   `-- file2
+ * `-- dir3
+ *     |-- dir4
+ *     |   `-- file3
+ *     |-- dir5
+ *     |   `-- file4
+ *     `-- dir6
+ * </pre>
+ *
+ * This is persisted to a single DynamoDB table as:
+ *
+ * <pre>
+ * =========================================================================
+ * | parent                 | child | is_dir | mod_time | len |     ...    |
+ * =========================================================================
+ * | /bucket                | dir1  | true   |          |     |            |
+ * | /bucket/dir1           | dir2  | true   |          |     |            |
+ * | /bucket/dir1           | dir3  | true   |          |     |            |
+ * | /bucket/dir1/dir2      | file1 |        |   100    | 111 |            |
+ * | /bucket/dir1/dir2      | file2 |        |   200    | 222 |            |
+ * | /bucket/dir1/dir3      | dir4  | true   |          |     |            |
+ * | /bucket/dir1/dir3      | dir5  | true   |          |     |            |
+ * | /bucket/dir1/dir3/dir4 | file3 |        |   300    | 333 |            |
+ * | /bucket/dir1/dir3/dir5 | file4 |        |   400    | 444 |            |
+ * | /bucket/dir1/dir3      | dir6  | true   |          |     |            |
+ * =========================================================================
+ * </pre>
+ *
+ * This choice of schema is efficient for read access patterns.
+ * {@link #get(Path)} can be served from a single item lookup.
+ * {@link #listChildren(Path)} can be served from a query against all rows
+ * matching the parent (the partition key) and the returned list is guaranteed
+ * to be sorted by child (the range key).  Tracking whether or not a path is a
+ * directory helps prevent unnecessary queries during traversal of an entire
+ * sub-tree.
+ *
+ * Some mutating operations, notably {@link #deleteSubtree(Path)} and
+ * {@link #move(Collection, Collection)}, are less efficient with this schema.
+ * They require mutating multiple items in the DynamoDB table.
+ *
+ * By default, DynamoDB access is performed within the same AWS region as
+ * the S3 bucket that hosts the S3A instance.  During initialization, it checks
+ * the location of the S3 bucket and creates a DynamoDB client connected to the
+ * same region. The region may also be set explicitly by setting the config
+ * parameter {@code fs.s3a.s3guard.ddb.region} to the corresponding region.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DynamoDBMetadataStore implements MetadataStore {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      DynamoDBMetadataStore.class);
+
+  /** parent/child name to use in the version marker. */
+  public static final String VERSION_MARKER = "../VERSION";
+
+  /** Current version number. */
+  public static final int VERSION = 100;
+
+  /** Error: version marker not found in table. */
+  public static final String E_NO_VERSION_MARKER
+      = "S3Guard table lacks version marker.";
+
+  /** Error: version mismatch. */
+  public static final String E_INCOMPATIBLE_VERSION
+      = "Database table is from an incompatible S3Guard version.";
+
+  /** Initial delay for retries when batched operations get throttled by
+   * DynamoDB. Value is {@value} msec. */
+  public static final long MIN_RETRY_SLEEP_MSEC = 100;
+
+  private static ValueMap deleteTrackingValueMap =
+      new ValueMap().withBoolean(":false", false);
+
+  private DynamoDB dynamoDB;
+  private String region;
+  private Table table;
+  private String tableName;
+  private Configuration conf;
+  private String username;
+
+  private RetryPolicy dataAccessRetryPolicy;
+  private S3AInstrumentation.S3GuardInstrumentation instrumentation;
+
+  /**
+   * A utility function to create DynamoDB instance.
+   * @param conf the file system configuration
+   * @param s3Region region of the associated S3 bucket (if any).
+   * @return DynamoDB instance.
+   * @throws IOException I/O error.
+   */
+  private static DynamoDB createDynamoDB(Configuration conf, String s3Region)
+      throws IOException {
+    Preconditions.checkNotNull(conf);
+    final Class<? extends DynamoDBClientFactory> cls = conf.getClass(
+        S3GUARD_DDB_CLIENT_FACTORY_IMPL,
+        S3GUARD_DDB_CLIENT_FACTORY_IMPL_DEFAULT,
+        DynamoDBClientFactory.class);
+    LOG.debug("Creating DynamoDB client {} with S3 region {}", cls, s3Region);
+    final AmazonDynamoDB dynamoDBClient = ReflectionUtils.newInstance(cls, conf)
+        .createDynamoDBClient(s3Region);
+    return new DynamoDB(dynamoDBClient);
+  }
+
+  @Override
+  public void initialize(FileSystem fs) throws IOException {
+    Preconditions.checkArgument(fs instanceof S3AFileSystem,
+        "DynamoDBMetadataStore only supports S3A filesystem.");
+    final S3AFileSystem s3afs = (S3AFileSystem) fs;
+    instrumentation = s3afs.getInstrumentation().getS3GuardInstrumentation();
+    final String bucket = s3afs.getBucket();
+    String confRegion = s3afs.getConf().getTrimmed(S3GUARD_DDB_REGION_KEY);
+    if (!StringUtils.isEmpty(confRegion)) {
+      region = confRegion;
+      LOG.debug("Overriding S3 region with configured DynamoDB region: {}",
+          region);
+    } else {
+      region = s3afs.getBucketLocation();
+      LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
+    }
+    username = s3afs.getUsername();
+    conf = s3afs.getConf();
+    dynamoDB = createDynamoDB(conf, region);
+
+    // use the bucket as the DynamoDB table name if not specified in config
+    tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY, bucket);
+    setMaxRetries(conf);
+
+    initTable();
+
+    instrumentation.initialized();
+  }
+
+  /**
+   * Performs one-time initialization of the metadata store via configuration.
+   *
+   * This initialization depends on the configuration object to get AWS
+   * credentials, DynamoDBFactory implementation class, DynamoDB endpoints,
+   * DynamoDB table names etc. After initialization, this metadata store does
+   * not explicitly relate to any S3 bucket, which be nonexistent.
+   *
+   * This is used to operate the metadata store directly beyond the scope of the
+   * S3AFileSystem integration, e.g. command line tools.
+   * Generally, callers should use {@link #initialize(FileSystem)}
+   * with an initialized {@code S3AFileSystem} instance.
+   *
+   * Without a filesystem to act as a reference point, the configuration itself
+   * must declare the table name and region in the
+   * {@link Constants#S3GUARD_DDB_TABLE_NAME_KEY} and
+   * {@link Constants#S3GUARD_DDB_REGION_KEY} respectively.
+   *
+   * @see #initialize(FileSystem)
+   * @throws IOException if there is an error
+   * @throws IllegalArgumentException if the configuration is incomplete
+   */
+  @Override
+  public void initialize(Configuration config) throws IOException {
+    conf = config;
+    // use the bucket as the DynamoDB table name if not specified in config
+    tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
+    Preconditions.checkArgument(!StringUtils.isEmpty(tableName),
+        "No DynamoDB table name configured");
+    region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+    Preconditions.checkArgument(!StringUtils.isEmpty(region),
+        "No DynamoDB region configured");
+    dynamoDB = createDynamoDB(conf, region);
+
+    username = UserGroupInformation.getCurrentUser().getShortUserName();
+    setMaxRetries(conf);
+
+    initTable();
+  }
+
+  /**
+   * Set retry policy. This is driven by the value of
+   * {@link Constants#S3GUARD_DDB_MAX_RETRIES} with an exponential backoff
+   * between each attempt of {@link #MIN_RETRY_SLEEP_MSEC} milliseconds.
+   * @param config
+   */
+  private void setMaxRetries(Configuration config) {
+    int maxRetries = config.getInt(S3GUARD_DDB_MAX_RETRIES,
+        S3GUARD_DDB_MAX_RETRIES_DEFAULT);
+    dataAccessRetryPolicy = RetryPolicies
+        .exponentialBackoffRetry(maxRetries, MIN_RETRY_SLEEP_MSEC,
+            TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public void delete(Path path) throws IOException {
+    innerDelete(path, true);
+  }
+
+  @Override
+  public void forgetMetadata(Path path) throws IOException {
+    innerDelete(path, false);
+  }
+
+  /**
+   * Inner delete option, action based on the {@code tombstone} flag.
+   * No tombstone: delete the entry. Tombstone: create a tombstone entry.
+   * There is no check as to whether the entry exists in the table first.
+   * @param path path to delete
+   * @param tombstone flag to create a tombstone marker
+   * @throws IOException I/O error.
+   */
+  private void innerDelete(Path path, boolean tombstone)
+      throws IOException {
+    path = checkPath(path);
+    LOG.debug("Deleting from table {} in region {}: {}",
+        tableName, region, path);
+
+    // deleting nonexistent item consumes 1 write capacity; skip it
+    if (path.isRoot()) {
+      LOG.debug("Skip deleting root directory as it does not exist in table");
+      return;
+    }
+
+    try {
+      if (tombstone) {
+        Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
+            PathMetadata.tombstone(path));
+        table.putItem(item);
+      } else {
+        table.deleteItem(pathToKey(path));
+      }
+    } catch (AmazonClientException e) {
+      throw translateException("delete", path, e);
+    }
+  }
+
+  @Override
+  public void deleteSubtree(Path path) throws IOException {
+    path = checkPath(path);
+    LOG.debug("Deleting subtree from table {} in region {}: {}",
+        tableName, region, path);
+
+    final PathMetadata meta = get(path);
+    if (meta == null || meta.isDeleted()) {
+      LOG.debug("Subtree path {} does not exist; this will be a no-op", path);
+      return;
+    }
+
+    for (DescendantsIterator desc = new DescendantsIterator(this, meta);
+         desc.hasNext();) {
+      innerDelete(desc.next().getPath(), true);
+    }
+  }
+
+  private Item getConsistentItem(PrimaryKey key) {
+    final GetItemSpec spec = new GetItemSpec()
+        .withPrimaryKey(key)
+        .withConsistentRead(true); // strictly consistent read
+    return table.getItem(spec);
+  }
+
+  @Override
+  public PathMetadata get(Path path) throws IOException {
+    return get(path, false);
+  }
+
+  @Override
+  public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    path = checkPath(path);
+    LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
+
+    try {
+      final PathMetadata meta;
+      if (path.isRoot()) {
+        // Root does not persist in the table
+        meta = new PathMetadata(makeDirStatus(username, path));
+      } else {
+        final Item item = getConsistentItem(pathToKey(path));
+        meta = itemToPathMetadata(item, username);
+        LOG.debug("Get from table {} in region {} returning for {}: {}",
+            tableName, region, path, meta);
+      }
+
+      if (wantEmptyDirectoryFlag && meta != null) {
+        final FileStatus status = meta.getFileStatus();
+        // for directory, we query its direct children to determine isEmpty bit
+        if (status.isDirectory()) {
+          final QuerySpec spec = new QuerySpec()
+              .withHashKey(pathToParentKeyAttribute(path))
+              .withConsistentRead(true)
+              .withFilterExpression(IS_DELETED + " = :false")
+              .withValueMap(deleteTrackingValueMap);
+          final ItemCollection<QueryOutcome> items = table.query(spec);
+          boolean hasChildren = items.iterator().hasNext();
+          // When this class has support for authoritative
+          // (fully-cached) directory listings, we may also be able to answer
+          // TRUE here.  Until then, we don't know if we have full listing or
+          // not, thus the UNKNOWN here:
+          meta.setIsEmptyDirectory(
+              hasChildren ? Tristate.FALSE : Tristate.UNKNOWN);
+        }
+      }
+
+      return meta;
+    } catch (AmazonClientException e) {
+      throw translateException("get", path, e);
+    }
+  }
+
+  /**
+   * Make a FileStatus object for a directory at given path.  The FileStatus
+   * only contains what S3A needs, and omits mod time since S3A uses its own
+   * implementation which returns current system time.
+   * @param owner  username of owner
+   * @param path   path to dir
+   * @return new FileStatus
+   */
+  private FileStatus makeDirStatus(String owner, Path path) {
+    return new FileStatus(0, true, 1, 0, 0, 0, null,
+            owner, null, path);
+  }
+
+  @Override
+  public DirListingMetadata listChildren(Path path) throws IOException {
+    path = checkPath(path);
+    LOG.debug("Listing table {} in region {}: {}", tableName, region, path);
+
+    // find the children in the table
+    try {
+      final QuerySpec spec = new QuerySpec()
+          .withHashKey(pathToParentKeyAttribute(path))
+          .withConsistentRead(true); // strictly consistent read
+      final ItemCollection<QueryOutcome> items = table.query(spec);
+
+      final List<PathMetadata> metas = new ArrayList<>();
+      for (Item item : items) {
+        PathMetadata meta = itemToPathMetadata(item, username);
+        metas.add(meta);
+      }
+      LOG.trace("Listing table {} in region {} for {} returning {}",
+          tableName, region, path, metas);
+
+      return (metas.isEmpty() && get(path) == null)
+          ? null
+          : new DirListingMetadata(path, metas, false);
+    } catch (AmazonClientException e) {
+      // failure, including the path not being present
+      throw translateException("listChildren", path, e);
+    }
+  }
+
+  // build the list of all parent entries.
+  Collection<PathMetadata> completeAncestry(
+      Collection<PathMetadata> pathsToCreate) {
+    // Key on path to allow fast lookup
+    Map<Path, PathMetadata> ancestry = new HashMap<>();
+
+    for (PathMetadata meta : pathsToCreate) {
+      Preconditions.checkArgument(meta != null);
+      Path path = meta.getFileStatus().getPath();
+      if (path.isRoot()) {
+        break;
+      }
+      ancestry.put(path, meta);
+      Path parent = path.getParent();
+      while (!parent.isRoot() && !ancestry.containsKey(parent)) {
+        LOG.debug("auto-create ancestor path {} for child path {}",
+            parent, path);
+        final FileStatus status = makeDirStatus(parent, username);
+        ancestry.put(parent, new PathMetadata(status, Tristate.FALSE, false));
+        parent = parent.getParent();
+      }
+    }
+    return ancestry.values();
+  }
+
+  @Override
+  public void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException {
+    if (pathsToDelete == null && pathsToCreate == null) {
+      return;
+    }
+
+    LOG.debug("Moving paths of table {} in region {}: {} paths to delete and {}"
+        + " paths to create", tableName, region,
+        pathsToDelete == null ? 0 : pathsToDelete.size(),
+        pathsToCreate == null ? 0 : pathsToCreate.size());
+    LOG.trace("move: pathsToDelete = {}, pathsToCreate = {}", pathsToDelete,
+        pathsToCreate);
+
+    // In DynamoDBMetadataStore implementation, we assume that if a path
+    // exists, all its ancestors will also exist in the table.
+    // Following code is to maintain this invariant by putting all ancestor
+    // directories of the paths to create.
+    // ancestor paths that are not explicitly added to paths to create
+    Collection<PathMetadata> newItems = new ArrayList<>();
+    if (pathsToCreate != null) {
+      newItems.addAll(completeAncestry(pathsToCreate));
+    }
+    if (pathsToDelete != null) {
+      for (Path meta : pathsToDelete) {
+        newItems.add(PathMetadata.tombstone(meta));
+      }
+    }
+
+    try {
+      processBatchWriteRequest(null, pathMetadataToItem(newItems));
+    } catch (AmazonClientException e) {
+      throw translateException("move", (String) null, e);
+    }
+  }
+
+  /**
+   * Helper method to issue a batch write request to DynamoDB.
+   *
+   * Callers of this method should catch the {@link AmazonClientException} and
+   * translate it for better error report and easier debugging.
+   * @param keysToDelete primary keys to be deleted; can be null
+   * @param itemsToPut new items to be put; can be null
+   */
+  private void processBatchWriteRequest(PrimaryKey[] keysToDelete,
+      Item[] itemsToPut) throws IOException {
+    final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length);
+    final int totalToPut = (itemsToPut == null ? 0 : itemsToPut.length);
+    int count = 0;
+    while (count < totalToDelete + totalToPut) {
+      final TableWriteItems writeItems = new TableWriteItems(tableName);
+      int numToDelete = 0;
+      if (keysToDelete != null
+          && count < totalToDelete) {
+        numToDelete = Math.min(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT,
+            totalToDelete - count);
+        writeItems.withPrimaryKeysToDelete(
+            Arrays.copyOfRange(keysToDelete, count, count + numToDelete));
+        count += numToDelete;
+      }
+
+      if (numToDelete < S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT
+          && itemsToPut != null
+          && count < totalToDelete + totalToPut) {
+        final int numToPut = Math.min(
+            S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT - numToDelete,
+            totalToDelete + totalToPut - count);
+        final int index = count - totalToDelete;
+        writeItems.withItemsToPut(
+            Arrays.copyOfRange(itemsToPut, index, index + numToPut));
+        count += numToPut;
+      }
+
+      BatchWriteItemOutcome res = dynamoDB.batchWriteItem(writeItems);
+      // Check for unprocessed keys in case of exceeding provisioned throughput
+      Map<String, List<WriteRequest>> unprocessed = res.getUnprocessedItems();
+      int retryCount = 0;
+      while (unprocessed.size() > 0) {
+        retryBackoff(retryCount++);
+        res = dynamoDB.batchWriteItemUnprocessed(unprocessed);
+        unprocessed = res.getUnprocessedItems();
+      }
+    }
+  }
+
+  /**
+   * Put the current thread to sleep to implement exponential backoff
+   * depending on retryCount.  If max retries are exceeded, throws an
+   * exception instead.
+   * @param retryCount number of retries so far
+   * @throws IOException when max retryCount is exceeded.
+   */
+  private void retryBackoff(int retryCount) throws IOException {
+    try {
+      // Our RetryPolicy ignores everything but retryCount here.
+      RetryPolicy.RetryAction action = dataAccessRetryPolicy.shouldRetry(null,
+          retryCount, 0, true);
+      if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+        throw new IOException(
+            String.format("Max retries exceeded (%d) for DynamoDB",
+                retryCount));
+      } else {
+        LOG.debug("Sleeping {} msec before next retry", action.delayMillis);
+        Thread.sleep(action.delayMillis);
+      }
+    } catch (Exception e) {
+      throw new IOException("Unexpected exception", e);
+    }
+  }
+
+  @Override
+  public void put(PathMetadata meta) throws IOException {
+    // For a deeply nested path, this method will automatically create the full
+    // ancestry and save respective item in DynamoDB table.
+    // So after put operation, we maintain the invariant that if a path exists,
+    // all its ancestors will also exist in the table.
+    // For performance purpose, we generate the full paths to put and use batch
+    // write item request to save the items.
+    LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
+
+    Collection<PathMetadata> wrapper = new ArrayList<>(1);
+    wrapper.add(meta);
+    put(wrapper);
+  }
+
+  @Override
+  public void put(Collection<PathMetadata> metas) throws IOException {
+    LOG.debug("Saving batch to table {} in region {}", tableName, region);
+
+    processBatchWriteRequest(null, pathMetadataToItem(completeAncestry(metas)));
+  }
+
+  /**
+   * Helper method to get full path of ancestors that are nonexistent in table.
+   */
+  private Collection<PathMetadata> fullPathsToPut(PathMetadata meta)
+      throws IOException {
+    checkPathMetadata(meta);
+    final Collection<PathMetadata> metasToPut = new ArrayList<>();
+    // root path is not persisted
+    if (!meta.getFileStatus().getPath().isRoot()) {
+      metasToPut.add(meta);
+    }
+
+    // put all its ancestors if not present; as an optimization we return at its
+    // first existent ancestor
+    Path path = meta.getFileStatus().getPath().getParent();
+    while (path != null && !path.isRoot()) {
+      final Item item = getConsistentItem(pathToKey(path));
+      if (!itemExists(item)) {
+        final FileStatus status = makeDirStatus(path, username);
+        metasToPut.add(new PathMetadata(status, Tristate.FALSE, false));
+        path = path.getParent();
+      } else {
+        break;
+      }
+    }
+    return metasToPut;
+  }
+
+  private boolean itemExists(Item item) {
+    if (item == null) {
+      return false;
+    }
+    if (item.hasAttribute(IS_DELETED) &&
+        item.getBoolean(IS_DELETED)) {
+      return false;
+    }
+    return true;
+  }
+
+  /** Create a directory FileStatus using current system time as mod time. */
+  static FileStatus makeDirStatus(Path f, String owner) {
+    return  new FileStatus(0, true, 1, 0, System.currentTimeMillis(), 0,
+        null, owner, owner, f);
+  }
+
+  @Override
+  public void put(DirListingMetadata meta) throws IOException {
+    LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
+
+    // directory path
+    PathMetadata p = new PathMetadata(makeDirStatus(meta.getPath(), username),
+        meta.isEmpty(), false);
+
+    // First add any missing ancestors...
+    final Collection<PathMetadata> metasToPut = fullPathsToPut(p);
+
+    // next add all children of the directory
+    metasToPut.addAll(meta.getListing());
+
+    try {
+      processBatchWriteRequest(null, pathMetadataToItem(metasToPut));
+    } catch (AmazonClientException e) {
+      throw translateException("put", (String) null, e);
+    }
+  }
+
+  @Override
+  public synchronized void close() {
+    if (instrumentation != null) {
+      instrumentation.storeClosed();
+    }
+    if (dynamoDB != null) {
+      LOG.debug("Shutting down {}", this);
+      dynamoDB.shutdown();
+      dynamoDB = null;
+    }
+  }
+
+  @Override
+  public void destroy() throws IOException {
+    if (table == null) {
+      LOG.info("In destroy(): no table to delete");
+      return;
+    }
+    LOG.info("Deleting DynamoDB table {} in region {}", tableName, region);
+    Preconditions.checkNotNull(dynamoDB, "Not connected to DynamoDB");
+    try {
+      table.delete();
+      table.waitForDelete();
+    } catch (ResourceNotFoundException rnfe) {
+      LOG.info("ResourceNotFoundException while deleting DynamoDB table {} in "
+              + "region {}.  This may indicate that the table does not exist, "
+              + "or has been deleted by another concurrent thread or process.",
+          tableName, region);
+    } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
+      LOG.warn("Interrupted while waiting for DynamoDB table {} being deleted",
+          tableName, ie);
+      throw new InterruptedIOException("Table " + tableName
+          + " in region " + region + " has not been deleted");
+    } catch (AmazonClientException e) {
+      throw translateException("destroy", (String) null, e);
+    }
+  }
+
+  private ItemCollection<ScanOutcome> expiredFiles(long modTime) {
+    String filterExpression = "mod_time < :mod_time";
+    String projectionExpression = "parent,child";
+    ValueMap map = new ValueMap().withLong(":mod_time", modTime);
+    return table.scan(filterExpression, projectionExpression, null, map);
+  }
+
+  @Override
+  public void prune(long modTime) throws IOException {
+    int itemCount = 0;
+    try {
+      Collection<Path> deletionBatch =
+          new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
+      int delay = conf.getInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
+          S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT);
+      for (Item item : expiredFiles(modTime)) {
+        PathMetadata md = PathMetadataDynamoDBTranslation
+            .itemToPathMetadata(item, username);
+        Path path = md.getFileStatus().getPath();
+        deletionBatch.add(path);
+        itemCount++;
+        if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) {
+          Thread.sleep(delay);
+          processBatchWriteRequest(pathToKey(deletionBatch), null);
+          deletionBatch.clear();
+        }
+      }
+      if (deletionBatch.size() > 0) {
+        Thread.sleep(delay);
+        processBatchWriteRequest(pathToKey(deletionBatch), null);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new InterruptedIOException("Pruning was interrupted");
+    }
+    LOG.info("Finished pruning {} items in batches of {}", itemCount,
+        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + '{'
+        + "region=" + region
+        + ", tableName=" + tableName
+        + '}';
+  }
+
+  /**
+   * Create a table if it does not exist and wait for it to become active.
+   *
+   * If a table with the intended name already exists, then it uses that table.
+   * Otherwise, it will automatically create the table if the config
+   * {@link org.apache.hadoop.fs.s3a.Constants#S3GUARD_DDB_TABLE_CREATE_KEY} is
+   * enabled. The DynamoDB table creation API is asynchronous.  This method wait
+   * for the table to become active after sending the creation request, so
+   * overall, this method is synchronous, and the table is guaranteed to exist
+   * after this method returns successfully.
+   *
+   * @throws IOException if table does not exist and auto-creation is disabled;
+   * or table is being deleted, or any other I/O exception occurred.
+   */
+  @VisibleForTesting
+  void initTable() throws IOException {
+    table = dynamoDB.getTable(tableName);
+    try {
+      try {
+        LOG.debug("Binding to table {}", tableName);
+        final String status = table.describe().getTableStatus();
+        switch (status) {
+        case "CREATING":
+        case "UPDATING":
+          LOG.debug("Table {} in region {} is being created/updated. This may"
+                  + " indicate that the table is being operated by another "
+                  + "concurrent thread or process. Waiting for active...",
+              tableName, region);
+          waitForTableActive(table);
+          break;
+        case "DELETING":
+          throw new FileNotFoundException("DynamoDB table "
+              + "'" + tableName + "' is being "
+              + "deleted in region " + region);
+        case "ACTIVE":
+          break;
+        default:
+          throw new IOException("Unknown DynamoDB table status " + status
+              + ": tableName='" + tableName + "', region=" + region);
+        }
+
+        final Item versionMarker = getVersionMarkerItem();
+        verifyVersionCompatibility(tableName, versionMarker);
+        Long created = extractCreationTimeFromMarker(versionMarker);
+        LOG.debug("Using existing DynamoDB table {} in region {} created {}",
+            tableName, region, (created != null) ? new Date(created) : null);
+      } catch (ResourceNotFoundException rnfe) {
+        if (conf.getBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, false)) {
+          final ProvisionedThroughput capacity = new ProvisionedThroughput(
+              conf.getLong(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY,
+                  S3GUARD_DDB_TABLE_CAPACITY_READ_DEFAULT),
+              conf.getLong(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY,
+                  S3GUARD_DDB_TABLE_CAPACITY_WRITE_DEFAULT));
+
+          createTable(capacity);
+        } else {
+          throw new FileNotFoundException("DynamoDB table "
+              + "'" + tableName + "' does not "
+              + "exist in region " + region + "; auto-creation is turned off");
+        }
+      }
+
+    } catch (AmazonClientException e) {
+      throw translateException("initTable", (String) null, e);
+    }
+  }
+
+  /**
+   * Get the version mark item in the existing DynamoDB table.
+   *
+   * As the version marker item may be created by another concurrent thread or
+   * process, we retry a limited times before we fail to get it.
+   */
+  private Item getVersionMarkerItem() throws IOException {
+    final PrimaryKey versionMarkerKey =
+        createVersionMarkerPrimaryKey(VERSION_MARKER);
+    int retryCount = 0;
+    Item versionMarker = table.getItem(versionMarkerKey);
+    while (versionMarker == null) {
+      try {
+        RetryPolicy.RetryAction action = dataAccessRetryPolicy.shouldRetry(null,
+            retryCount, 0, true);
+        if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+          break;
+        } else {
+          LOG.debug("Sleeping {} ms before next retry", action.delayMillis);
+          Thread.sleep(action.delayMillis);
+        }
+      } catch (Exception e) {
+        throw new IOException("initTable: Unexpected exception", e);
+      }
+      retryCount++;
+      versionMarker = table.getItem(versionMarkerKey);
+    }
+    return versionMarker;
+  }
+
+  /**
+   * Verify that a table version is compatible with this S3Guard client.
+   * @param tableName name of the table (for error messages)
+   * @param versionMarker the version marker retrieved from the table
+   * @throws IOException on any incompatibility
+   */
+  @VisibleForTesting
+  static void verifyVersionCompatibility(String tableName,
+      Item versionMarker) throws IOException {
+    if (versionMarker == null) {
+      LOG.warn("Table {} contains no version marker", tableName);
+      throw new IOException(E_NO_VERSION_MARKER
+      + " Table: " + tableName);
+    } else {
+      final int version = extractVersionFromMarker(versionMarker);
+      if (VERSION != version) {
+        // version mismatch. Unless/until there is support for
+        // upgrading versions, treat this as an incompatible change
+        // and fail.
+        throw new IOException(E_INCOMPATIBLE_VERSION
+            + " Table "+  tableName
+            + " Expected version " + VERSION + " actual " + version);
+      }
+    }
+  }
+
+  /**
+   * Wait for table being active.
+   * @param t table to block on.
+   * @throws IOException IO problems
+   * @throws InterruptedIOException if the wait was interrupted
+   */
+  private void waitForTableActive(Table t) throws IOException {
+    try {
+      t.waitForActive();
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted while waiting for table {} in region {} active",
+          tableName, region, e);
+      Thread.currentThread().interrupt();
+      throw (IOException) new InterruptedIOException("DynamoDB table '"
+          + tableName + "' is not active yet in region " + region).initCause(e);
+    }
+  }
+
+  /**
+   * Create a table, wait for it to become active, then add the version
+   * marker.
+   * @param capacity capacity to provision
+   * @throws IOException on any failure.
+   * @throws InterruptedIOException if the wait was interrupted
+   */
+  private void createTable(ProvisionedThroughput capacity) throws IOException {
+    try {
+      LOG.info("Creating non-existent DynamoDB table {} in region {}",
+          tableName, region);
+      table = dynamoDB.createTable(new CreateTableRequest()
+          .withTableName(tableName)
+          .withKeySchema(keySchema())
+          .withAttributeDefinitions(attributeDefinitions())
+          .withProvisionedThroughput(capacity));
+      LOG.debug("Awaiting table becoming active");
+    } catch (ResourceInUseException e) {
+      LOG.warn("ResourceInUseException while creating DynamoDB table {} "
+              + "in region {}.  This may indicate that the table was "
+              + "created by another concurrent thread or process.",
+          tableName, region);
+    }
+    waitForTableActive(table);
+    final Item marker = createVersionMarker(VERSION_MARKER, VERSION,
+        System.currentTimeMillis());
+    putItem(marker);
+  }
+
+  /**
+   * PUT a single item to the table.
+   * @param item item to put
+   * @return the outcome.
+   */
+  PutItemOutcome putItem(Item item) {
+    LOG.debug("Putting item {}", item);
+    return table.putItem(item);
+  }
+
+  /**
+   * Provision the table with given read and write capacity units.
+   */
+  void provisionTable(Long readCapacity, Long writeCapacity)
+      throws IOException {
+    final ProvisionedThroughput toProvision = new ProvisionedThroughput()
+        .withReadCapacityUnits(readCapacity)
+        .withWriteCapacityUnits(writeCapacity);
+    try {
+      final ProvisionedThroughputDescription p =
+          table.updateTable(toProvision).getProvisionedThroughput();
+      LOG.info("Provision table {} in region {}: readCapacityUnits={}, "
+              + "writeCapacityUnits={}",
+          tableName, region, p.getReadCapacityUnits(),
+          p.getWriteCapacityUnits());
+    } catch (AmazonClientException e) {
+      throw translateException("provisionTable", (String) null, e);
+    }
+  }
+
+  Table getTable() {
+    return table;
+  }
+
+  String getRegion() {
+    return region;
+  }
+
+  @VisibleForTesting
+  DynamoDB getDynamoDB() {
+    return dynamoDB;
+  }
+
+  /**
+   * Validates a path object; it must be absolute, and contain a host
+   * (bucket) component.
+   */
+  private Path checkPath(Path path) {
+    Preconditions.checkNotNull(path);
+    Preconditions.checkArgument(path.isAbsolute(), "Path %s is not absolute",
+        path);
+    URI uri = path.toUri();
+    Preconditions.checkNotNull(uri.getScheme(), "Path %s missing scheme", path);
+    Preconditions.checkArgument(uri.getScheme().equals(Constants.FS_S3A),
+        "Path %s scheme must be %s", path, Constants.FS_S3A);
+    Preconditions.checkArgument(!StringUtils.isEmpty(uri.getHost()), "Path %s" +
+        " is missing bucket.", path);
+    return path;
+  }
+
+  /**
+   * Validates a path meta-data object.
+   */
+  private static void checkPathMetadata(PathMetadata meta) {
+    Preconditions.checkNotNull(meta);
+    Preconditions.checkNotNull(meta.getFileStatus());
+    Preconditions.checkNotNull(meta.getFileStatus().getPath());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
new file mode 100644
index 0000000..1ef8b0d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
@@ -0,0 +1,435 @@
+/*
+ * 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.s3a.s3guard;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * This is a local, in-memory, implementation of MetadataStore.
+ * This is <i>not</i> a coherent cache across processes.  It is only
+ * locally-coherent.
+ *
+ * The purpose of this is for unit and integration testing.
+ * It could also be used to accelerate local-only operations where only one
+ * process is operating on a given object store, or multiple processes are
+ * accessing a read-only storage bucket.
+ *
+ * This MetadataStore does not enforce filesystem rules such as disallowing
+ * non-recursive removal of non-empty directories.  It is assumed the caller
+ * already has to perform these sorts of checks.
+ */
+public class LocalMetadataStore implements MetadataStore {
+
+  public static final Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
+  // TODO HADOOP-13649: use time instead of capacity for eviction.
+  public static final int DEFAULT_MAX_RECORDS = 128;
+
+  /**
+   * Maximum number of records.
+   */
+  public static final String CONF_MAX_RECORDS =
+      "fs.metadatastore.local.max_records";
+
+  /** Contains directories and files. */
+  private LruHashMap<Path, PathMetadata> fileHash;
+
+  /** Contains directory listings. */
+  private LruHashMap<Path, DirListingMetadata> dirHash;
+
+  private FileSystem fs;
+  /* Null iff this FS does not have an associated URI host. */
+  private String uriHost;
+
+  @Override
+  public void initialize(FileSystem fileSystem) throws IOException {
+    Preconditions.checkNotNull(fileSystem);
+    fs = fileSystem;
+    URI fsURI = fs.getUri();
+    uriHost = fsURI.getHost();
+    if (uriHost != null && uriHost.equals("")) {
+      uriHost = null;
+    }
+
+    initialize(fs.getConf());
+  }
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+    Preconditions.checkNotNull(conf);
+    int maxRecords = conf.getInt(CONF_MAX_RECORDS, DEFAULT_MAX_RECORDS);
+    if (maxRecords < 4) {
+      maxRecords = 4;
+    }
+    // Start w/ less than max capacity.  Space / time trade off.
+    fileHash = new LruHashMap<>(maxRecords/2, maxRecords);
+    dirHash = new LruHashMap<>(maxRecords/4, maxRecords);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "LocalMetadataStore{");
+    sb.append(", uriHost='").append(uriHost).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public void delete(Path p) throws IOException {
+    doDelete(p, false, true);
+  }
+
+  @Override
+  public void forgetMetadata(Path p) throws IOException {
+    doDelete(p, false, false);
+  }
+
+  @Override
+  public void deleteSubtree(Path path) throws IOException {
+    doDelete(path, true, true);
+  }
+
+  private synchronized void doDelete(Path p, boolean recursive, boolean
+      tombstone) {
+
+    Path path = standardize(p);
+
+    // Delete entry from file cache, then from cached parent directory, if any
+
+    deleteHashEntries(path, tombstone);
+
+    if (recursive) {
+      // Remove all entries that have this dir as path prefix.
+      deleteHashByAncestor(path, dirHash, tombstone);
+      deleteHashByAncestor(path, fileHash, tombstone);
+    }
+  }
+
+  @Override
+  public synchronized PathMetadata get(Path p) throws IOException {
+    return get(p, false);
+  }
+
+  @Override
+  public PathMetadata get(Path p, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    Path path = standardize(p);
+    synchronized (this) {
+      PathMetadata m = fileHash.mruGet(path);
+
+      if (wantEmptyDirectoryFlag && m != null &&
+          m.getFileStatus().isDirectory()) {
+        m.setIsEmptyDirectory(isEmptyDirectory(p));
+      }
+
+      LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
+      return m;
+    }
+  }
+
+  /**
+   * Determine if directory is empty.
+   * Call with lock held.
+   * @param p a Path, already filtered through standardize()
+   * @return TRUE / FALSE if known empty / not-empty, UNKNOWN otherwise.
+   */
+  private Tristate isEmptyDirectory(Path p) {
+    DirListingMetadata dirMeta = dirHash.get(p);
+    return dirMeta.withoutTombstones().isEmpty();
+  }
+
+  @Override
+  public synchronized DirListingMetadata listChildren(Path p) throws
+      IOException {
+    Path path = standardize(p);
+    DirListingMetadata listing = dirHash.mruGet(path);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("listChildren({}) -> {}", path,
+          listing == null ? "null" : listing.prettyPrint());
+    }
+    // Make a copy so callers can mutate without affecting our state
+    return listing == null ? null : new DirListingMetadata(listing);
+  }
+
+  @Override
+  public void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException {
+
+    Preconditions.checkNotNull(pathsToDelete, "pathsToDelete is null");
+    Preconditions.checkNotNull(pathsToCreate, "pathsToCreate is null");
+    Preconditions.checkArgument(pathsToDelete.size() == pathsToCreate.size(),
+        "Must supply same number of paths to delete/create.");
+
+    // I feel dirty for using reentrant lock. :-|
+    synchronized (this) {
+
+      // 1. Delete pathsToDelete
+      for (Path meta : pathsToDelete) {
+        LOG.debug("move: deleting metadata {}", meta);
+        delete(meta);
+      }
+
+      // 2. Create new destination path metadata
+      for (PathMetadata meta : pathsToCreate) {
+        LOG.debug("move: adding metadata {}", meta);
+        put(meta);
+      }
+
+      // 3. We now know full contents of all dirs in destination subtree
+      for (PathMetadata meta : pathsToCreate) {
+        FileStatus status = meta.getFileStatus();
+        if (status == null || status.isDirectory()) {
+          continue;
+        }
+        DirListingMetadata dir = listChildren(status.getPath());
+        if (dir != null) {  // could be evicted already
+          dir.setAuthoritative(true);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void put(PathMetadata meta) throws IOException {
+
+    Preconditions.checkNotNull(meta);
+    FileStatus status = meta.getFileStatus();
+    Path path = standardize(status.getPath());
+    synchronized (this) {
+
+      /* Add entry for this file. */
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("put {} -> {}", path, meta.prettyPrint());
+      }
+      fileHash.put(path, meta);
+
+      /* Directory case:
+       * We also make sure we have an entry in the dirHash, so subsequent
+       * listStatus(path) at least see the directory.
+       *
+       * If we had a boolean flag argument "isNew", we would know whether this
+       * is an existing directory the client discovered via getFileStatus(),
+       * or if it is a newly-created directory.  In the latter case, we would
+       * be able to mark the directory as authoritative (fully-cached),
+       * saving round trips to underlying store for subsequent listStatus()
+       */
+
+      if (status.isDirectory()) {
+        DirListingMetadata dir = dirHash.mruGet(path);
+        if (dir == null) {
+          dirHash.put(path, new DirListingMetadata(path, DirListingMetadata
+              .EMPTY_DIR, false));
+        }
+      }
+
+      /* Update cached parent dir. */
+      Path parentPath = path.getParent();
+      if (parentPath != null) {
+        DirListingMetadata parent = dirHash.mruGet(parentPath);
+        if (parent == null) {
+        /* Track this new file's listing in parent.  Parent is not
+         * authoritative, since there may be other items in it we don't know
+         * about. */
+          parent = new DirListingMetadata(parentPath,
+              DirListingMetadata.EMPTY_DIR, false);
+          dirHash.put(parentPath, parent);
+        }
+        parent.put(status);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void put(DirListingMetadata meta) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("put dirMeta {}", meta.prettyPrint());
+    }
+    dirHash.put(standardize(meta.getPath()), meta);
+  }
+
+  public synchronized void put(Collection<PathMetadata> metas) throws
+      IOException {
+    for (PathMetadata meta : metas) {
+      put(meta);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  @Override
+  public void destroy() throws IOException {
+    if (dirHash != null) {
+      dirHash.clear();
+    }
+  }
+
+  @Override
+  public synchronized void prune(long modTime) throws IOException {
+    Iterator<Map.Entry<Path, PathMetadata>> files =
+        fileHash.entrySet().iterator();
+    while (files.hasNext()) {
+      Map.Entry<Path, PathMetadata> entry = files.next();
+      if (expired(entry.getValue().getFileStatus(), modTime)) {
+        files.remove();
+      }
+    }
+    Iterator<Map.Entry<Path, DirListingMetadata>> dirs =
+        dirHash.entrySet().iterator();
+    while (dirs.hasNext()) {
+      Map.Entry<Path, DirListingMetadata> entry = dirs.next();
+      Path path = entry.getKey();
+      DirListingMetadata metadata = entry.getValue();
+      Collection<PathMetadata> oldChildren = metadata.getListing();
+      Collection<PathMetadata> newChildren = new LinkedList<>();
+
+      for (PathMetadata child : oldChildren) {
+        FileStatus status = child.getFileStatus();
+        if (!expired(status, modTime)) {
+          newChildren.add(child);
+        }
+      }
+      if (newChildren.size() != oldChildren.size()) {
+        dirHash.put(path, new DirListingMetadata(path, newChildren, false));
+        if (!path.isRoot()) {
+          DirListingMetadata parent = dirHash.get(path.getParent());
+          if (parent != null) {
+            parent.setAuthoritative(false);
+          }
+        }
+      }
+    }
+  }
+
+  private boolean expired(FileStatus status, long expiry) {
+    // Note: S3 doesn't track modification time on directories, so for
+    // consistency with the DynamoDB implementation we ignore that here
+    return status.getModificationTime() < expiry && !status.isDirectory();
+  }
+
+  @VisibleForTesting
+  static <T> void deleteHashByAncestor(Path ancestor, Map<Path, T> hash,
+                                       boolean tombstone) {
+    for (Iterator<Map.Entry<Path, T>> it = hash.entrySet().iterator();
+         it.hasNext();) {
+      Map.Entry<Path, T> entry = it.next();
+      Path f = entry.getKey();
+      T meta = entry.getValue();
+      if (isAncestorOf(ancestor, f)) {
+        if (tombstone) {
+          if (meta instanceof PathMetadata) {
+            entry.setValue((T) PathMetadata.tombstone(f));
+          } else if (meta instanceof DirListingMetadata) {
+            it.remove();
+          } else {
+            throw new IllegalStateException("Unknown type in hash");
+          }
+        } else {
+          it.remove();
+        }
+      }
+    }
+  }
+
+  /**
+   * @return true iff 'ancestor' is ancestor dir in path 'f'.
+   * All paths here are absolute.  Dir does not count as its own ancestor.
+   */
+  private static boolean isAncestorOf(Path ancestor, Path f) {
+    String aStr = ancestor.toString();
+    if (!ancestor.isRoot()) {
+      aStr += "/";
+    }
+    String fStr = f.toString();
+    return (fStr.startsWith(aStr));
+  }
+
+  /**
+   * Update fileHash and dirHash to reflect deletion of file 'f'.  Call with
+   * lock held.
+   */
+  private void deleteHashEntries(Path path, boolean tombstone) {
+
+    // Remove target file/dir
+    LOG.debug("delete file entry for {}", path);
+    if (tombstone) {
+      fileHash.put(path, PathMetadata.tombstone(path));
+    } else {
+      fileHash.remove(path);
+    }
+
+    // Update this and parent dir listing, if any
+
+    /* If this path is a dir, remove its listing */
+    LOG.debug("removing listing of {}", path);
+
+    dirHash.remove(path);
+
+    /* Remove this path from parent's dir listing */
+    Path parent = path.getParent();
+    if (parent != null) {
+      DirListingMetadata dir = dirHash.get(parent);
+      if (dir != null) {
+        LOG.debug("removing parent's entry for {} ", path);
+        if (tombstone) {
+          dir.markDeleted(path);
+        } else {
+          dir.remove(path);
+        }
+      }
+    }
+  }
+
+  /**
+   * Return a "standardized" version of a path so we always have a consistent
+   * hash value.  Also asserts the path is absolute, and contains host
+   * component.
+   * @param p input Path
+   * @return standardized version of Path, suitable for hash key
+   */
+  private Path standardize(Path p) {
+    Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
+    URI uri = p.toUri();
+    if (uriHost != null) {
+      Preconditions.checkArgument(!isEmpty(uri.getHost()));
+    }
+    return p;
+  }
+
+  private static boolean isEmpty(String s) {
+    return (s == null || s.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java
new file mode 100644
index 0000000..e355095
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java
@@ -0,0 +1,50 @@
+/*
+ *  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.s3a.s3guard;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * LinkedHashMap that implements a maximum size and LRU eviction policy.
+ */
+public class LruHashMap<K, V> extends LinkedHashMap<K, V> {
+  private final int maxSize;
+  public LruHashMap(int initialCapacity, int maxSize) {
+    super(initialCapacity);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+    return size() > maxSize;
+  }
+
+  /**
+   * get() plus side-effect of making the element Most Recently Used.
+   * @param key lookup key
+   * @return value
+   */
+
+  public V mruGet(K key) {
+    V val = remove(key);
+    if (val != null) {
+      put(key, val);
+    }
+    return val;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
new file mode 100644
index 0000000..dd8077b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
@@ -0,0 +1,221 @@
+/**
+ * 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.s3a.s3guard;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * {@code MetadataStore} defines the set of operations that any metadata store
+ * implementation must provide.  Note that all {@link Path} objects provided
+ * to methods must be absolute, not relative paths.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface MetadataStore extends Closeable {
+
+  /**
+   * Performs one-time initialization of the metadata store.
+   *
+   * @param fs {@code FileSystem} associated with the MetadataStore
+   * @throws IOException if there is an error
+   */
+  void initialize(FileSystem fs) throws IOException;
+
+  /**
+   * Performs one-time initialization of the metadata store via configuration.
+   * @see #initialize(FileSystem)
+   * @param conf Configuration.
+   * @throws IOException if there is an error
+   */
+  void initialize(Configuration conf) throws IOException;
+
+  /**
+   * Deletes exactly one path, leaving a tombstone to prevent lingering,
+   * inconsistent copies of it from being listed.
+   *
+   * @param path the path to delete
+   * @throws IOException if there is an error
+   */
+  void delete(Path path) throws IOException;
+
+  /**
+   * Removes the record of exactly one path.  Does not leave a tombstone (see
+   * {@link MetadataStore#delete(Path)}. It is currently intended for testing
+   * only, and a need to use it as part of normal FileSystem usage is not
+   * anticipated.
+   *
+   * @param path the path to delete
+   * @throws IOException if there is an error
+   */
+  @VisibleForTesting
+  void forgetMetadata(Path path) throws IOException;
+
+  /**
+   * Deletes the entire sub-tree rooted at the given path, leaving tombstones
+   * to prevent lingering, inconsistent copies of it from being listed.
+   *
+   * In addition to affecting future calls to {@link #get(Path)},
+   * implementations must also update any stored {@code DirListingMetadata}
+   * objects which track the parent of this file.
+   *
+   * @param path the root of the sub-tree to delete
+   * @throws IOException if there is an error
+   */
+  void deleteSubtree(Path path) throws IOException;
+
+  /**
+   * Gets metadata for a path.
+   *
+   * @param path the path to get
+   * @return metadata for {@code path}, {@code null} if not found
+   * @throws IOException if there is an error
+   */
+  PathMetadata get(Path path) throws IOException;
+
+  /**
+   * Gets metadata for a path.  Alternate method that includes a hint
+   * whether or not the MetadataStore should do work to compute the value for
+   * {@link PathMetadata#isEmptyDirectory()}.  Since determining emptiness
+   * may be an expensive operation, this can save wasted work.
+   *
+   * @param path the path to get
+   * @param wantEmptyDirectoryFlag Set to true to give a hint to the
+   *   MetadataStore that it should try to compute the empty directory flag.
+   * @return metadata for {@code path}, {@code null} if not found
+   * @throws IOException if there is an error
+   */
+  PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException;
+
+  /**
+   * Lists metadata for all direct children of a path.
+   *
+   * @param path the path to list
+   * @return metadata for all direct children of {@code path} which are being
+   *     tracked by the MetadataStore, or {@code null} if the path was not found
+   *     in the MetadataStore.
+   * @throws IOException if there is an error
+   */
+  DirListingMetadata listChildren(Path path) throws IOException;
+
+  /**
+   * Record the effects of a {@link FileSystem#rename(Path, Path)} in the
+   * MetadataStore.  Clients provide explicit enumeration of the affected
+   * paths (recursively), before and after the rename.
+   *
+   * This operation is not atomic, unless specific implementations claim
+   * otherwise.
+   *
+   * On the need to provide an enumeration of directory trees instead of just
+   * source and destination paths:
+   * Since a MetadataStore does not have to track all metadata for the
+   * underlying storage system, and a new MetadataStore may be created on an
+   * existing underlying filesystem, this move() may be the first time the
+   * MetadataStore sees the affected paths.  Therefore, simply providing src
+   * and destination paths may not be enough to record the deletions (under
+   * src path) and creations (at destination) that are happening during the
+   * rename().
+   *
+   * @param pathsToDelete Collection of all paths that were removed from the
+   *                      source directory tree of the move.
+   * @param pathsToCreate Collection of all PathMetadata for the new paths
+   *                      that were created at the destination of the rename
+   *                      ().
+   * @throws IOException if there is an error
+   */
+  void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException;
+
+  /**
+   * Saves metadata for exactly one path.
+   *
+   * Implementations may pre-create all the path's ancestors automatically.
+   * Implementations must update any {@code DirListingMetadata} objects which
+   * track the immediate parent of this file.
+   *
+   * @param meta the metadata to save
+   * @throws IOException if there is an error
+   */
+  void put(PathMetadata meta) throws IOException;
+
+  /**
+   * Saves metadata for any number of paths.
+   *
+   * Semantics are otherwise the same as single-path puts.
+   *
+   * @param metas the metadata to save
+   * @throws IOException if there is an error
+   */
+  void put(Collection<PathMetadata> metas) throws IOException;
+
+  /**
+   * Save directory listing metadata. Callers may save a partial directory
+   * listing for a given path, or may store a complete and authoritative copy
+   * of the directory listing.  {@code MetadataStore} implementations may
+   * subsequently keep track of all modifications to the directory contents at
+   * this path, and return authoritative results from subsequent calls to
+   * {@link #listChildren(Path)}. See {@link DirListingMetadata}.
+   *
+   * Any authoritative results returned are only authoritative for the scope
+   * of the {@code MetadataStore}:  A per-process {@code MetadataStore}, for
+   * example, would only show results visible to that process, potentially
+   * missing metadata updates (create, delete) made to the same path by
+   * another process.
+   *
+   * @param meta Directory listing metadata.
+   * @throws IOException if there is an error
+   */
+  void put(DirListingMetadata meta) throws IOException;
+
+  /**
+   * Destroy all resources associated with the metadata store.
+   *
+   * The destroyed resources can be DynamoDB tables, MySQL databases/tables, or
+   * HDFS directories. Any operations after calling this method may possibly
+   * fail.
+   *
+   * This operation is idempotent.
+   *
+   * @throws IOException if there is an error
+   */
+  void destroy() throws IOException;
+
+  /**
+   * Clear any metadata older than a specified time from the repository.
+   * Implementations MUST clear file metadata, and MAY clear directory metadata
+   * (s3a itself does not track modification time for directories).
+   * Implementations may also choose to throw UnsupportedOperationException
+   * istead. Note that modification times should be in UTC, as returned by
+   * System.currentTimeMillis at the time of modification.
+   *
+   * @param modTime Oldest modification time to allow
+   * @throws IOException if there is an error
+   * @throws UnsupportedOperationException if not implemented
+   */
+  void prune(long modTime) throws IOException, UnsupportedOperationException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
new file mode 100644
index 0000000..378d109
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
@@ -0,0 +1,169 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * {@code MetadataStoreListFilesIterator} is a {@link RemoteIterator} that
+ * is similar to {@code DescendantsIterator} but does not return directories
+ * that have (or may have) children, and will also provide access to the set of
+ * tombstones to allow recently deleted S3 objects to be filtered out from a
+ * corresponding request.  In other words, it returns tombstones and the same
+ * set of objects that should exist in S3: empty directories, and files, and not
+ * other directories whose existence is inferred therefrom.
+ *
+ * For example, assume the consistent store contains metadata representing this
+ * file system structure:
+ *
+ * <pre>
+ * /dir1
+ * |-- dir2
+ * |   |-- file1
+ * |   `-- file2
+ * `-- dir3
+ *     |-- dir4
+ *     |   `-- file3
+ *     |-- dir5
+ *     |   `-- file4
+ *     `-- dir6
+ * </pre>
+ *
+ * Consider this code sample:
+ * <pre>
+ * final PathMetadata dir1 = get(new Path("/dir1"));
+ * for (MetadataStoreListFilesIterator files =
+ *     new MetadataStoreListFilesIterator(dir1); files.hasNext(); ) {
+ *   final FileStatus status = files.next().getFileStatus();
+ *   System.out.printf("%s %s%n", status.isDirectory() ? 'D' : 'F',
+ *       status.getPath());
+ * }
+ * </pre>
+ *
+ * The output is:
+ * <pre>
+ * F /dir1/dir2/file1
+ * F /dir1/dir2/file2
+ * F /dir1/dir3/dir4/file3
+ * F /dir1/dir3/dir5/file4
+ * D /dir1/dir3/dir6
+ * </pre>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MetadataStoreListFilesIterator implements
+    RemoteIterator<FileStatus> {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      MetadataStoreListFilesIterator.class);
+
+  private final boolean allowAuthoritative;
+  private final MetadataStore metadataStore;
+  private final Set<Path> tombstones = new HashSet<>();
+  private Iterator<FileStatus> leafNodesIterator = null;
+
+  public MetadataStoreListFilesIterator(MetadataStore ms, PathMetadata meta,
+      boolean allowAuthoritative) throws IOException {
+    Preconditions.checkNotNull(ms);
+    this.metadataStore = ms;
+    this.allowAuthoritative = allowAuthoritative;
+    prefetch(meta);
+  }
+
+  private void prefetch(PathMetadata meta) throws IOException {
+    final Queue<PathMetadata> queue = new LinkedList<>();
+    final Collection<FileStatus> leafNodes = new ArrayList<>();
+
+    if (meta != null) {
+      final Path path = meta.getFileStatus().getPath();
+      if (path.isRoot()) {
+        DirListingMetadata rootListing = metadataStore.listChildren(path);
+        if (rootListing != null) {
+          tombstones.addAll(rootListing.listTombstones());
+          queue.addAll(rootListing.withoutTombstones().getListing());
+        }
+      } else {
+        queue.add(meta);
+      }
+    }
+
+    while(!queue.isEmpty()) {
+      PathMetadata nextMetadata = queue.poll();
+      FileStatus nextStatus = nextMetadata.getFileStatus();
+      if (nextStatus.isFile()) {
+        // All files are leaf nodes by definition
+        leafNodes.add(nextStatus);
+        continue;
+      }
+      if (nextStatus.isDirectory()) {
+        final Path path = nextStatus.getPath();
+        DirListingMetadata children = metadataStore.listChildren(path);
+        if (children != null) {
+          tombstones.addAll(children.listTombstones());
+          Collection<PathMetadata> liveChildren =
+              children.withoutTombstones().getListing();
+          if (!liveChildren.isEmpty()) {
+            // If it's a directory, has children, not all deleted, then we
+            // add the children to the queue and move on to the next node
+            queue.addAll(liveChildren);
+            continue;
+          } else if (allowAuthoritative && children.isAuthoritative()) {
+            leafNodes.add(nextStatus);
+          }
+        }
+      }
+      // Directories that *might* be empty are ignored for now, since we
+      // cannot confirm that they are empty without incurring other costs.
+      // Users of this class can still discover empty directories via S3's
+      // fake directories, subject to the same consistency semantics as before.
+      // The only other possibility is a symlink, which is unsupported on S3A.
+    }
+    leafNodesIterator = leafNodes.iterator();
+  }
+
+  @Override
+  public boolean hasNext() {
+    return leafNodesIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() {
+    return leafNodesIterator.next();
+  }
+
+  public Set<Path> listTombstones() {
+    return tombstones;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
new file mode 100644
index 0000000..08ae89e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.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.fs.s3a.s3guard;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * A no-op implementation of MetadataStore.  Clients that use this
+ * implementation should behave the same as they would without any
+ * MetadataStore.
+ */
+public class NullMetadataStore implements MetadataStore {
+
+  @Override
+  public void initialize(FileSystem fs) throws IOException {
+  }
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void delete(Path path) throws IOException {
+  }
+
+  @Override
+  public void forgetMetadata(Path path) throws IOException {
+  }
+
+  @Override
+  public void deleteSubtree(Path path) throws IOException {
+  }
+
+  @Override
+  public PathMetadata get(Path path) throws IOException {
+    return null;
+  }
+
+  @Override
+  public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public DirListingMetadata listChildren(Path path) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException {
+  }
+
+  @Override
+  public void put(PathMetadata meta) throws IOException {
+  }
+
+  @Override
+  public void put(Collection<PathMetadata> meta) throws IOException {
+  }
+
+  @Override
+  public void put(DirListingMetadata meta) throws IOException {
+  }
+
+  @Override
+  public void destroy() throws IOException {
+  }
+
+  @Override
+  public void prune(long modTime) {
+  }
+
+  @Override
+  public String toString() {
+    return "NullMetadataStore";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
new file mode 100644
index 0000000..2a0219e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
@@ -0,0 +1,143 @@
+/*
+ * 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.s3a.s3guard;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * {@code PathMetadata} models path metadata stored in the
+ * {@link MetadataStore}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class PathMetadata {
+
+  private final FileStatus fileStatus;
+  private Tristate isEmptyDirectory;
+  private boolean isDeleted;
+
+  /**
+   * Create a tombstone from the current time.
+   * @param path path to tombstone
+   * @return the entry.
+   */
+  public static PathMetadata tombstone(Path path) {
+    long now = System.currentTimeMillis();
+    FileStatus status = new FileStatus(0, false, 0, 0, now, path);
+    return new PathMetadata(status, Tristate.UNKNOWN, true);
+  }
+
+  /**
+   * Creates a new {@code PathMetadata} containing given {@code FileStatus}.
+   * @param fileStatus file status containing an absolute path.
+   */
+  public PathMetadata(FileStatus fileStatus) {
+    this(fileStatus, Tristate.UNKNOWN);
+  }
+
+  public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir) {
+    this(fileStatus, isEmptyDir, false);
+  }
+
+  public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir, boolean
+      isDeleted) {
+    Preconditions.checkNotNull(fileStatus, "fileStatus must be non-null");
+    Preconditions.checkNotNull(fileStatus.getPath(), "fileStatus path must be" +
+        " non-null");
+    Preconditions.checkArgument(fileStatus.getPath().isAbsolute(), "path must" +
+        " be absolute");
+    this.fileStatus = fileStatus;
+    this.isEmptyDirectory = isEmptyDir;
+    this.isDeleted = isDeleted;
+  }
+
+  /**
+   * @return {@code FileStatus} contained in this {@code PathMetadata}.
+   */
+  public final FileStatus getFileStatus() {
+    return fileStatus;
+  }
+
+  /**
+   * Query if a directory is empty.
+   * @return Tristate.TRUE if this is known to be an empty directory,
+   * Tristate.FALSE if known to not be empty, and Tristate.UNKNOWN if the
+   * MetadataStore does have enough information to determine either way.
+   */
+  public Tristate isEmptyDirectory() {
+    return isEmptyDirectory;
+  }
+
+  void setIsEmptyDirectory(Tristate isEmptyDirectory) {
+    this.isEmptyDirectory = isEmptyDirectory;
+  }
+
+  public boolean isDeleted() {
+    return isDeleted;
+  }
+
+  void setIsDeleted(boolean isDeleted) {
+    this.isDeleted = isDeleted;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof PathMetadata)) {
+      return false;
+    }
+    return this.fileStatus.equals(((PathMetadata)o).fileStatus);
+  }
+
+  @Override
+  public int hashCode() {
+    return fileStatus.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "PathMetadata{" +
+        "fileStatus=" + fileStatus +
+        "; isEmptyDirectory=" + isEmptyDirectory +
+        "; isDeleted=" + isDeleted +
+        '}';
+  }
+
+  /**
+   * Log contents to supplied StringBuilder in a pretty fashion.
+   * @param sb target StringBuilder
+   */
+  public void prettyPrint(StringBuilder sb) {
+    sb.append(String.format("%-5s %-20s %-7d %-8s %-6s",
+        fileStatus.isDirectory() ? "dir" : "file",
+        fileStatus.getPath().toString(), fileStatus.getLen(),
+        isEmptyDirectory.name(), isDeleted));
+    sb.append(fileStatus);
+  }
+
+  public String prettyPrint() {
+    StringBuilder sb = new StringBuilder();
+    prettyPrint(sb);
+    return sb.toString();
+  }
+}


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


[21/48] hadoop git commit: Preparing for 3.1.0 development

Posted by in...@apache.org.
Preparing for 3.1.0 development


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

Branch: refs/heads/HDFS-10467
Commit: 0d419c984f4bde451f45f1c135f990f499375f01
Parents: 063b6d0
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Sep 1 11:53:48 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Sep 1 11:53:48 2017 -0700

----------------------------------------------------------------------
 hadoop-assemblies/pom.xml                                        | 4 ++--
 hadoop-build-tools/pom.xml                                       | 2 +-
 hadoop-client-modules/hadoop-client-api/pom.xml                  | 4 ++--
 hadoop-client-modules/hadoop-client-check-invariants/pom.xml     | 4 ++--
 .../hadoop-client-check-test-invariants/pom.xml                  | 4 ++--
 hadoop-client-modules/hadoop-client-integration-tests/pom.xml    | 4 ++--
 hadoop-client-modules/hadoop-client-minicluster/pom.xml          | 4 ++--
 hadoop-client-modules/hadoop-client-runtime/pom.xml              | 4 ++--
 hadoop-client-modules/hadoop-client/pom.xml                      | 4 ++--
 hadoop-client-modules/pom.xml                                    | 2 +-
 hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml        | 4 ++--
 hadoop-cloud-storage-project/pom.xml                             | 4 ++--
 hadoop-common-project/hadoop-annotations/pom.xml                 | 4 ++--
 hadoop-common-project/hadoop-auth-examples/pom.xml               | 4 ++--
 hadoop-common-project/hadoop-auth/pom.xml                        | 4 ++--
 hadoop-common-project/hadoop-common/pom.xml                      | 4 ++--
 hadoop-common-project/hadoop-kms/pom.xml                         | 4 ++--
 hadoop-common-project/hadoop-minikdc/pom.xml                     | 4 ++--
 hadoop-common-project/hadoop-nfs/pom.xml                         | 4 ++--
 hadoop-common-project/pom.xml                                    | 4 ++--
 hadoop-dist/pom.xml                                              | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml                   | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml                   | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml            | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml                      | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs/pom.xml                          | 4 ++--
 hadoop-hdfs-project/pom.xml                                      | 4 ++--
 .../hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml  | 4 ++--
 .../hadoop-mapreduce-client-common/pom.xml                       | 4 ++--
 .../hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml | 4 ++--
 .../hadoop-mapreduce-client-hs-plugins/pom.xml                   | 4 ++--
 .../hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml   | 4 ++--
 .../hadoop-mapreduce-client-jobclient/pom.xml                    | 4 ++--
 .../hadoop-mapreduce-client-nativetask/pom.xml                   | 4 ++--
 .../hadoop-mapreduce-client-shuffle/pom.xml                      | 4 ++--
 hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml         | 4 ++--
 hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml       | 4 ++--
 hadoop-mapreduce-project/pom.xml                                 | 4 ++--
 hadoop-maven-plugins/pom.xml                                     | 2 +-
 hadoop-minicluster/pom.xml                                       | 4 ++--
 hadoop-project-dist/pom.xml                                      | 4 ++--
 hadoop-project/pom.xml                                           | 4 ++--
 hadoop-tools/hadoop-aliyun/pom.xml                               | 2 +-
 hadoop-tools/hadoop-archive-logs/pom.xml                         | 4 ++--
 hadoop-tools/hadoop-archives/pom.xml                             | 4 ++--
 hadoop-tools/hadoop-aws/pom.xml                                  | 4 ++--
 hadoop-tools/hadoop-azure-datalake/pom.xml                       | 2 +-
 hadoop-tools/hadoop-azure/pom.xml                                | 2 +-
 hadoop-tools/hadoop-datajoin/pom.xml                             | 4 ++--
 hadoop-tools/hadoop-distcp/pom.xml                               | 4 ++--
 hadoop-tools/hadoop-extras/pom.xml                               | 4 ++--
 hadoop-tools/hadoop-gridmix/pom.xml                              | 4 ++--
 hadoop-tools/hadoop-kafka/pom.xml                                | 4 ++--
 hadoop-tools/hadoop-openstack/pom.xml                            | 4 ++--
 hadoop-tools/hadoop-pipes/pom.xml                                | 4 ++--
 hadoop-tools/hadoop-rumen/pom.xml                                | 4 ++--
 hadoop-tools/hadoop-sls/pom.xml                                  | 4 ++--
 hadoop-tools/hadoop-streaming/pom.xml                            | 4 ++--
 hadoop-tools/hadoop-tools-dist/pom.xml                           | 4 ++--
 hadoop-tools/pom.xml                                             | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml          | 4 ++--
 .../hadoop-yarn-applications-distributedshell/pom.xml            | 4 ++--
 .../hadoop-yarn-applications-unmanaged-am-launcher/pom.xml       | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml       | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml       | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml     | 4 ++--
 .../hadoop-yarn-server-applicationhistoryservice/pom.xml         | 4 ++--
 .../hadoop-yarn-server/hadoop-yarn-server-common/pom.xml         | 4 ++--
 .../hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml    | 4 ++--
 .../hadoop-yarn-server-resourcemanager/pom.xml                   | 4 ++--
 .../hadoop-yarn-server/hadoop-yarn-server-router/pom.xml         | 4 ++--
 .../hadoop-yarn-server-sharedcachemanager/pom.xml                | 4 ++--
 .../hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml          | 4 ++--
 .../hadoop-yarn-server-timeline-pluginstorage/pom.xml            | 4 ++--
 .../hadoop-yarn-server-timelineservice-hbase-tests/pom.xml       | 4 ++--
 .../hadoop-yarn-server-timelineservice-hbase/pom.xml             | 2 +-
 .../hadoop-yarn-server-timelineservice/pom.xml                   | 4 ++--
 .../hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml      | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml       | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/pom.xml         | 4 ++--
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml           | 4 ++--
 hadoop-yarn-project/hadoop-yarn/pom.xml                          | 4 ++--
 hadoop-yarn-project/pom.xml                                      | 4 ++--
 pom.xml                                                          | 2 +-
 85 files changed, 162 insertions(+), 162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-assemblies/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/pom.xml b/hadoop-assemblies/pom.xml
index 7a38421..9f99503 100644
--- a/hadoop-assemblies/pom.xml
+++ b/hadoop-assemblies/pom.xml
@@ -23,11 +23,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-assemblies</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop Assemblies</name>
   <description>Apache Hadoop Assemblies</description>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
index 8933495..6597889 100644
--- a/hadoop-build-tools/pom.xml
+++ b/hadoop-build-tools/pom.xml
@@ -18,7 +18,7 @@
   <parent>
     <artifactId>hadoop-main</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-build-tools</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-api/pom.xml b/hadoop-client-modules/hadoop-client-api/pom.xml
index 2d3548b..a50ffed 100644
--- a/hadoop-client-modules/hadoop-client-api/pom.xml
+++ b/hadoop-client-modules/hadoop-client-api/pom.xml
@@ -18,11 +18,11 @@
 <parent>
    <groupId>org.apache.hadoop</groupId>
    <artifactId>hadoop-project</artifactId>
-   <version>3.0.0-beta1-SNAPSHOT</version>
+   <version>3.1.0-SNAPSHOT</version>
    <relativePath>../../hadoop-project</relativePath>
 </parent>
   <artifactId>hadoop-client-api</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <description>Apache Hadoop Client</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
index 2f31fa6..75f4d19 100644
--- a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
+++ b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-client-check-invariants</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <description>Enforces our invariants for the api and runtime client modules.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
index 0e23db9..d0d0712 100644
--- a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
+++ b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-client-check-test-invariants</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <description>Enforces our invariants for the testing client modules.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-integration-tests/pom.xml b/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
index c180965..1295b59 100644
--- a/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
+++ b/hadoop-client-modules/hadoop-client-integration-tests/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-client-integration-tests</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
 
   <description>Checks that we can use the generated artifacts</description>
   <name>Apache Hadoop Client Packaging Integration Tests</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
index 5cf1fad..ba84e87 100644
--- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-client-minicluster</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <description>Apache Hadoop Minicluster for Clients</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-runtime/pom.xml b/hadoop-client-modules/hadoop-client-runtime/pom.xml
index 24c6b7a..6879a23 100644
--- a/hadoop-client-modules/hadoop-client-runtime/pom.xml
+++ b/hadoop-client-modules/hadoop-client-runtime/pom.xml
@@ -18,11 +18,11 @@
 <parent>
    <groupId>org.apache.hadoop</groupId>
    <artifactId>hadoop-project</artifactId>
-   <version>3.0.0-beta1-SNAPSHOT</version>
+   <version>3.1.0-SNAPSHOT</version>
    <relativePath>../../hadoop-project</relativePath>
 </parent>
   <artifactId>hadoop-client-runtime</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <description>Apache Hadoop Client</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml
index a32a2a3..b802416 100644
--- a/hadoop-client-modules/hadoop-client/pom.xml
+++ b/hadoop-client-modules/hadoop-client/pom.xml
@@ -18,11 +18,11 @@
 <parent>
    <groupId>org.apache.hadoop</groupId>
    <artifactId>hadoop-project-dist</artifactId>
-   <version>3.0.0-beta1-SNAPSHOT</version>
+   <version>3.1.0-SNAPSHOT</version>
    <relativePath>../../hadoop-project-dist</relativePath>
 </parent>
   <artifactId>hadoop-client</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
 
   <description>Apache Hadoop Client aggregation pom with dependencies exposed</description>
   <name>Apache Hadoop Client Aggregator</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-client-modules/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/pom.xml b/hadoop-client-modules/pom.xml
index c60b7e1..7baca38 100644
--- a/hadoop-client-modules/pom.xml
+++ b/hadoop-client-modules/pom.xml
@@ -18,7 +18,7 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-client-modules</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
index 829316e..9711e52 100644
--- a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
+++ b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-cloud-storage</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <description>Apache Hadoop Cloud Storage</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-cloud-storage-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-cloud-storage-project/pom.xml b/hadoop-cloud-storage-project/pom.xml
index 0907d2f..e07b284 100644
--- a/hadoop-cloud-storage-project/pom.xml
+++ b/hadoop-cloud-storage-project/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-cloud-storage-project</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Cloud Storage Project</description>
   <name>Apache Hadoop Cloud Storage Project</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-annotations/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-annotations/pom.xml b/hadoop-common-project/hadoop-annotations/pom.xml
index 0b41f78..17e49eb 100644
--- a/hadoop-common-project/hadoop-annotations/pom.xml
+++ b/hadoop-common-project/hadoop-annotations/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-annotations</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Annotations</description>
   <name>Apache Hadoop Annotations</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-auth-examples/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth-examples/pom.xml b/hadoop-common-project/hadoop-auth-examples/pom.xml
index 83cee15..66a97af 100644
--- a/hadoop-common-project/hadoop-auth-examples/pom.xml
+++ b/hadoop-common-project/hadoop-auth-examples/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-auth-examples</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>war</packaging>
 
   <name>Apache Hadoop Auth Examples</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-auth/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml
index 078ac40..8cffeae 100644
--- a/hadoop-common-project/hadoop-auth/pom.xml
+++ b/hadoop-common-project/hadoop-auth/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-auth</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <name>Apache Hadoop Auth</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 1d188ba..aae040a 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project-dist</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project-dist</relativePath>
   </parent>
   <artifactId>hadoop-common</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Common</description>
   <name>Apache Hadoop Common</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-kms/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/pom.xml b/hadoop-common-project/hadoop-kms/pom.xml
index ae2a325..c3e612c 100644
--- a/hadoop-common-project/hadoop-kms/pom.xml
+++ b/hadoop-common-project/hadoop-kms/pom.xml
@@ -22,11 +22,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-kms</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <name>Apache Hadoop KMS</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-minikdc/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-minikdc/pom.xml b/hadoop-common-project/hadoop-minikdc/pom.xml
index 759786c..8305ec0 100644
--- a/hadoop-common-project/hadoop-minikdc/pom.xml
+++ b/hadoop-common-project/hadoop-minikdc/pom.xml
@@ -18,12 +18,12 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-minikdc</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop MiniKDC</description>
   <name>Apache Hadoop MiniKDC</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/hadoop-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml
index bd5ab92..90fdf61 100644
--- a/hadoop-common-project/hadoop-nfs/pom.xml
+++ b/hadoop-common-project/hadoop-nfs/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-nfs</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <name>Apache Hadoop NFS</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-common-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/pom.xml b/hadoop-common-project/pom.xml
index 1b3a176..9413c20 100644
--- a/hadoop-common-project/pom.xml
+++ b/hadoop-common-project/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-common-project</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Common Project</description>
   <name>Apache Hadoop Common Project</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml
index 84d5ab3..65a352c 100644
--- a/hadoop-dist/pom.xml
+++ b/hadoop-dist/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-dist</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Distribution</description>
   <name>Apache Hadoop Distribution</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index cc5481f..baf63ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -20,11 +20,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project-dist</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project-dist</relativePath>
   </parent>
   <artifactId>hadoop-hdfs-client</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop HDFS Client</description>
   <name>Apache Hadoop HDFS Client</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
index 16b94f3..0e63527 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
@@ -22,11 +22,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-hdfs-httpfs</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <name>Apache Hadoop HttpFS</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
index db17c22..54871d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
@@ -20,11 +20,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project-dist</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project-dist</relativePath>
   </parent>
   <artifactId>hadoop-hdfs-native-client</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop HDFS Native Client</description>
   <name>Apache Hadoop HDFS Native Client</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
index a70b9b1..7fc7d32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
@@ -20,11 +20,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-hdfs-nfs</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop HDFS-NFS</description>
   <name>Apache Hadoop HDFS-NFS</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/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 fa1044d..425572f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -20,11 +20,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project-dist</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project-dist</relativePath>
   </parent>
   <artifactId>hadoop-hdfs</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop HDFS</description>
   <name>Apache Hadoop HDFS</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-hdfs-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/pom.xml b/hadoop-hdfs-project/pom.xml
index a3fdd02..dc343a2 100644
--- a/hadoop-hdfs-project/pom.xml
+++ b/hadoop-hdfs-project/pom.xml
@@ -20,11 +20,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-hdfs-project</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop HDFS Project</description>
   <name>Apache Hadoop HDFS Project</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml
index e5fb153..b040112 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-app</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce App</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
index b88b012..8fc8a07 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-common</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce Common</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index a23827d..c34f7bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-core</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce Core</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs-plugins/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs-plugins/pom.xml
index 577d92d..4138d02 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs-plugins/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs-plugins/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-hs-plugins</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce HistoryServer Plugins</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
index b185d45..e56c007 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-hs</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce HistoryServer</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
index d618325..7b2390e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce JobClient</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
index 9fd5d1e..df892cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-nativetask</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce NativeTask</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml
index 3589350..77bfe0a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-mapreduce-client</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-mapreduce-client-shuffle</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce Shuffle</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index 212078b..aa7c7b1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-mapreduce-client</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop MapReduce Client</name>
   <packaging>pom</packaging>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
index 13dc340..4a50ed3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-mapreduce-examples</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop MapReduce Examples</description>
   <name>Apache Hadoop MapReduce Examples</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-mapreduce-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/pom.xml b/hadoop-mapreduce-project/pom.xml
index c2816e5..773eb1f 100644
--- a/hadoop-mapreduce-project/pom.xml
+++ b/hadoop-mapreduce-project/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-mapreduce</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Hadoop MapReduce</name>
   <url>http://hadoop.apache.org/mapreduce/</url>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml
index 2ff93f7..d0f4a73 100644
--- a/hadoop-maven-plugins/pom.xml
+++ b/hadoop-maven-plugins/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-maven-plugins</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-minicluster/pom.xml b/hadoop-minicluster/pom.xml
index 7529f9e..4558e99 100644
--- a/hadoop-minicluster/pom.xml
+++ b/hadoop-minicluster/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-minicluster</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>jar</packaging>
 
   <description>Apache Hadoop Mini-Cluster</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index cf43218..addc2a5 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-project-dist</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Project Dist POM</description>
   <name>Apache Hadoop Project Dist POM</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 9a52d76..f74c0e8 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -20,10 +20,10 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-main</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>hadoop-project</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Project POM</description>
   <name>Apache Hadoop Project POM</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-aliyun/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/pom.xml b/hadoop-tools/hadoop-aliyun/pom.xml
index 7ffcc3d..a298efa 100644
--- a/hadoop-tools/hadoop-aliyun/pom.xml
+++ b/hadoop-tools/hadoop-aliyun/pom.xml
@@ -18,7 +18,7 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-aliyun</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-archive-logs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archive-logs/pom.xml b/hadoop-tools/hadoop-archive-logs/pom.xml
index 0ddd448..17d444b 100644
--- a/hadoop-tools/hadoop-archive-logs/pom.xml
+++ b/hadoop-tools/hadoop-archive-logs/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-archive-logs</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Archive Logs</description>
   <name>Apache Hadoop Archive Logs</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-archives/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/pom.xml b/hadoop-tools/hadoop-archives/pom.xml
index 3389057..d4641ea 100644
--- a/hadoop-tools/hadoop-archives/pom.xml
+++ b/hadoop-tools/hadoop-archives/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-archives</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Archives</description>
   <name>Apache Hadoop Archives</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index bcb0e07..6bab9a7 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-aws</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop Amazon Web Services support</name>
   <description>
     This module contains code to support integration with Amazon Web Services.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index f699464..b06c774 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-azure-datalake</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-azure/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index 03b531b..0c5ac63 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-azure</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-datajoin/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-datajoin/pom.xml b/hadoop-tools/hadoop-datajoin/pom.xml
index 3c71309..b37f465 100644
--- a/hadoop-tools/hadoop-datajoin/pom.xml
+++ b/hadoop-tools/hadoop-datajoin/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-datajoin</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Data Join</description>
   <name>Apache Hadoop Data Join</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-distcp/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/pom.xml b/hadoop-tools/hadoop-distcp/pom.xml
index ce78fb7..a044a81 100644
--- a/hadoop-tools/hadoop-distcp/pom.xml
+++ b/hadoop-tools/hadoop-distcp/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-distcp</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Distributed Copy</description>
   <name>Apache Hadoop Distributed Copy</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-extras/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-extras/pom.xml b/hadoop-tools/hadoop-extras/pom.xml
index 5968a71..f28b5cd 100644
--- a/hadoop-tools/hadoop-extras/pom.xml
+++ b/hadoop-tools/hadoop-extras/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-extras</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Extras</description>
   <name>Apache Hadoop Extras</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-gridmix/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/pom.xml b/hadoop-tools/hadoop-gridmix/pom.xml
index 1494c7b..7171c9a 100644
--- a/hadoop-tools/hadoop-gridmix/pom.xml
+++ b/hadoop-tools/hadoop-gridmix/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-gridmix</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Gridmix</description>
   <name>Apache Hadoop Gridmix</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-kafka/pom.xml b/hadoop-tools/hadoop-kafka/pom.xml
index 3b61650..1ae7270 100644
--- a/hadoop-tools/hadoop-kafka/pom.xml
+++ b/hadoop-tools/hadoop-kafka/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-kafka</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop Kafka Library support</name>
   <description>
     This module contains code to support integration with Kafka.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-openstack/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/pom.xml b/hadoop-tools/hadoop-openstack/pom.xml
index 00e6f43..9e492ef 100644
--- a/hadoop-tools/hadoop-openstack/pom.xml
+++ b/hadoop-tools/hadoop-openstack/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-openstack</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop OpenStack support</name>
   <description>
     This module contains code to support integration with OpenStack.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-pipes/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-pipes/pom.xml b/hadoop-tools/hadoop-pipes/pom.xml
index 1a951a0..4259385 100644
--- a/hadoop-tools/hadoop-pipes/pom.xml
+++ b/hadoop-tools/hadoop-pipes/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-pipes</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Pipes</description>
   <name>Apache Hadoop Pipes</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-rumen/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-rumen/pom.xml b/hadoop-tools/hadoop-rumen/pom.xml
index 864ad2f..b6e7e6b 100644
--- a/hadoop-tools/hadoop-rumen/pom.xml
+++ b/hadoop-tools/hadoop-rumen/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-rumen</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Rumen</description>
   <name>Apache Hadoop Rumen</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-sls/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/pom.xml b/hadoop-tools/hadoop-sls/pom.xml
index eba0514..0882f77 100644
--- a/hadoop-tools/hadoop-sls/pom.xml
+++ b/hadoop-tools/hadoop-sls/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-sls</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Scheduler Load Simulator</description>
   <name>Apache Hadoop Scheduler Load Simulator</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/pom.xml b/hadoop-tools/hadoop-streaming/pom.xml
index 5ad8f4d..7c9cff4 100644
--- a/hadoop-tools/hadoop-streaming/pom.xml
+++ b/hadoop-tools/hadoop-streaming/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-streaming</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop MapReduce Streaming</description>
   <name>Apache Hadoop MapReduce Streaming</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index 4e399a8..a5aa264 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project-dist</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project-dist</relativePath>
   </parent>
   <artifactId>hadoop-tools-dist</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Tools Dist</description>
   <name>Apache Hadoop Tools Dist</name>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index a6b925f..18381a3 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-tools</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Tools</description>
   <name>Apache Hadoop Tools</name>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
index f8f2973..450207c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-api</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN API</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
index 1fea954..529f5ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn-applications</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-applications-distributedshell</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN DistributedShell</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml
index ef0c7f9..4e0fca9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn-applications</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-applications-unmanaged-am-launcher</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Unmanaged Am Launcher</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
index 4c11d4d..644a1dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-applications</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Applications</name>
   <packaging>pom</packaging>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
index b83bff8..8413f15 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
@@ -17,10 +17,10 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>hadoop-yarn-client</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Client</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index c2a5c67..024b3c9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-common</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Common</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
index 1435007..4233104 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-registry</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Registry</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
index cace493..f81a5d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
@@ -22,11 +22,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN ApplicationHistoryService</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index e8d3880..43ae3af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-common</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Server Common</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
index a50a769..70b29f4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN NodeManager</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 6c016fe..ba17b51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN ResourceManager</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
index e8b4d56..3e7cc11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
@@ -19,12 +19,12 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-yarn-server-router</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Router</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/pom.xml
index a3b7879..dcc44ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/pom.xml
@@ -17,10 +17,10 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>hadoop-yarn-server-sharedcachemanager</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN SharedCacheManager</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
index babe9fe..a7b4a6d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
@@ -19,10 +19,10 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>hadoop-yarn-server-tests</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Server Tests</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
index 8b57dcd..e970fc2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
@@ -22,11 +22,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Timeline Plugin Storage</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
index b090954..ffba1fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
@@ -22,11 +22,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-timelineservice-hbase-tests</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Timeline Service HBase tests</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
index acea867..f200e9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-timelineservice-hbase</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
index 3dc6222..5cbfbf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
@@ -22,11 +22,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-timelineservice</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Timeline Service</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
index 0951528..d185bfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server-web-proxy</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Web Proxy</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
index 8335fc8..8156016 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-server</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Server</name>
   <packaging>pom</packaging>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/pom.xml
index 7576f76..128c328 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/pom.xml
@@ -19,11 +19,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-site</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Site</name>
   <packaging>pom</packaging>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
index 1d1b1b0..7aa03ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
@@ -20,11 +20,11 @@
   <parent>
     <artifactId>hadoop-yarn</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hadoop-yarn-ui</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN UI</name>
   <packaging>${packagingType}</packaging>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/hadoop-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index 12e4604..be435d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -16,11 +16,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-yarn</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Hadoop YARN</name>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/hadoop-yarn-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml
index 2798192..3cbbaa7 100644
--- a/hadoop-yarn-project/pom.xml
+++ b/hadoop-yarn-project/pom.xml
@@ -18,11 +18,11 @@
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-beta1-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../hadoop-project</relativePath>
   </parent>
   <artifactId>hadoop-yarn-project</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Hadoop YARN Project</name>
   <url>http://hadoop.apache.org/yarn/</url>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d419c98/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3fded0f..efd7578 100644
--- a/pom.xml
+++ b/pom.xml
@@ -18,7 +18,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-main</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <description>Apache Hadoop Main</description>
   <name>Apache Hadoop Main</name>
   <packaging>pom</packaging>


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


[32/48] 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/9203e566/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/9203e566/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/9203e566/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


[36/48] 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/5da7d3f4/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


[23/48] hadoop git commit: Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index bc98e2f..03569d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planne
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.junit.After;
@@ -47,6 +48,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+/**
+ * Testing the class {@link InMemoryPlan}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestInMemoryPlan {
 
   private String user = "yarn";
@@ -62,6 +67,7 @@ public class TestInMemoryPlan {
   private ReservationAgent agent;
   private Planner replanner;
   private RMContext context;
+  private long maxPeriodicity;
 
   @Before
   public void setUp() throws PlanningException {
@@ -72,7 +78,7 @@ public class TestInMemoryPlan {
 
     clock = mock(Clock.class);
     queueMetrics = mock(QueueMetrics.class);
-    policy = mock(SharingPolicy.class);
+    policy = new NoOverCommitPolicy();
     replanner = mock(Planner.class);
 
     when(clock.getTime()).thenReturn(1L);
@@ -95,15 +101,41 @@ public class TestInMemoryPlan {
 
   @Test
   public void testAddReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, 0);
+  }
+
+  @Test
+  public void testAddPeriodicReservation() throws PlanningException {
+
+    maxPeriodicity = 100;
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
+        context, new UTCClock());
+
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 50 };
+    int start = 10;
+    long period = 20;
+    ReservationAllocation rAllocation = createReservationAllocation(
+        reservationID, start, alloc, String.valueOf(period));
+    // use periodicity of 1hr
+    rAllocation.setPeriodicity(period);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -111,32 +143,54 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, period);
+
+    RLESparseResourceAllocation available =
+        plan.getAvailableResourceOverTime(user, reservationID, 150, 330, 50);
+    System.out.println(available);
   }
 
-  private void checkAllocation(Plan plan, int[] alloc, int start) {
+  private void checkAllocation(Plan plan, int[] alloc, int start,
+      long periodicity) {
+    long end = start + alloc.length;
+    if (periodicity > 0) {
+      end = end + maxPeriodicity;
+    }
     RLESparseResourceAllocation userCons =
-        plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
+        plan.getConsumptionForUserOverTime(user, start, end * 3);
 
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
-          userCons.getCapacityAtTime(start + i));
+      // only one instance for non-periodic reservation
+      if (periodicity <= 0) {
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            plan.getTotalCommittedResources(start + i));
+        Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+            userCons.getCapacityAtTime(start + i));
+      } else {
+        // periodic reservations should repeat
+        long y = 0;
+        Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
+        while (y <= end * 2) {
+          Assert.assertEquals("At time: " + start + i + y, res,
+              plan.getTotalCommittedResources(start + i + y));
+          Assert.assertEquals(" At time: " + (start + i + y), res,
+              userCons.getCapacityAtTime(start + i + y));
+          y = y + periodicity;
+        }
+      }
     }
   }
 
   @Test
   public void testAddEmptyReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = {};
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -148,15 +202,14 @@ public class TestInMemoryPlan {
   @Test
   public void testAddReservationAlreadyExists() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -164,7 +217,7 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc, start);
+    checkAllocation(plan, alloc, start, 0);
 
     // Try to add it again
     try {
@@ -180,16 +233,15 @@ public class TestInMemoryPlan {
 
   @Test
   public void testUpdateReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // First add a reservation
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -210,8 +262,8 @@ public class TestInMemoryPlan {
     // Now update it
     start = 110;
     int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
-    rAllocation = createReservationAllocation(reservationID, start,
-            updatedAlloc, true);
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc, true);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -219,32 +271,71 @@ public class TestInMemoryPlan {
     }
     doAssertions(plan, rAllocation);
 
-    userCons =
-        plan.getConsumptionForUserOverTime(user, start, start
-            + updatedAlloc.length);
+    userCons = plan.getConsumptionForUserOverTime(user, start,
+        start + updatedAlloc.length);
 
     for (int i = 0; i < updatedAlloc.length; i++) {
-      Assert.assertEquals(
-     Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), plan.getTotalCommittedResources(start + i));
-      Assert.assertEquals(
-          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
-              + i), userCons.getCapacityAtTime(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (updatedAlloc[i] + i),
+          updatedAlloc[i] + i), userCons.getCapacityAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testUpdatePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now update it
+    start = 110;
+    int[] updatedAlloc = { 30, 40, 50 };
+    rAllocation =
+        createReservationAllocation(reservationID, start, updatedAlloc);
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    try {
+      plan.updateReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
     }
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, updatedAlloc, start, period);
   }
 
   @Test
   public void testUpdateNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to update a reservation without adding
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc);
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -260,15 +351,14 @@ public class TestInMemoryPlan {
   @Test
   public void testDeleteReservation() {
     // First add a reservation
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID, start, alloc, true);
+        createReservationAllocation(reservationID, start, alloc, true);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -307,10 +397,46 @@ public class TestInMemoryPlan {
   }
 
   @Test
+  public void testDeletePeriodicReservation() {
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    // First add a reservation
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 20 };
+    int start = 100;
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
+    // use periodicity of 1hr
+    long period = 3600000;
+    rAllocation.getReservationDefinition()
+        .setRecurrenceExpression(String.valueOf(period));
+    rAllocation.setPeriodicity(period);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation, false);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    System.out.println(plan.toString());
+    doAssertions(plan, rAllocation);
+    checkAllocation(plan, alloc, start, period);
+
+    // Now delete it
+    try {
+      plan.deleteReservation(reservationID);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+    System.out.print(plan);
+    checkAllocation(plan, new int[] { 0, 0 }, start, period);
+  }
+
+  @Test
   public void testDeleteNonExistingReservation() {
-    Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-            resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
@@ -328,8 +454,9 @@ public class TestInMemoryPlan {
 
   @Test
   public void testArchiveCompletedReservations() {
+    SharingPolicy sharingPolicy = mock(SharingPolicy.class);
     Plan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        new InMemoryPlan(queueMetrics, sharingPolicy, agent, totalCapacity, 1L,
             resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID1 =
         ReservationSystemTestUtil.getNewReservationId();
@@ -337,7 +464,7 @@ public class TestInMemoryPlan {
     int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     ReservationAllocation rAllocation =
-            createReservationAllocation(reservationID1, start, alloc1);
+        createReservationAllocation(reservationID1, start, alloc1);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
@@ -345,15 +472,14 @@ public class TestInMemoryPlan {
       Assert.fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
-    checkAllocation(plan, alloc1, start);
-
+    checkAllocation(plan, alloc1, start, 0);
 
     // Now add another one
     ReservationId reservationID2 =
         ReservationSystemTestUtil.getNewReservationId();
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
-            createReservationAllocation(reservationID2, start, alloc2, true);
+        createReservationAllocation(reservationID2, start, alloc2, true);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
@@ -367,16 +493,18 @@ public class TestInMemoryPlan {
 
     for (int i = 0; i < alloc2.length; i++) {
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          plan.getTotalCommittedResources(start + i));
       Assert.assertEquals(
-          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
-              + alloc2[i] + i), userCons.getCapacityAtTime(start + i));
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i),
+              alloc1[i] + alloc2[i] + i),
+          userCons.getCapacityAtTime(start + i));
     }
 
     // Now archive completed reservations
     when(clock.getTime()).thenReturn(106L);
-    when(policy.getValidWindow()).thenReturn(1L);
+    when(sharingPolicy.getValidWindow()).thenReturn(1L);
     try {
       // will only remove 2nd reservation as only that has fallen out of the
       // archival window
@@ -386,7 +514,7 @@ public class TestInMemoryPlan {
     }
     Assert.assertNotNull(plan.getReservationById(reservationID1));
     Assert.assertNull(plan.getReservationById(reservationID2));
-    checkAllocation(plan, alloc1, start);
+    checkAllocation(plan, alloc1, start, 0);
 
     when(clock.getTime()).thenReturn(107L);
     try {
@@ -411,15 +539,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsById() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -429,31 +556,30 @@ public class TestInMemoryPlan {
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(reservationID, null, "");
+        plan.getReservations(reservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsByInvalidId() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -463,23 +589,22 @@ public class TestInMemoryPlan {
 
     // If reservationId is null, then nothing is returned.
     ReservationId invalidReservationID =
-            ReservationSystemTestUtil.getNewReservationId();
+        ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(invalidReservationID, null, "");
+        plan.getReservations(invalidReservationID, null, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
   @Test
   public void testGetReservationsByTimeInterval() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -489,23 +614,24 @@ public class TestInMemoryPlan {
 
     // Verify that get by time interval works if the selection interval
     // completely overlaps with the allocation.
-    ReservationInterval interval = new ReservationInterval(rAllocation
-            .getStartTime(), rAllocation.getEndTime());
+    ReservationInterval interval = new ReservationInterval(
+        rAllocation.getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
     // falls within the allocation
     long duration = rAllocation.getEndTime() - rAllocation.getStartTime();
-    interval = new ReservationInterval(rAllocation.getStartTime() + duration
-            * (long)0.3, rAllocation.getEndTime() - duration * (long)0.3);
+    interval = new ReservationInterval(
+        rAllocation.getStartTime() + duration * (long) 0.3,
+        rAllocation.getEndTime() - duration * (long) 0.3);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
     // time of the selection interval falls right at the start of the
@@ -513,13 +639,13 @@ public class TestInMemoryPlan {
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
     // time of the selection interval falls right at the end of the allocation.
-    interval = new ReservationInterval(rAllocation
-            .getEndTime(), Long.MAX_VALUE);
+    interval =
+        new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
 
@@ -532,15 +658,14 @@ public class TestInMemoryPlan {
 
   @Test
   public void testGetReservationsAtTime() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -549,23 +674,22 @@ public class TestInMemoryPlan {
     }
 
     Set<ReservationAllocation> rAllocations =
-            plan.getReservationsAtTime(rAllocation.getStartTime());
+        plan.getReservationsAtTime(rAllocation.getStartTime());
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoInput() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     ReservationId reservationID =
-            ReservationSystemTestUtil.getNewReservationId();
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    ReservationAllocation rAllocation = createReservationAllocation
-            (reservationID, start, alloc);
+    ReservationAllocation rAllocation =
+        createReservationAllocation(reservationID, start, alloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
@@ -576,22 +700,21 @@ public class TestInMemoryPlan {
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, null, "");
+        plan.getReservations(null, null, "");
     Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation.compareTo(
-            (ReservationAllocation) rAllocations.toArray()[0]) == 0);
+    Assert.assertTrue(rAllocation
+        .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
   @Test
   public void testGetReservationsWithNoReservation() {
-    Plan plan =
-            new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-                    resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
+    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+        resCalc, minAlloc, maxAlloc, planName, replanner, true, context);
     // Verify that get reservation returns no entries if no queries are made.
 
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
-            plan.getReservations(null, interval, "");
+        plan.getReservations(null, interval, "");
     Assert.assertTrue(rAllocations.size() == 0);
   }
 
@@ -600,7 +723,9 @@ public class TestInMemoryPlan {
     Assert.assertNotNull(plan.getReservationById(reservationID));
     Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
     Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
-    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    if (rAllocation.getPeriodicity() <= 0) {
+      Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    }
     Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
     Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
     Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
@@ -610,7 +735,8 @@ public class TestInMemoryPlan {
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,
-      long deadline, long duration, Collection<ReservationRequest> resources) {
+      long deadline, long duration, Collection<ReservationRequest> resources,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
@@ -619,6 +745,7 @@ public class TestInMemoryPlan {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrenceExpression);
     return rDef;
   }
 
@@ -633,31 +760,43 @@ public class TestInMemoryPlan {
       } else {
         numContainers = alloc[i];
       }
-      ReservationRequest rr =
-          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-              (numContainers));
+      ReservationRequest rr = ReservationRequest
+          .newInstance(Resource.newInstance(1024, 1), (numContainers));
       req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
     }
     return req;
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc) {
-    return createReservationAllocation(reservationID, start, alloc, false);
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc) {
+    return createReservationAllocation(reservationID, start, alloc, false, "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep) {
+    return createReservationAllocation(reservationID, start, alloc, isStep,
+        "0");
+  }
+
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc,
+      String recurrenceExp) {
+    return createReservationAllocation(reservationID, start, alloc, false,
+        recurrenceExp);
   }
 
-  private ReservationAllocation createReservationAllocation(ReservationId
-            reservationID, int start, int[] alloc, boolean isStep) {
+  private ReservationAllocation createReservationAllocation(
+      ReservationId reservationID, int start, int[] alloc, boolean isStep,
+      String recurrenceExp) {
     Map<ReservationInterval, ReservationRequest> allocations =
-            generateAllocation(start, alloc, isStep);
+        generateAllocation(start, alloc, isStep);
     ReservationDefinition rDef =
-            createSimpleReservationDefinition(start, start + alloc.length,
-                    alloc.length, allocations.values());
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values(), recurrenceExp);
     Map<ReservationInterval, Resource> allocs =
-            ReservationSystemUtil.toResources(allocations);
+        ReservationSystemUtil.toResources(allocations);
     return new InMemoryReservationAllocation(reservationID, rDef, user,
-            planName,
-                    start, start + alloc.length, allocs, resCalc, minAlloc);
+        planName, start, start + alloc.length, allocs, resCalc, minAlloc);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
index 554eb58..457e2ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
@@ -19,26 +19,27 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Testing the class PeriodicRLESparseResourceAllocation.
+ * Testing the class {@link PeriodicRLESparseResourceAllocation}.
  */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestPeriodicRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestPeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPeriodicRLESparseResourceAllocation.class);
 
   @Test
   public void testPeriodicCapacity() {
-    int[] alloc = {10, 7, 5, 2, 0};
-    long[] timeSteps = {0L, 5L, 10L, 15L, 19L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 10, 7, 5, 2, 0 };
+    long[] timeSteps = { 0L, 5L, 10L, 15L, 19L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
@@ -54,43 +55,74 @@ public class TestPeriodicRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(8, 2),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(16, 3),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(17, 4),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        periodicVector.getMaximumPeriodicCapacity(32, 5),
+    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
   @Test
+  public void testMixPeriodicAndNonPeriodic() throws PlanningException {
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation tempPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
+    PeriodicRLESparseResourceAllocation periodic =
+        new PeriodicRLESparseResourceAllocation(tempPeriodic, 10L);
+
+    int[] alloc2 = { 10, 10, 0 };
+    long[] timeSteps2 = { 12L, 13L, 14L };
+    RLESparseResourceAllocation nonPeriodic = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc2, timeSteps2);
+
+    RLESparseResourceAllocation merged =
+        RLESparseResourceAllocation.merge(nonPeriodic.getResourceCalculator(),
+            Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
+            RLESparseResourceAllocation.RLEOperator.add, 2, 25);
+
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(2L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(3L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(11L));
+    Assert.assertEquals(Resource.newInstance(15, 15),
+        merged.getCapacityAtTime(12L));
+    Assert.assertEquals(Resource.newInstance(10, 10),
+        merged.getCapacityAtTime(13L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(14L));
+    Assert.assertEquals(Resource.newInstance(2, 2),
+        merged.getCapacityAtTime(21L));
+    Assert.assertEquals(Resource.newInstance(5, 5),
+        merged.getCapacityAtTime(22L));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        merged.getCapacityAtTime(23L));
+  }
+
+  @Test
   public void testSetCapacityInInterval() {
-    int[] alloc = {2, 5, 0};
-    long[] timeSteps = {1L, 2L, 3L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 0 };
+    long[] timeSteps = { 1L, 2L, 3L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
-    periodicVector.addInterval(
-        interval, Resource.newInstance(8, 8));
+    periodicVector.addInterval(interval, Resource.newInstance(8, 8));
     Assert.assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
     Assert.assertEquals(Resource.newInstance(8, 8),
@@ -99,21 +131,20 @@ public class TestPeriodicRLESparseResourceAllocation {
         periodicVector.getCapacityAtTime(10L));
     Assert.assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    Assert.assertFalse(periodicVector.addInterval(
-        new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
+    // Assert.assertFalse(periodicVector.addInterval(
+    // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
   public void testRemoveInterval() {
-    int[] alloc = {2, 5, 3, 4, 0};
-    long[] timeSteps = {1L, 3L, 5L, 7L, 9L};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    int[] alloc = { 2, 5, 3, 4, 0 };
+    long[] timeSteps = { 1L, 3L, 5L, 7L, 9L };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(periodicVector.removeInterval(
-        interval, Resource.newInstance(3, 3)));
+    Assert.assertTrue(
+        periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
     Assert.assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
     Assert.assertEquals(Resource.newInstance(2, 2),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index bfe46e1..0027ceb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -40,10 +40,14 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Testing the class {@link RLESparseResourceAllocation}.
+ */
+@SuppressWarnings("checkstyle:nowhitespaceafter")
 public class TestRLESparseResourceAllocation {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRLESparseResourceAllocation.class);
 
   @Test
   public void testMergeAdd() throws PlanningException {
@@ -196,7 +200,8 @@ public class TestRLESparseResourceAllocation {
       // Expected!
     }
 
-    // Testing that the subtractTestNonNegative detects problems even if only one
+    // Testing that the subtractTestNonNegative detects problems even if only
+    // one
     // of the resource dimensions is "<0"
     a.put(10L, Resource.newInstance(10, 5));
     b.put(11L, Resource.newInstance(5, 6));
@@ -286,9 +291,8 @@ public class TestRLESparseResourceAllocation {
   public void testRangeOverlapping() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 
-    RLESparseResourceAllocation r =
-        new RLESparseResourceAllocation(resCalc);
-    int[] alloc = {10, 10, 10, 10, 10, 10};
+    RLESparseResourceAllocation r = new RLESparseResourceAllocation(resCalc);
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
     Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
@@ -299,9 +303,9 @@ public class TestRLESparseResourceAllocation {
     long d = r.getLatestNonNullTime();
 
     // tries to trigger "out-of-range" bug
-    r =  r.getRangeOverlapping(s, d);
-    r = r.getRangeOverlapping(s-1, d-1);
-    r = r.getRangeOverlapping(s+1, d+1);
+    r = r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s - 1, d - 1);
+    r = r.getRangeOverlapping(s + 1, d + 1);
   }
 
   @Test
@@ -370,25 +374,29 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
-        .getMemorySize());
-    Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
-        .getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(10).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(19).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(21).getMemorySize());
+    Assert.assertEquals(2 * 102400,
+        rleSparseVector.getCapacityAtTime(26).getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
-        .getMemorySize());
-    Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
-        .getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(11).getMemorySize());
+    Assert.assertEquals(-102400,
+        rleSparseVector.getCapacityAtTime(9).getMemorySize());
+    Assert.assertEquals(0,
+        rleSparseVector.getCapacityAtTime(13).getMemorySize());
+    Assert.assertEquals(102400,
+        rleSparseVector.getCapacityAtTime(20).getMemorySize());
 
   }
 
@@ -500,7 +508,8 @@ public class TestRLESparseResourceAllocation {
     }
     mapAllocations = rleSparseVector.toIntervalMap();
     Assert.assertTrue(mapAllocations.size() == 5);
-    for (Entry<ReservationInterval, Resource> entry : mapAllocations.entrySet()) {
+    for (Entry<ReservationInterval, Resource> entry : mapAllocations
+        .entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
@@ -526,59 +535,46 @@ public class TestRLESparseResourceAllocation {
 
   @Test
   public void testMaxPeriodicCapacity() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 6, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 6, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(
-        rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
   @Test
   public void testGetMinimumCapacityInInterval() {
-    long[] timeSteps = {0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L};
-    int[] alloc = {2, 5, 7, 10, 3, 4, 0, 8};
-    RLESparseResourceAllocation rleSparseVector =
-        ReservationSystemTestUtil.generateRLESparseResourceAllocation(
-            alloc, timeSteps);
+    long[] timeSteps = { 0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L };
+    int[] alloc = { 2, 5, 7, 10, 3, 4, 0, 8 };
+    RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
+        .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(
-        rleSparseVector.getMinimumCapacityInInterval(
-            new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
+    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+        new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 
-  private void setupArrays(
-      TreeMap<Long, Resource> a, TreeMap<Long, Resource> b) {
+  private void setupArrays(TreeMap<Long, Resource> a,
+      TreeMap<Long, Resource> b) {
     a.put(10L, Resource.newInstance(5, 5));
     a.put(20L, Resource.newInstance(10, 10));
     a.put(30L, Resource.newInstance(15, 15));
@@ -620,8 +616,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
-              Resource.newInstance(1024, 1), (numContainers))));
+          ReservationSystemUtil.toResource(ReservationRequest
+              .newInstance(Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index c4f94c2..ddd290d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryPlan;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.InMemoryReservationAllocation;
@@ -78,9 +79,10 @@ public class TestSimpleCapacityReplanner {
     enf.init("blah", conf);
 
     // Initialize the plan with more resources
-    InMemoryPlan plan =
-        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
-            res, minAlloc, maxAlloc, "dedicated", enf, true, context, clock);
+    InMemoryPlan plan = new InMemoryPlan(queueMetrics, policy, agent,
+        clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", enf, true,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        context, clock);
 
     // add reservation filling the plan (separating them 1ms, so we are sure
     // s2 follows s1 on acceptance


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


[37/48] 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/5da7d3f4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5da7d3f4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5da7d3f4

Branch: refs/heads/HDFS-10467
Commit: 5da7d3f4fa5190107246ad3e9a8b15e1e0d30918
Parents: 9203e56
Author: Inigo Goiri <in...@apache.org>
Authored: Thu May 11 09:57:03 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:08 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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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


[11/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
new file mode 100644
index 0000000..6cff533
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.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.fs.s3a;
+
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.AmazonS3;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.junit.Assume;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
+
+/**
+ * Test S3Guard list consistency feature by injecting delayed listObjects()
+ * visibility via {@link InconsistentAmazonS3Client}.
+ *
+ * Tests here generally:
+ * 1. Use the inconsistency injection mentioned above.
+ * 2. Only run when S3Guard is enabled.
+ */
+public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    conf.setClass(S3_CLIENT_FACTORY_IMPL, InconsistentS3ClientFactory.class,
+        S3ClientFactory.class);
+    // Other configs would break test assumptions
+    conf.set(FAIL_INJECT_INCONSISTENCY_KEY, DEFAULT_DELAY_KEY_SUBSTRING);
+    conf.setFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY, 1.0f);
+    conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, DEFAULT_DELAY_KEY_MSEC);
+    return new S3AContract(conf);
+  }
+
+  /**
+   * Helper function for other test cases: does a single rename operation and
+   * validates the aftermath.
+   * @param mkdirs Directories to create
+   * @param srcdirs Source paths for rename operation
+   * @param dstdirs Destination paths for rename operation
+   * @param yesdirs Files that must exist post-rename (e.g. srcdirs children)
+   * @param nodirs Files that must not exist post-rename (e.g. dstdirs children)
+   * @throws Exception
+   */
+  private void doTestRenameSequence(Path[] mkdirs, Path[] srcdirs,
+      Path[] dstdirs, Path[] yesdirs, Path[] nodirs) throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    if (mkdirs != null) {
+      for (Path mkdir : mkdirs) {
+        assertTrue(fs.mkdirs(mkdir));
+      }
+      clearInconsistency(fs);
+    }
+
+    assertTrue("srcdirs and dstdirs must have equal length",
+        srcdirs.length == dstdirs.length);
+    for (int i = 0; i < srcdirs.length; i++) {
+      assertTrue("Rename returned false: " + srcdirs[i] + " -> " + dstdirs[i],
+          fs.rename(srcdirs[i], dstdirs[i]));
+    }
+
+    for (Path yesdir : yesdirs) {
+      assertTrue("Path was supposed to exist: " + yesdir, fs.exists(yesdir));
+    }
+    for (Path nodir : nodirs) {
+      assertFalse("Path is not supposed to exist: " + nodir, fs.exists(nodir));
+    }
+  }
+
+  /**
+   * Tests that after renaming a directory, the original directory and its
+   * contents are indeed missing and the corresponding new paths are visible.
+   * @throws Exception
+   */
+  @Test
+  public void testConsistentListAfterRename() throws Exception {
+    Path[] mkdirs = {
+      path("d1/f"),
+      path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING)
+    };
+    Path[] srcdirs = {path("d1")};
+    Path[] dstdirs = {path("d2")};
+    Path[] yesdirs = {path("d2"), path("d2/f"),
+        path("d2/f" + DEFAULT_DELAY_KEY_SUBSTRING)};
+    Path[] nodirs = {path("d1"), path("d1/f"),
+        path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING)};
+    doTestRenameSequence(mkdirs, srcdirs, dstdirs, yesdirs, nodirs);
+    getFileSystem().delete(path("d1"), true);
+    getFileSystem().delete(path("d2"), true);
+  }
+
+  /**
+   * Tests a circular sequence of renames to verify that overwriting recently
+   * deleted files and reading recently created files from rename operations
+   * works as expected.
+   * @throws Exception
+   */
+  @Test
+  public void testRollingRenames() throws Exception {
+    Path[] dir0 = {path("rolling/1")};
+    Path[] dir1 = {path("rolling/2")};
+    Path[] dir2 = {path("rolling/3")};
+    // These sets have to be in reverse order compared to the movement
+    Path[] setA = {dir1[0], dir0[0]};
+    Path[] setB = {dir2[0], dir1[0]};
+    Path[] setC = {dir0[0], dir2[0]};
+
+    for(int i = 0; i < 2; i++) {
+      Path[] firstSet = i == 0 ? setA : null;
+      doTestRenameSequence(firstSet, setA, setB, setB, dir0);
+      doTestRenameSequence(null, setB, setC, setC, dir1);
+      doTestRenameSequence(null, setC, setA, setA, dir2);
+    }
+
+    S3AFileSystem fs = getFileSystem();
+    assertFalse("Renaming deleted file should have failed",
+        fs.rename(dir2[0], dir1[0]));
+    assertTrue("Renaming over existing file should have succeeded",
+        fs.rename(dir1[0], dir0[0]));
+  }
+
+  /**
+   * Tests that deleted files immediately stop manifesting in list operations
+   * even when the effect in S3 is delayed.
+   * @throws Exception
+   */
+  @Test
+  public void testConsistentListAfterDelete() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    // test will fail if NullMetadataStore (the default) is configured: skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+    // in listObjects() results via InconsistentS3Client
+    Path inconsistentPath =
+        path("a/b/dir3-" + DEFAULT_DELAY_KEY_SUBSTRING);
+
+    Path[] testDirs = {path("a/b/dir1"),
+        path("a/b/dir2"),
+        inconsistentPath};
+
+    for (Path path : testDirs) {
+      assertTrue(fs.mkdirs(path));
+    }
+    clearInconsistency(fs);
+    for (Path path : testDirs) {
+      assertTrue(fs.delete(path, false));
+    }
+
+    FileStatus[] paths = fs.listStatus(path("a/b/"));
+    List<Path> list = new ArrayList<>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath());
+    }
+    assertFalse(list.contains(path("a/b/dir1")));
+    assertFalse(list.contains(path("a/b/dir2")));
+    // This should fail without S3Guard, and succeed with it.
+    assertFalse(list.contains(inconsistentPath));
+  }
+
+  /**
+   * Tests that rename immediately after files in the source directory are
+   * deleted results in exactly the correct set of destination files and none
+   * of the source files.
+   * @throws Exception
+   */
+  @Test
+  public void testConsistentRenameAfterDelete() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    // test will fail if NullMetadataStore (the default) is configured: skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+    // in listObjects() results via InconsistentS3Client
+    Path inconsistentPath =
+        path("a/b/dir3-" + DEFAULT_DELAY_KEY_SUBSTRING);
+
+    Path[] testDirs = {path("a/b/dir1"),
+        path("a/b/dir2"),
+        inconsistentPath};
+
+    for (Path path : testDirs) {
+      assertTrue(fs.mkdirs(path));
+    }
+    clearInconsistency(fs);
+    assertTrue(fs.delete(testDirs[1], false));
+    assertTrue(fs.delete(testDirs[2], false));
+
+    fs.rename(path("a"), path("a3"));
+    FileStatus[] paths = fs.listStatus(path("a3/b"));
+    List<Path> list = new ArrayList<>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath());
+    }
+    assertTrue(list.contains(path("a3/b/dir1")));
+    assertFalse(list.contains(path("a3/b/dir2")));
+    // This should fail without S3Guard, and succeed with it.
+    assertFalse(list.contains(path("a3/b/dir3-" +
+        DEFAULT_DELAY_KEY_SUBSTRING)));
+
+    try {
+      RemoteIterator<LocatedFileStatus> old = fs.listFilesAndEmptyDirectories(
+          path("a"), true);
+      fail("Recently renamed dir should not be visible");
+    } catch(FileNotFoundException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testConsistentListStatusAfterPut() throws Exception {
+
+    S3AFileSystem fs = getFileSystem();
+
+    // This test will fail if NullMetadataStore (the default) is configured:
+    // skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+    // in listObjects() results via InconsistentS3Client
+    Path inconsistentPath =
+        path("a/b/dir3-" + DEFAULT_DELAY_KEY_SUBSTRING);
+
+    Path[] testDirs = {path("a/b/dir1"),
+        path("a/b/dir2"),
+        inconsistentPath};
+
+    for (Path path : testDirs) {
+      assertTrue(fs.mkdirs(path));
+    }
+
+    FileStatus[] paths = fs.listStatus(path("a/b/"));
+    List<Path> list = new ArrayList<>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath());
+    }
+    assertTrue(list.contains(path("a/b/dir1")));
+    assertTrue(list.contains(path("a/b/dir2")));
+    // This should fail without S3Guard, and succeed with it.
+    assertTrue(list.contains(inconsistentPath));
+  }
+
+  /**
+   * Similar to {@link #testConsistentListStatusAfterPut()}, this tests that the
+   * FS listLocatedStatus() call will return consistent list.
+   */
+  @Test
+  public void testConsistentListLocatedStatusAfterPut() throws Exception {
+    final S3AFileSystem fs = getFileSystem();
+    // This test will fail if NullMetadataStore (the default) is configured:
+    // skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+    String rootDir = "doTestConsistentListLocatedStatusAfterPut";
+    fs.mkdirs(path(rootDir));
+
+    final int[] numOfPaths = {0, 1, 5};
+    for (int normalPathNum : numOfPaths) {
+      for (int delayedPathNum : new int[] {0, 2}) {
+        LOG.info("Testing with normalPathNum={}, delayedPathNum={}",
+            normalPathNum, delayedPathNum);
+        doTestConsistentListLocatedStatusAfterPut(fs, rootDir, normalPathNum,
+            delayedPathNum);
+      }
+    }
+  }
+
+  /**
+   * Helper method to implement the tests of consistent listLocatedStatus().
+   * @param fs The S3 file system from contract
+   * @param normalPathNum number paths listed directly from S3 without delaying
+   * @param delayedPathNum number paths listed with delaying
+   * @throws Exception
+   */
+  private void doTestConsistentListLocatedStatusAfterPut(S3AFileSystem fs,
+      String rootDir, int normalPathNum, int delayedPathNum) throws Exception {
+    final List<Path> testDirs = new ArrayList<>(normalPathNum + delayedPathNum);
+    int index = 0;
+    for (; index < normalPathNum; index++) {
+      testDirs.add(path(rootDir + "/dir-" +
+          index));
+    }
+    for (; index < normalPathNum + delayedPathNum; index++) {
+      // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+      // in listObjects() results via InconsistentS3Client
+      testDirs.add(path(rootDir + "/dir-" + index +
+          DEFAULT_DELAY_KEY_SUBSTRING));
+    }
+
+    for (Path path : testDirs) {
+      // delete the old test path (if any) so that when we call mkdirs() later,
+      // the to delay directories will be tracked via putObject() request.
+      fs.delete(path, true);
+      assertTrue(fs.mkdirs(path));
+    }
+
+    // this should return the union data from S3 and MetadataStore
+    final RemoteIterator<LocatedFileStatus> statusIterator =
+        fs.listLocatedStatus(path(rootDir + "/"));
+    List<Path> list = new ArrayList<>();
+    for (; statusIterator.hasNext();) {
+      list.add(statusIterator.next().getPath());
+    }
+
+    // This should fail without S3Guard, and succeed with it because part of the
+    // children under test path are delaying visibility
+    for (Path path : testDirs) {
+      assertTrue("listLocatedStatus should list " + path, list.contains(path));
+    }
+  }
+
+  /**
+   * Tests that the S3AFS listFiles() call will return consistent file list.
+   */
+  @Test
+  public void testConsistentListFiles() throws Exception {
+    final S3AFileSystem fs = getFileSystem();
+    // This test will fail if NullMetadataStore (the default) is configured:
+    // skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    final int[] numOfPaths = {0, 2};
+    for (int dirNum : numOfPaths) {
+      for (int normalFile : numOfPaths) {
+        for (int delayedFile : new int[] {0, 1}) {
+          for (boolean recursive : new boolean[] {true, false}) {
+            doTestListFiles(fs, dirNum, normalFile, delayedFile, recursive);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Helper method to implement the tests of consistent listFiles().
+   *
+   * The file structure has dirNum subdirectories, and each directory (including
+   * the test base directory itself) has normalFileNum normal files and
+   * delayedFileNum delayed files.
+   *
+   * @param fs The S3 file system from contract
+   * @param dirNum number of subdirectories
+   * @param normalFileNum number files in each directory without delay to list
+   * @param delayedFileNum number files in each directory with delay to list
+   * @param recursive listFiles recursively if true
+   * @throws Exception if any unexpected error
+   */
+  private void doTestListFiles(S3AFileSystem fs, int dirNum, int normalFileNum,
+      int delayedFileNum, boolean recursive) throws Exception {
+    describe("Testing dirNum=%d, normalFile=%d, delayedFile=%d, "
+        + "recursive=%s", dirNum, normalFileNum, delayedFileNum, recursive);
+    final Path baseTestDir = path("doTestListFiles-" + dirNum + "-"
+        + normalFileNum + "-" + delayedFileNum + "-" + recursive);
+    // delete the old test path (if any) so that when we call mkdirs() later,
+    // the to delay sub directories will be tracked via putObject() request.
+    fs.delete(baseTestDir, true);
+
+    // make subdirectories (if any)
+    final List<Path> testDirs = new ArrayList<>(dirNum + 1);
+    assertTrue(fs.mkdirs(baseTestDir));
+    testDirs.add(baseTestDir);
+    for (int i = 0; i < dirNum; i++) {
+      final Path subdir = path(baseTestDir + "/dir-" + i);
+      assertTrue(fs.mkdirs(subdir));
+      testDirs.add(subdir);
+    }
+
+    final Collection<String> fileNames
+        = new ArrayList<>(normalFileNum + delayedFileNum);
+    int index = 0;
+    for (; index < normalFileNum; index++) {
+      fileNames.add("file-" + index);
+    }
+    for (; index < normalFileNum + delayedFileNum; index++) {
+      // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+      // in listObjects() results via InconsistentS3Client
+      fileNames.add("file-" + index + "-" + DEFAULT_DELAY_KEY_SUBSTRING);
+    }
+
+    int filesAndEmptyDirectories = 0;
+
+    // create files under each test directory
+    for (Path dir : testDirs) {
+      for (String fileName : fileNames) {
+        writeTextFile(fs, new Path(dir, fileName), "I, " + fileName, false);
+        filesAndEmptyDirectories++;
+      }
+    }
+
+    // this should return the union data from S3 and MetadataStore
+    final RemoteIterator<LocatedFileStatus> statusIterator
+        = fs.listFiles(baseTestDir, recursive);
+    final Collection<Path> listedFiles = new HashSet<>();
+    for (; statusIterator.hasNext();) {
+      final FileStatus status = statusIterator.next();
+      assertTrue("FileStatus " + status + " is not a file!", status.isFile());
+      listedFiles.add(status.getPath());
+    }
+    LOG.info("S3AFileSystem::listFiles('{}', {}) -> {}",
+        baseTestDir, recursive, listedFiles);
+
+    // This should fail without S3Guard, and succeed with it because part of the
+    // files to list are delaying visibility
+    if (!recursive) {
+      // in this case only the top level files are listed
+      assertEquals("Unexpected number of files returned by listFiles() call",
+          normalFileNum + delayedFileNum, listedFiles.size());
+      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
+    } else {
+      assertEquals("Unexpected number of files returned by listFiles() call",
+          filesAndEmptyDirectories,
+          listedFiles.size());
+      for (Path dir : testDirs) {
+        verifyFileIsListed(listedFiles, dir, fileNames);
+      }
+    }
+  }
+
+  private static void verifyFileIsListed(Collection<Path> listedFiles,
+      Path currentDir, Collection<String> fileNames) {
+    for (String fileName : fileNames) {
+      final Path file = new Path(currentDir, fileName);
+      assertTrue(file + " should have been listed", listedFiles.contains(file));
+    }
+  }
+
+  @Test
+  public void testCommitByRenameOperations() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+    Path work = path("test-commit-by-rename-" + DEFAULT_DELAY_KEY_SUBSTRING);
+    Path task00 = new Path(work, "task00");
+    fs.mkdirs(task00);
+    String name = "part-00";
+    try (FSDataOutputStream out =
+             fs.create(new Path(task00, name), false)) {
+      out.writeChars("hello");
+    }
+    for (FileStatus stat : fs.listStatus(task00)) {
+      fs.rename(stat.getPath(), work);
+    }
+    List<FileStatus> files = new ArrayList<>(2);
+    for (FileStatus stat : fs.listStatus(work)) {
+      if (stat.isFile()) {
+        files.add(stat);
+      }
+    }
+    assertFalse("renamed file " + name + " not found in " + work,
+        files.isEmpty());
+    assertEquals("more files found than expected in " + work
+        + " " + ls(work), 1, files.size());
+    FileStatus status = files.get(0);
+    assertEquals("Wrong filename in " + status,
+        name, status.getPath().getName());
+  }
+
+  @Test
+  public void testInconsistentS3ClientDeletes() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    Path root = path("testInconsistentClient" + DEFAULT_DELAY_KEY_SUBSTRING);
+    for (int i = 0; i < 3; i++) {
+      fs.mkdirs(new Path(root, "dir" + i));
+      touch(fs, new Path(root, "file" + i));
+      for (int j = 0; j < 3; j++) {
+        touch(fs, new Path(new Path(root, "dir" + i), "file" + i + "-" + j));
+      }
+    }
+    clearInconsistency(fs);
+
+    AmazonS3 client = fs.getAmazonS3Client();
+    String key = fs.pathToKey(root) + "/";
+
+    ObjectListing preDeleteDelimited = client.listObjects(
+        fs.createListObjectsRequest(key, "/"));
+    ObjectListing preDeleteUndelimited = client.listObjects(
+        fs.createListObjectsRequest(key, null));
+
+    fs.delete(root, true);
+
+    ObjectListing postDeleteDelimited = client.listObjects(
+        fs.createListObjectsRequest(key, "/"));
+    ObjectListing postDeleteUndelimited = client.listObjects(
+        fs.createListObjectsRequest(key, null));
+
+    assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
+            "in a non-recursive listing",
+        preDeleteDelimited.getObjectSummaries().size(),
+        postDeleteDelimited.getObjectSummaries().size()
+    );
+    assertEquals("InconsistentAmazonS3Client added back prefixes incorrectly " +
+            "in a non-recursive listing",
+        preDeleteDelimited.getCommonPrefixes().size(),
+        postDeleteDelimited.getCommonPrefixes().size()
+    );
+    assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
+            "in a recursive listing",
+        preDeleteUndelimited.getObjectSummaries().size(),
+        postDeleteUndelimited.getObjectSummaries().size()
+    );
+    assertEquals("InconsistentAmazonS3Client added back prefixes incorrectly " +
+            "in a recursive listing",
+        preDeleteUndelimited.getCommonPrefixes().size(),
+        postDeleteUndelimited.getCommonPrefixes().size()
+    );
+  }
+
+  private static void clearInconsistency(S3AFileSystem fs) throws Exception {
+    AmazonS3 s3 = fs.getAmazonS3Client();
+    InconsistentAmazonS3Client ic = InconsistentAmazonS3Client.castFrom(s3);
+    ic.clearInconsistency();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
new file mode 100644
index 0000000..a63b696
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.junit.Assume;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+
+/**
+ * Test cases that validate S3Guard's behavior for writing things like
+ * directory listings back to the MetadataStore.
+ */
+public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
+
+  /**
+   * In listStatus(), when S3Guard is enabled, the full listing for a
+   * directory is "written back" to the MetadataStore before the listing is
+   * returned.  Currently this "write back" behavior occurs when
+   * fs.s3a.metadatastore.authoritative is true.  This test validates this
+   * behavior.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testListStatusWriteBack() throws Exception {
+    Assume.assumeTrue(getFileSystem().hasMetadataStore());
+
+    Path directory = path("ListStatusWriteBack");
+
+    // "raw" S3AFileSystem without S3Guard
+    S3AFileSystem noS3Guard = createTestFS(directory.toUri(), true, false);
+
+    // Another with S3Guard and write-back disabled
+    S3AFileSystem noWriteBack = createTestFS(directory.toUri(), false, false);
+
+    // Another S3Guard and write-back enabled
+    S3AFileSystem yesWriteBack = createTestFS(directory.toUri(), false, true);
+
+    // delete the existing directory (in case of last test failure)
+    noS3Guard.delete(directory, true);
+    // Create a directory on S3 only
+    noS3Guard.mkdirs(new Path(directory, "OnS3"));
+    // Create a directory on both S3 and metadata store
+    Path p = new Path(directory, "OnS3AndMS");
+    assertPathDoesntExist(noWriteBack, p);
+    noWriteBack.mkdirs(p);
+
+    FileStatus[] fsResults;
+    DirListingMetadata mdResults;
+
+    // FS should return both even though S3Guard is not writing back to MS
+    fsResults = noWriteBack.listStatus(directory);
+    assertEquals("Filesystem enabled S3Guard without write back should have "
+            + "both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
+        2, fsResults.length);
+
+    // Metadata store without write-back should still only contain /OnS3AndMS,
+    // because newly discovered /OnS3 is not written back to metadata store
+    mdResults = noWriteBack.getMetadataStore().listChildren(directory);
+    assertEquals("Metadata store without write back should still only know "
+            + "about /OnS3AndMS, but it has: " + mdResults,
+        1, mdResults.numEntries());
+
+    // FS should return both (and will write it back)
+    fsResults = yesWriteBack.listStatus(directory);
+    assertEquals("Filesystem enabled S3Guard with write back should have "
+            + " both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
+        2, fsResults.length);
+
+    // Metadata store with write-back should contain both because the newly
+    // discovered /OnS3 should have been written back to metadata store
+    mdResults = yesWriteBack.getMetadataStore().listChildren(directory);
+    assertEquals("Unexpected number of results from metadata store. "
+            + "Should have /OnS3 and /OnS3AndMS: " + mdResults,
+        2, mdResults.numEntries());
+
+    // If we don't clean this up, the next test run will fail because it will
+    // have recorded /OnS3 being deleted even after it's written to noS3Guard.
+    getFileSystem().getMetadataStore().forgetMetadata(
+        new Path(directory, "OnS3"));
+  }
+
+  /** Create a separate S3AFileSystem instance for testing. */
+  private S3AFileSystem createTestFS(URI fsURI, boolean disableS3Guard,
+      boolean authoritativeMeta) throws IOException {
+    Configuration conf;
+
+    // Create a FileSystem that is S3-backed only
+    conf = createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    if (disableS3Guard) {
+      conf.set(Constants.S3_METADATA_STORE_IMPL,
+          Constants.S3GUARD_METASTORE_NULL);
+    } else {
+      S3ATestUtils.maybeEnableS3Guard(conf);
+      conf.setBoolean(Constants.METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+    }
+    FileSystem fs = FileSystem.get(fsURI, conf);
+    return asS3AFS(fs);
+  }
+
+  private static S3AFileSystem asS3AFS(FileSystem fs) {
+    assertTrue("Not a S3AFileSystem: " + fs, fs instanceof S3AFileSystem);
+    return (S3AFileSystem)fs;
+  }
+
+  private static void assertPathDoesntExist(FileSystem fs, Path p)
+      throws IOException {
+    try {
+      FileStatus s = fs.getFileStatus(p);
+    } catch (FileNotFoundException e) {
+      return;
+    }
+    fail("Path should not exist: " + p);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
index 9e0a5e4..4e25380 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -23,6 +23,7 @@ import static org.mockito.Mockito.*;
 import java.net.URI;
 
 import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.Region;
 
 /**
  * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
@@ -35,6 +36,8 @@ public class MockS3ClientFactory implements S3ClientFactory {
     String bucket = name.getHost();
     AmazonS3 s3 = mock(AmazonS3.class);
     when(s3.doesBucketExist(bucket)).thenReturn(true);
+    when(s3.getBucketLocation(anyString()))
+        .thenReturn(Region.US_West.toString());
     return s3;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
index acbe610..2c4f009 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
@@ -135,6 +135,18 @@ public interface S3ATestConstants {
   String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
 
   /**
+   * Various S3Guard tests.
+   */
+  String TEST_S3GUARD_PREFIX = "fs.s3a.s3guard.test";
+  String TEST_S3GUARD_ENABLED = TEST_S3GUARD_PREFIX + ".enabled";
+  String TEST_S3GUARD_AUTHORITATIVE = TEST_S3GUARD_PREFIX + ".authoritative";
+  String TEST_S3GUARD_IMPLEMENTATION = TEST_S3GUARD_PREFIX + ".implementation";
+  String TEST_S3GUARD_IMPLEMENTATION_LOCAL = "local";
+  String TEST_S3GUARD_IMPLEMENTATION_DYNAMO = "dynamo";
+  String TEST_S3GUARD_IMPLEMENTATION_DYNAMODBLOCAL = "dynamodblocal";
+  String TEST_S3GUARD_IMPLEMENTATION_NONE = "none";
+
+  /**
    * Timeout in Milliseconds for standard tests: {@value}.
    */
   int S3A_TEST_TIMEOUT = 10 * 60 * 1000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 9528967..8dbf90a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -22,7 +22,14 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBLocalClientFactory;
+import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
+
+import org.hamcrest.core.Is;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.internal.AssumptionViolatedException;
@@ -31,11 +38,13 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.List;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions;
 import static org.junit.Assert.*;
 
 /**
@@ -52,6 +61,15 @@ public final class S3ATestUtils {
   public static final String UNSET_PROPERTY = "unset";
 
   /**
+   * Get S3A FS name.
+   * @param conf configuration.
+   * @return S3A fs name.
+   */
+  public static String getFsName(Configuration conf) {
+    return conf.getTrimmed(TEST_FS_S3A_NAME, "");
+  }
+
+  /**
    * Create the test filesystem.
    *
    * If the test.fs.s3a.name property is not set, this will
@@ -97,6 +115,8 @@ public final class S3ATestUtils {
       throw new AssumptionViolatedException(
           "No test filesystem in " + TEST_FS_S3A_NAME);
     }
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
     S3AFileSystem fs1 = new S3AFileSystem();
     //enable purging in tests
     if (purge) {
@@ -137,6 +157,8 @@ public final class S3ATestUtils {
       throw new AssumptionViolatedException("No test filesystem in "
           + TEST_FS_S3A_NAME);
     }
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
     FileContext fc = FileContext.getFileContext(testURI, conf);
     return fc;
   }
@@ -301,13 +323,96 @@ public final class S3ATestUtils {
    * @return a path
    */
   public static Path createTestPath(Path defVal) {
-    String testUniqueForkId = System.getProperty(
-        S3ATestConstants.TEST_UNIQUE_FORK_ID);
+    String testUniqueForkId =
+        System.getProperty(S3ATestConstants.TEST_UNIQUE_FORK_ID);
     return testUniqueForkId == null ? defVal :
         new Path("/" + testUniqueForkId, "test");
   }
 
   /**
+   * Test assumption that S3Guard is/is not enabled.
+   * @param shouldBeEnabled should S3Guard be enabled?
+   * @param originalConf configuration to check
+   * @throws URISyntaxException
+   */
+  public static void assumeS3GuardState(boolean shouldBeEnabled,
+      Configuration originalConf) throws URISyntaxException {
+    boolean isEnabled = getTestPropertyBool(originalConf, TEST_S3GUARD_ENABLED,
+        originalConf.getBoolean(TEST_S3GUARD_ENABLED, false));
+    Assume.assumeThat("Unexpected S3Guard test state:"
+            + " shouldBeEnabled=" + shouldBeEnabled
+            + " and isEnabled=" + isEnabled,
+        shouldBeEnabled, Is.is(isEnabled));
+
+    final String fsname = originalConf.getTrimmed(TEST_FS_S3A_NAME);
+    Assume.assumeNotNull(fsname);
+    final String bucket = new URI(fsname).getHost();
+    final Configuration conf = propagateBucketOptions(originalConf, bucket);
+    boolean usingNullImpl = S3GUARD_METASTORE_NULL.equals(
+        conf.getTrimmed(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL));
+    Assume.assumeThat("Unexpected S3Guard test state:"
+            + " shouldBeEnabled=" + shouldBeEnabled
+            + " but usingNullImpl=" + usingNullImpl,
+        shouldBeEnabled, Is.is(!usingNullImpl));
+  }
+
+  /**
+   * Conditionally set the S3Guard options from test properties.
+   * @param conf configuration
+   */
+  public static void maybeEnableS3Guard(Configuration conf) {
+    if (getTestPropertyBool(conf, TEST_S3GUARD_ENABLED,
+        conf.getBoolean(TEST_S3GUARD_ENABLED, false))) {
+      // S3Guard is enabled.
+      boolean authoritative = getTestPropertyBool(conf,
+          TEST_S3GUARD_AUTHORITATIVE,
+          conf.getBoolean(TEST_S3GUARD_AUTHORITATIVE, true));
+      String impl = getTestProperty(conf, TEST_S3GUARD_IMPLEMENTATION,
+          conf.get(TEST_S3GUARD_IMPLEMENTATION,
+              TEST_S3GUARD_IMPLEMENTATION_LOCAL));
+      String implClass = "";
+      switch (impl) {
+      case TEST_S3GUARD_IMPLEMENTATION_LOCAL:
+        implClass = S3GUARD_METASTORE_LOCAL;
+        break;
+      case TEST_S3GUARD_IMPLEMENTATION_DYNAMODBLOCAL:
+        conf.setClass(S3Guard.S3GUARD_DDB_CLIENT_FACTORY_IMPL,
+            DynamoDBLocalClientFactory.class, DynamoDBClientFactory.class);
+      case TEST_S3GUARD_IMPLEMENTATION_DYNAMO:
+        implClass = S3GUARD_METASTORE_DYNAMO;
+        break;
+      case TEST_S3GUARD_IMPLEMENTATION_NONE:
+        implClass = S3GUARD_METASTORE_NULL;
+        break;
+      default:
+        fail("Unknown s3guard back end: \"" + impl + "\"");
+      }
+      LOG.debug("Enabling S3Guard, authoritative={}, implementation={}",
+          authoritative, implClass);
+      conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritative);
+      conf.set(S3_METADATA_STORE_IMPL, implClass);
+      conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+    }
+  }
+
+  /**
+   * Is there a MetadataStore configured for s3a with authoritative enabled?
+   * @param conf Configuration to test.
+   * @return true iff there is a MetadataStore configured, and it is
+   * configured allow authoritative results.  This can result in reducing
+   * round trips to S3 service for cached results, which may affect FS/FC
+   * statistics.
+   */
+  public static boolean isMetadataStoreAuthoritative(Configuration conf) {
+    if (conf == null) {
+      return Constants.DEFAULT_METADATASTORE_AUTHORITATIVE;
+    }
+    return conf.getBoolean(
+        Constants.METADATASTORE_AUTHORITATIVE,
+        Constants.DEFAULT_METADATASTORE_AUTHORITATIVE);
+  }
+
+  /**
    * Reset all metrics in a list.
    * @param metrics metrics to reset
    */
@@ -504,6 +609,94 @@ public final class S3ATestUtils {
   }
 
   /**
+   * Verify the core size, block size and timestamp values of a file.
+   * @param status status entry to check
+   * @param size file size
+   * @param blockSize block size
+   * @param modTime modified time
+   */
+  public static void verifyFileStatus(FileStatus status, long size,
+      long blockSize, long modTime) {
+    verifyFileStatus(status, size, 0, modTime, 0, blockSize, null, null, null);
+  }
+
+  /**
+   * Verify the status entry of a file matches that expected.
+   * @param status status entry to check
+   * @param size file size
+   * @param replication replication factor (may be 0)
+   * @param modTime modified time
+   * @param accessTime access time (may be 0)
+   * @param blockSize block size
+   * @param owner owner (may be null)
+   * @param group user group (may be null)
+   * @param permission permission (may be null)
+   */
+  public static void verifyFileStatus(FileStatus status,
+      long size,
+      int replication,
+      long modTime,
+      long accessTime,
+      long blockSize,
+      String owner,
+      String group,
+      FsPermission permission) {
+    String details = status.toString();
+    assertFalse("Not a dir: " + details, status.isDirectory());
+    assertEquals("Mod time: " + details, modTime, status.getModificationTime());
+    assertEquals("File size: " + details, size, status.getLen());
+    assertEquals("Block size: " + details, blockSize, status.getBlockSize());
+    if (replication > 0) {
+      assertEquals("Replication value: " + details, replication,
+          status.getReplication());
+    }
+    if (accessTime != 0) {
+      assertEquals("Access time: " + details, accessTime,
+          status.getAccessTime());
+    }
+    if (owner != null) {
+      assertEquals("Owner: " + details, owner, status.getOwner());
+    }
+    if (group != null) {
+      assertEquals("Group: " + details, group, status.getGroup());
+    }
+    if (permission != null) {
+      assertEquals("Permission: " + details, permission,
+          status.getPermission());
+    }
+  }
+
+  /**
+   * Verify the status entry of a directory matches that expected.
+   * @param status status entry to check
+   * @param replication replication factor
+   * @param modTime modified time
+   * @param accessTime access time
+   * @param owner owner
+   * @param group user group
+   * @param permission permission.
+   */
+  public static void verifyDirStatus(FileStatus status,
+      int replication,
+      long modTime,
+      long accessTime,
+      String owner,
+      String group,
+      FsPermission permission) {
+    String details = status.toString();
+    assertTrue("Is a dir: " + details, status.isDirectory());
+    assertEquals("zero length: " + details, 0, status.getLen());
+
+    assertEquals("Mod time: " + details, modTime, status.getModificationTime());
+    assertEquals("Replication value: " + details, replication,
+        status.getReplication());
+    assertEquals("Access time: " + details, accessTime, status.getAccessTime());
+    assertEquals("Owner: " + details, owner, status.getOwner());
+    assertEquals("Group: " + details, group, status.getGroup());
+    assertEquals("Permission: " + details, permission, status.getPermission());
+  }
+
+  /**
    * Set a bucket specific property to a particular value.
    * If the generic key passed in has an {@code fs.s3a. prefix},
    * that's stripped off, so that when the the bucket properties are propagated

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java
new file mode 100644
index 0000000..e647327
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java
@@ -0,0 +1,118 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+import static org.apache.hadoop.fs.s3a.Listing.ACCEPT_ALL;
+import static org.apache.hadoop.fs.s3a.Listing.ProvidedFileStatusIterator;
+
+/**
+ * Place for the S3A listing classes; keeps all the small classes under control.
+ */
+public class TestListing extends AbstractS3AMockTest {
+
+  private static class MockRemoteIterator<FileStatus> implements
+      RemoteIterator<FileStatus> {
+    private Iterator<FileStatus> iterator;
+
+    MockRemoteIterator(Collection<FileStatus> source) {
+      iterator = source.iterator();
+    }
+
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    public FileStatus next() {
+      return iterator.next();
+    }
+  }
+
+  private FileStatus blankFileStatus(Path path) {
+    return new FileStatus(0, true, 0, 0, 0, path);
+  }
+
+  @Test
+  public void testTombstoneReconcilingIterator() throws Exception {
+    Path parent = new Path("/parent");
+    Path liveChild = new Path(parent, "/liveChild");
+    Path deletedChild = new Path(parent, "/deletedChild");
+    Path[] allFiles = {parent, liveChild, deletedChild};
+    Path[] liveFiles = {parent, liveChild};
+
+    Listing listing = new Listing(fs);
+    Collection<FileStatus> statuses = new ArrayList<>();
+    statuses.add(blankFileStatus(parent));
+    statuses.add(blankFileStatus(liveChild));
+    statuses.add(blankFileStatus(deletedChild));
+
+    Set<Path> tombstones = new HashSet<>();
+    tombstones.add(deletedChild);
+
+    RemoteIterator<FileStatus> sourceIterator = new MockRemoteIterator(
+        statuses);
+    RemoteIterator<LocatedFileStatus> locatedIterator =
+        listing.createLocatedFileStatusIterator(sourceIterator);
+    RemoteIterator<LocatedFileStatus> reconcilingIterator =
+        listing.createTombstoneReconcilingIterator(locatedIterator, tombstones);
+
+    Set<Path> expectedPaths = new HashSet<>();
+    expectedPaths.add(parent);
+    expectedPaths.add(liveChild);
+
+    Set<Path> actualPaths = new HashSet<>();
+    while (reconcilingIterator.hasNext()) {
+      actualPaths.add(reconcilingIterator.next().getPath());
+    }
+    Assert.assertTrue(actualPaths.equals(expectedPaths));
+  }
+
+  @Test
+  public void testProvidedFileStatusIteratorEnd() throws Exception {
+    FileStatus[] statuses = {
+        new FileStatus(100, false, 1, 8192, 0, new Path("s3a://blah/blah"))
+    };
+    ProvidedFileStatusIterator it = new ProvidedFileStatusIterator(statuses,
+        ACCEPT_ALL, new Listing.AcceptAllButS3nDirs());
+
+    Assert.assertTrue("hasNext() should return true first time", it.hasNext());
+    Assert.assertNotNull("first element should not be null", it.next());
+    Assert.assertFalse("hasNext() should now be false", it.hasNext());
+    try {
+      it.next();
+      Assert.fail("next() should have thrown exception");
+    } catch (NoSuchElementException e) {
+      // Correct behavior.  Any other exceptions are propagated as failure.
+      return;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
index e1aef75..e493818 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
@@ -39,7 +39,9 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest {
 
   @After
   public void tearDown() throws Exception {
-    fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
+    if (fc != null) {
+      fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
index fff1fcb..725646c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
@@ -16,19 +16,29 @@ package org.apache.hadoop.fs.s3a.fileContext;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContextURIBase;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem;
+
 /**
  * S3a implementation of FileContextURIBase.
  */
 public class ITestS3AFileContextURI extends FileContextURIBase {
 
+  private Configuration conf;
+  private boolean hasMetadataStore;
+
   @Before
   public void setUp() throws IOException, Exception {
-    Configuration conf = new Configuration();
+    conf = new Configuration();
+    try(S3AFileSystem s3aFS = createTestFileSystem(conf)) {
+      hasMetadataStore = s3aFS.hasMetadataStore();
+    }
     fc1 = S3ATestUtils.createTestFileContext(conf);
     fc2 = S3ATestUtils.createTestFileContext(conf); //different object, same FS
     super.setUp();
@@ -41,4 +51,11 @@ public class ITestS3AFileContextURI extends FileContextURIBase {
     // (the statistics tested with this method are not relevant for an S3FS)
   }
 
+  @Test
+  @Override
+  public void testModificationTime() throws IOException {
+    // skip modtime tests as there may be some inconsistency during creation
+    assume("modification time tests are skipped", !hasMetadataStore);
+    super.testModificationTime();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java
new file mode 100644
index 0000000..921d4a6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.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.fs.s3a.s3guard;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/**
+ * Test specification for MetadataStore contract tests. Supplies configuration
+ * and MetadataStore instance.
+ */
+public abstract class AbstractMSContract {
+
+  public abstract FileSystem getFileSystem() throws IOException;
+  public abstract MetadataStore getMetadataStore() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
new file mode 100644
index 0000000..ceacdf3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+
+/**
+ * Common functionality for S3GuardTool test cases.
+ */
+public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
+
+  protected static final String OWNER = "hdfs";
+
+  private MetadataStore ms;
+
+  protected static void expectResult(int expected,
+      String message,
+      S3GuardTool tool,
+      String... args) throws Exception {
+    assertEquals(message, expected, tool.run(args));
+  }
+
+  protected static void expectSuccess(
+      String message,
+      S3GuardTool tool,
+      String... args) throws Exception {
+    assertEquals(message, SUCCESS, tool.run(args));
+  }
+
+  protected MetadataStore getMetadataStore() {
+    return ms;
+  }
+
+  protected abstract MetadataStore newMetadataStore();
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    S3ATestUtils.assumeS3GuardState(true, getConfiguration());
+    ms = newMetadataStore();
+    ms.initialize(getFileSystem());
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.cleanupWithLogger(LOG, ms);
+  }
+
+  protected void mkdirs(Path path, boolean onS3, boolean onMetadataStore)
+      throws IOException {
+    if (onS3) {
+      getFileSystem().mkdirs(path);
+    }
+    if (onMetadataStore) {
+      S3AFileStatus status = new S3AFileStatus(true, path, OWNER);
+      ms.put(new PathMetadata(status));
+    }
+  }
+
+  protected static void putFile(MetadataStore ms, S3AFileStatus f)
+      throws IOException {
+    assertNotNull(f);
+    ms.put(new PathMetadata(f));
+    Path parent = f.getPath().getParent();
+    while (parent != null) {
+      S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner());
+      ms.put(new PathMetadata(dir));
+      parent = parent.getParent();
+    }
+  }
+
+  /**
+   * Create file either on S3 or in metadata store.
+   * @param path the file path.
+   * @param onS3 set to true to create the file on S3.
+   * @param onMetadataStore set to true to create the file on the
+   *                        metadata store.
+   * @throws IOException IO problem
+   */
+  protected void createFile(Path path, boolean onS3, boolean onMetadataStore)
+      throws IOException {
+    if (onS3) {
+      ContractTestUtils.touch(getFileSystem(), path);
+    }
+
+    if (onMetadataStore) {
+      S3AFileStatus status = new S3AFileStatus(100L, System.currentTimeMillis(),
+          getFileSystem().qualify(path), 512L, "hdfs");
+      putFile(ms, status);
+    }
+  }
+
+  private void testPruneCommand(Configuration cmdConf, String...args)
+      throws Exception {
+    Path parent = path("prune-cli");
+    try {
+      getFileSystem().mkdirs(parent);
+
+      S3GuardTool.Prune cmd = new S3GuardTool.Prune(cmdConf);
+      cmd.setMetadataStore(ms);
+
+      createFile(new Path(parent, "stale"), true, true);
+      Thread.sleep(TimeUnit.SECONDS.toMillis(2));
+      createFile(new Path(parent, "fresh"), true, true);
+
+      assertEquals(2, ms.listChildren(parent).getListing().size());
+      expectSuccess("Prune command did not exit successfully - see output", cmd,
+          args);
+      assertEquals(1, ms.listChildren(parent).getListing().size());
+    } finally {
+      getFileSystem().delete(parent, true);
+      ms.prune(Long.MAX_VALUE);
+    }
+  }
+
+  @Test
+  public void testPruneCommandCLI() throws Exception {
+    String testPath = path("testPruneCommandCLI").toString();
+    testPruneCommand(getFileSystem().getConf(),
+        "prune", "-seconds", "1", testPath);
+  }
+
+  @Test
+  public void testPruneCommandConf() throws Exception {
+    getConfiguration().setLong(Constants.S3GUARD_CLI_PRUNE_AGE,
+        TimeUnit.SECONDS.toMillis(1));
+    String testPath = path("testPruneCommandConf").toString();
+    testPruneCommand(getConfiguration(), "prune", testPath);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java
new file mode 100644
index 0000000..0291acd
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java
@@ -0,0 +1,157 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.File;
+import java.io.IOException;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
+import com.amazonaws.services.dynamodbv2.local.main.ServerRunner;
+import com.amazonaws.services.dynamodbv2.local.server.DynamoDBProxyServer;
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
+import org.apache.hadoop.net.ServerSocketUtil;
+
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory.DefaultDynamoDBClientFactory.getRegion;
+
+/**
+ * A DynamoDBClientFactory implementation that creates AmazonDynamoDB clients
+ * against an in-memory DynamoDBLocal server instance.
+ *
+ * You won't be charged bills for issuing any DynamoDB requests. However, the
+ * DynamoDBLocal is considered a simulator of the DynamoDB web service, so it
+ * may be stale or different. For example, the throttling is not yet supported
+ * in DynamoDBLocal. This is for testing purpose only.
+ *
+ * To use this for creating DynamoDB client in tests:
+ * <ol>
+ * <li>
+ *    As all DynamoDBClientFactory implementations, this should be configured.
+ * </li>
+ * <li>
+ *    The singleton DynamoDBLocal server instance is started automatically when
+ *    creating the AmazonDynamoDB client for the first time. It still merits to
+ *    launch the server before all the tests and fail fast if error happens.
+ * </li>
+ * <li>
+ *    The server can be stopped explicitly, which is not actually needed in
+ *    tests as JVM termination will do that.
+ * </li>
+ * </ol>
+ *
+ * @see DefaultDynamoDBClientFactory
+ */
+public class DynamoDBLocalClientFactory extends Configured
+    implements DynamoDBClientFactory {
+
+  /** The DynamoDBLocal dynamoDBLocalServer instance for testing. */
+  private static DynamoDBProxyServer dynamoDBLocalServer;
+  private static String ddbEndpoint;
+
+  private static final String SYSPROP_SQLITE_LIB = "sqlite4java.library.path";
+
+  @Override
+  public AmazonDynamoDB createDynamoDBClient(String defaultRegion)
+      throws IOException {
+    startSingletonServer();
+
+    final Configuration conf = getConf();
+    final AWSCredentialsProvider credentials =
+        createAWSCredentialProviderSet(null, conf);
+    final ClientConfiguration awsConf =
+        DefaultS3ClientFactory.createAwsConf(conf);
+    // fail fast in case of service errors
+    awsConf.setMaxErrorRetry(3);
+
+    final String region = getRegion(conf, defaultRegion);
+    LOG.info("Creating DynamoDBLocal client using endpoint {} in region {}",
+        ddbEndpoint, region);
+
+    return AmazonDynamoDBClientBuilder.standard()
+        .withCredentials(credentials)
+        .withClientConfiguration(awsConf)
+        .withEndpointConfiguration(
+            new AwsClientBuilder.EndpointConfiguration(ddbEndpoint, region))
+        .build();
+  }
+
+  /**
+   * Start a singleton in-memory DynamoDBLocal server if not started yet.
+   * @throws IOException if any error occurs
+   */
+  public synchronized static void startSingletonServer() throws IOException {
+    if (dynamoDBLocalServer != null) {
+      return;
+    }
+
+    // Set this property if it has not been set elsewhere
+    if (StringUtils.isEmpty(System.getProperty(SYSPROP_SQLITE_LIB))) {
+      String projectBuildDir = System.getProperty("project.build.directory");
+      if (StringUtils.isEmpty(projectBuildDir)) {
+        projectBuildDir = "target";
+      }
+      // sqlite4java lib should have been copied to $projectBuildDir/native-libs
+      System.setProperty(SYSPROP_SQLITE_LIB,
+          projectBuildDir + File.separator + "native-libs");
+      LOG.info("Setting {} -> {}",
+          SYSPROP_SQLITE_LIB, System.getProperty(SYSPROP_SQLITE_LIB));
+    }
+
+    try {
+      // Start an in-memory local DynamoDB instance
+      final String port = String.valueOf(ServerSocketUtil.getPort(0, 100));
+      ddbEndpoint = "http://localhost:" + port;
+      dynamoDBLocalServer = ServerRunner.createServerFromCommandLineArgs(
+          new String[]{"-inMemory", "-port", port});
+      dynamoDBLocalServer.start();
+      LOG.info("DynamoDBLocal singleton server was started at {}", ddbEndpoint);
+    } catch (Exception t) {
+      String msg = "Error starting DynamoDBLocal server at " + ddbEndpoint
+          + " " + t;
+      LOG.error(msg, t);
+      throw new IOException(msg, t);
+    }
+  }
+
+  /**
+   * Stop the in-memory DynamoDBLocal server if it is started.
+   * @throws IOException if any error occurs
+   */
+  public synchronized static void stopSingletonServer() throws IOException {
+    if (dynamoDBLocalServer != null) {
+      LOG.info("Shutting down the in-memory DynamoDBLocal server");
+      try {
+        dynamoDBLocalServer.stop();
+      } catch (Throwable t) {
+        String msg = "Error stopping DynamoDBLocal server at " + ddbEndpoint;
+        LOG.error(msg, t);
+        throw new IOException(msg, t);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
new file mode 100644
index 0000000..c6838a0
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import org.junit.Assume;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.Constants;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
+
+/**
+ * Tests concurrent operations on S3Guard.
+ */
+public class ITestS3GuardConcurrentOps extends AbstractS3ATestBase {
+
+  @Rule
+  public final Timeout timeout = new Timeout(5 * 60 * 1000);
+
+  private void failIfTableExists(DynamoDB db, String tableName) {
+    boolean tableExists = true;
+    try {
+      Table table = db.getTable(tableName);
+      table.describe();
+    } catch (ResourceNotFoundException e) {
+      tableExists = false;
+    }
+    if (tableExists) {
+      fail("Table already exists: " + tableName);
+    }
+  }
+
+  private void deleteTable(DynamoDB db, String tableName) throws
+      InterruptedException {
+    try {
+      Table table = db.getTable(tableName);
+      table.waitForActive();
+      table.delete();
+      table.waitForDelete();
+    } catch (ResourceNotFoundException e) {
+      LOG.warn("Failed to delete {}, as it was not found", tableName, e);
+    }
+  }
+
+  @Test
+  public void testConcurrentTableCreations() throws Exception {
+    final Configuration conf = getConfiguration();
+    Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard",
+        conf.get(Constants.S3_METADATA_STORE_IMPL).equals(
+            Constants.S3GUARD_METASTORE_DYNAMO));
+
+    DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+    ms.initialize(getFileSystem());
+    DynamoDB db = ms.getDynamoDB();
+
+    String tableName = "testConcurrentTableCreations" + new Random().nextInt();
+    conf.setBoolean(Constants.S3GUARD_DDB_TABLE_CREATE_KEY, true);
+    conf.set(Constants.S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+
+    String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+    if (StringUtils.isEmpty(region)) {
+      // no region set, so pick it up from the test bucket
+      conf.set(S3GUARD_DDB_REGION_KEY, getFileSystem().getBucketLocation());
+    }
+    int concurrentOps = 16;
+    int iterations = 4;
+
+    failIfTableExists(db, tableName);
+
+    for (int i = 0; i < iterations; i++) {
+      ExecutorService executor = Executors.newFixedThreadPool(
+          concurrentOps, new ThreadFactory() {
+            private AtomicInteger count = new AtomicInteger(0);
+
+            public Thread newThread(Runnable r) {
+              return new Thread(r,
+                  "testConcurrentTableCreations" + count.getAndIncrement());
+            }
+          });
+      ((ThreadPoolExecutor) executor).prestartAllCoreThreads();
+      Future<Exception>[] futures = new Future[concurrentOps];
+      for (int f = 0; f < concurrentOps; f++) {
+        final int index = f;
+        futures[f] = executor.submit(new Callable<Exception>() {
+          @Override
+          public Exception call() throws Exception {
+
+            ContractTestUtils.NanoTimer timer =
+                new ContractTestUtils.NanoTimer();
+
+            Exception result = null;
+            try (DynamoDBMetadataStore store = new DynamoDBMetadataStore()) {
+              store.initialize(conf);
+            } catch (Exception e) {
+              LOG.error(e.getClass() + ": " + e.getMessage());
+              result = e;
+            }
+
+            timer.end("Parallel DynamoDB client creation %d", index);
+            LOG.info("Parallel DynamoDB client creation {} ran from {} to {}",
+                index, timer.getStartTime(), timer.getEndTime());
+            return result;
+          }
+        });
+      }
+      List<Exception> exceptions = new ArrayList<>(concurrentOps);
+      for (int f = 0; f < concurrentOps; f++) {
+        Exception outcome = futures[f].get();
+        if (outcome != null) {
+          exceptions.add(outcome);
+        }
+      }
+      deleteTable(db, tableName);
+      int exceptionsThrown = exceptions.size();
+      if (exceptionsThrown > 0) {
+        // at least one exception was thrown. Fail the test & nest the first
+        // exception caught
+        throw new AssertionError(exceptionsThrown + "/" + concurrentOps +
+            " threads threw exceptions while initializing on iteration " + i,
+            exceptions.get(0));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
new file mode 100644
index 0000000..c13dfc4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Destroy;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Init;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+/**
+ * Test S3Guard related CLI commands against DynamoDB.
+ */
+public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
+
+  @Override
+  protected MetadataStore newMetadataStore() {
+    return new DynamoDBMetadataStore();
+  }
+
+  // Check the existence of a given DynamoDB table.
+  private static boolean exist(DynamoDB dynamoDB, String tableName) {
+    assertNotNull(dynamoDB);
+    assertNotNull(tableName);
+    assertFalse("empty table name", tableName.isEmpty());
+    try {
+      Table table = dynamoDB.getTable(tableName);
+      table.describe();
+    } catch (ResourceNotFoundException e) {
+      return false;
+    }
+    return true;
+  }
+
+  @Test
+  public void testInvalidRegion() throws Exception {
+    final String testTableName = "testInvalidRegion" + new Random().nextInt();
+    final String testRegion = "invalidRegion";
+    // Initialize MetadataStore
+    final Init initCmd = new Init(getFileSystem().getConf());
+    LambdaTestUtils.intercept(IOException.class,
+        new Callable<String>() {
+          @Override
+          public String call() throws Exception {
+            int res = initCmd.run(new String[]{
+                "init",
+                "-region", testRegion,
+                "-meta", "dynamodb://" + testTableName
+            });
+            return "Use of invalid region did not fail, returning " + res
+                + "- table may have been " +
+                "created and not cleaned up: " + testTableName;
+          }
+        });
+  }
+
+  @Test
+  public void testDynamoDBInitDestroyCycle() throws Exception {
+    String testTableName = "testDynamoDBInitDestroy" + new Random().nextInt();
+    String testS3Url = path(testTableName).toString();
+    S3AFileSystem fs = getFileSystem();
+    DynamoDB db = null;
+    try {
+      // Initialize MetadataStore
+      Init initCmd = new Init(fs.getConf());
+      expectSuccess("Init command did not exit successfully - see output",
+          initCmd,
+          "init", "-meta", "dynamodb://" + testTableName, testS3Url);
+      // Verify it exists
+      MetadataStore ms = getMetadataStore();
+      assertTrue("metadata store should be DynamoDBMetadataStore",
+          ms instanceof DynamoDBMetadataStore);
+      DynamoDBMetadataStore dynamoMs = (DynamoDBMetadataStore) ms;
+      db = dynamoMs.getDynamoDB();
+      assertTrue(String.format("%s does not exist", testTableName),
+          exist(db, testTableName));
+
+      // Destroy MetadataStore
+      Destroy destroyCmd = new Destroy(fs.getConf());
+
+      expectSuccess("Destroy command did not exit successfully - see output",
+          destroyCmd,
+          "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
+      // Verify it does not exist
+      assertFalse(String.format("%s still exists", testTableName),
+          exist(db, testTableName));
+
+      // delete again and expect success again
+      expectSuccess("Destroy command did not exit successfully - see output",
+          destroyCmd,
+          "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
+    } catch (ResourceNotFoundException e) {
+      throw new AssertionError(
+          String.format("DynamoDB table %s does not exist", testTableName),
+          e);
+    } finally {
+      LOG.warn("Table may have not been cleaned up: " +
+          testTableName);
+      if (db != null) {
+        Table table = db.getTable(testTableName);
+        if (table != null) {
+          try {
+            table.delete();
+            table.waitForDelete();
+          } catch (ResourceNotFoundException e) { /* Ignore */ }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
new file mode 100644
index 0000000..181cdfb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
+
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+
+/**
+ * Test S3Guard related CLI commands against a LocalMetadataStore.
+ */
+public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
+
+  @Override
+  protected MetadataStore newMetadataStore() {
+    return new LocalMetadataStore();
+  }
+
+  @Test
+  public void testImportCommand() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    MetadataStore ms = getMetadataStore();
+    Path parent = path("test-import");
+    fs.mkdirs(parent);
+    Path dir = new Path(parent, "a");
+    fs.mkdirs(dir);
+    Path emptyDir = new Path(parent, "emptyDir");
+    fs.mkdirs(emptyDir);
+    for (int i = 0; i < 10; i++) {
+      String child = String.format("file-%d", i);
+      try (FSDataOutputStream out = fs.create(new Path(dir, child))) {
+        out.write(1);
+      }
+    }
+
+    S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf());
+    cmd.setStore(ms);
+
+    expectSuccess("Import command did not exit successfully - see output",
+        cmd,
+        "import", parent.toString());
+
+    DirListingMetadata children =
+        ms.listChildren(dir);
+    assertEquals("Unexpected number of paths imported", 10, children
+        .getListing().size());
+    assertEquals("Expected 2 items: empty directory and a parent directory", 2,
+        ms.listChildren(parent).getListing().size());
+    // assertTrue(children.isAuthoritative());
+  }
+
+  @Test
+  public void testDiffCommand() throws IOException {
+    S3AFileSystem fs = getFileSystem();
+    MetadataStore ms = getMetadataStore();
+    Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
+    Set<Path> filesOnMS = new HashSet<>(); // files on metadata store.
+
+    Path testPath = path("test-diff");
+    mkdirs(testPath, true, true);
+
+    Path msOnlyPath = new Path(testPath, "ms_only");
+    mkdirs(msOnlyPath, false, true);
+    filesOnMS.add(msOnlyPath);
+    for (int i = 0; i < 5; i++) {
+      Path file = new Path(msOnlyPath, String.format("file-%d", i));
+      createFile(file, false, true);
+      filesOnMS.add(file);
+    }
+
+    Path s3OnlyPath = new Path(testPath, "s3_only");
+    mkdirs(s3OnlyPath, true, false);
+    filesOnS3.add(s3OnlyPath);
+    for (int i = 0; i < 5; i++) {
+      Path file = new Path(s3OnlyPath, String.format("file-%d", i));
+      createFile(file, true, false);
+      filesOnS3.add(file);
+    }
+
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(buf);
+    Diff cmd = new Diff(fs.getConf());
+    cmd.setStore(ms);
+    assertEquals("Diff command did not exit successfully - see output", SUCCESS,
+        cmd.run(new String[]{"diff", "-meta", "local://metadata",
+            testPath.toString()}, out));
+    out.close();
+
+    Set<Path> actualOnS3 = new HashSet<>();
+    Set<Path> actualOnMS = new HashSet<>();
+    boolean duplicates = false;
+    try (BufferedReader reader =
+             new BufferedReader(new InputStreamReader(
+                 new ByteArrayInputStream(buf.toByteArray())))) {
+      String line;
+      while ((line = reader.readLine()) != null) {
+        String[] fields = line.split("\\s");
+        assertEquals("[" + line + "] does not have enough fields",
+            4, fields.length);
+        String where = fields[0];
+        Path path = new Path(fields[3]);
+        if (Diff.S3_PREFIX.equals(where)) {
+          duplicates = duplicates || actualOnS3.contains(path);
+          actualOnS3.add(path);
+        } else if (Diff.MS_PREFIX.equals(where)) {
+          duplicates = duplicates || actualOnMS.contains(path);
+          actualOnMS.add(path);
+        } else {
+          fail("Unknown prefix: " + where);
+        }
+      }
+    }
+    String actualOut = out.toString();
+    assertEquals("Mismatched metadata store outputs: " + actualOut,
+        filesOnMS, actualOnMS);
+    assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
+    assertFalse("Diff contained duplicates", duplicates);
+  }
+}


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


[18/48] hadoop git commit: HADOOP-14414. Calling maven-site-plugin directly for docs profile is unnecessary (Andras Bokor via aw)

Posted by in...@apache.org.
HADOOP-14414. Calling maven-site-plugin directly for docs profile is unnecessary (Andras Bokor via aw)


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

Branch: refs/heads/HDFS-10467
Commit: a3fee475f72c8548aafd8574da1d3f1745f0471d
Parents: a756704
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Sep 1 08:38:30 2017 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Sep 1 08:38:30 2017 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-auth/pom.xml      | 12 -----------
 hadoop-common-project/hadoop-kms/pom.xml       | 24 ---------------------
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml | 24 ---------------------
 hadoop-tools/hadoop-sls/pom.xml                | 22 -------------------
 4 files changed, 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fee475/hadoop-common-project/hadoop-auth/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml
index 36a70ae..078ac40 100644
--- a/hadoop-common-project/hadoop-auth/pom.xml
+++ b/hadoop-common-project/hadoop-auth/pom.xml
@@ -245,18 +245,6 @@
         <plugins>
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-site-plugin</artifactId>
-            <executions>
-              <execution>
-                <phase>package</phase>
-                <goals>
-                  <goal>site</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-javadoc-plugin</artifactId>
             <executions>
               <execution>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fee475/hadoop-common-project/hadoop-kms/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/pom.xml b/hadoop-common-project/hadoop-kms/pom.xml
index 27be05a..ae2a325 100644
--- a/hadoop-common-project/hadoop-kms/pom.xml
+++ b/hadoop-common-project/hadoop-kms/pom.xml
@@ -247,30 +247,6 @@
 
   <profiles>
     <profile>
-      <id>docs</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-site-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>docs</id>
-                <phase>prepare-package</phase>
-                <goals>
-                  <goal>site</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-
-    <profile>
       <id>dist</id>
       <activation>
         <activeByDefault>false</activeByDefault>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fee475/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
index b7adda0..16b94f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
@@ -386,30 +386,6 @@
     </profile>
     
     <profile>
-      <id>docs</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-site-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>docs</id>
-                <phase>prepare-package</phase>
-                <goals>
-                  <goal>site</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-
-    <profile>
       <id>dist</id>
       <activation>
         <activeByDefault>false</activeByDefault>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fee475/hadoop-tools/hadoop-sls/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/pom.xml b/hadoop-tools/hadoop-sls/pom.xml
index 7dd3718..eba0514 100644
--- a/hadoop-tools/hadoop-sls/pom.xml
+++ b/hadoop-tools/hadoop-sls/pom.xml
@@ -145,28 +145,6 @@
 
   <profiles>
     <profile>
-      <id>docs</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-site-plugin</artifactId>
-            <executions>
-              <execution>
-                <phase>package</phase>
-                <goals>
-                  <goal>site</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <profile>
       <id>dist</id>
       <activation>
         <activeByDefault>false</activeByDefault>


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


[02/48] hadoop git commit: HADOOP-14781. Clarify that HADOOP_CONF_DIR shouldn't actually be set in hadoop-env.sh

Posted by in...@apache.org.
HADOOP-14781. Clarify that HADOOP_CONF_DIR shouldn't actually be set in hadoop-env.sh

Signed-off-by: Andrew Wang <wa...@apache.org>


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

Branch: refs/heads/HDFS-10467
Commit: 27359b713982b480d456067e4a71bf0c4ffb1df2
Parents: 0adc3a0
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Aug 29 10:10:56 2017 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Thu Aug 31 21:10:52 2017 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/main/conf/hadoop-env.sh            | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/27359b71/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
index fbc7bc3..bef4dab 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
+++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
@@ -58,10 +58,13 @@
 # export HADOOP_HOME=
 
 # Location of Hadoop's configuration information.  i.e., where this
-# file is probably living. Many sites will also set this in the
-# same location where JAVA_HOME is defined.  If this is not defined
-# Hadoop will attempt to locate it based upon its execution
-# path.
+# file is living. If this is not defined, Hadoop will attempt to
+# locate it based upon its execution path.
+#
+# NOTE: It is recommend that this variable not be set here but in
+# /etc/profile.d or equivalent.  Some options (such as
+# --config) may react strangely otherwise.
+#
 # export HADOOP_CONF_DIR=${HADOOP_HOME}/etc/hadoop
 
 # The maximum amount of heap to use (Java -Xmx).  If no unit


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


[19/48] hadoop git commit: YARN-7120. CapacitySchedulerPage NPE in "Aggregate scheduler counts" section. Contributed by Eric Payne

Posted by in...@apache.org.
YARN-7120. CapacitySchedulerPage NPE in "Aggregate scheduler counts" section. Contributed by Eric Payne


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

Branch: refs/heads/HDFS-10467
Commit: 99a7f5d451fb91ef956c926f891283f70f0ada1c
Parents: a3fee47
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Sep 1 11:09:14 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Sep 1 11:09:14 2017 -0500

----------------------------------------------------------------------
 .../server/resourcemanager/webapp/CapacitySchedulerPage.java | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99a7f5d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 93e1c97..3c00cb8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -556,8 +556,12 @@ class CapacitySchedulerPage extends RmView {
               .$class("ui-state-default").__("Queue").__().__().__().tbody();
         SchedulerHealth.DetailedInformation di = entry.getValue();
         if (di.getTimestamp() != 0) {
-          containerId = di.getContainerId().toString();
-          nodeId = di.getNodeId().toString();
+          if (di.getContainerId() != null) {
+            containerId = di.getContainerId().toString();
+          }
+          if (di.getNodeId() != null) {
+            nodeId = di.getNodeId().toString();
+          }
           queue = di.getQueue();
         }
         tbody.$class("ui-widget-content").tr()


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


[06/48] hadoop git commit: HDFS-12380. Simplify dataQueue.wait condition logical operation in DataStreamer::run(). Contributed by liaoyuxiangqin

Posted by in...@apache.org.
HDFS-12380. Simplify dataQueue.wait condition logical operation in DataStreamer::run(). Contributed by liaoyuxiangqin


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

Branch: refs/heads/HDFS-10467
Commit: 36f33a1efb35e9f6986516499b54fdfa38fac2a1
Parents: dcd0bed
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Aug 31 23:15:07 2017 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Aug 31 23:18:48 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DataStreamer.java        | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/36f33a1e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 838da7e..4eafca1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -689,8 +689,7 @@ class DataStreamer extends Daemon {
           long now = Time.monotonicNow();
           while ((!shouldStop() && dataQueue.size() == 0 &&
               (stage != BlockConstructionStage.DATA_STREAMING ||
-                  stage == BlockConstructionStage.DATA_STREAMING &&
-                      now - lastPacket < halfSocketTimeout)) || doSleep ) {
+                  now - lastPacket < halfSocketTimeout)) || doSleep) {
             long timeout = halfSocketTimeout - (now-lastPacket);
             timeout = timeout <= 0 ? 1000 : timeout;
             timeout = (stage == BlockConstructionStage.DATA_STREAMING)?


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


[07/48] hadoop git commit: HDFS-12300. Audit-log delegation token related operations.

Posted by in...@apache.org.
HDFS-12300. Audit-log delegation token related operations.


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

Branch: refs/heads/HDFS-10467
Commit: 1b3b9938cf663c71d2e5d9032fdfb1460bae0d3f
Parents: 36f33a1
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Aug 31 23:17:16 2017 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Aug 31 23:20:01 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    | 22 +++++++++
 .../hdfs/server/namenode/FSNamesystem.java      | 35 ++++++++++---
 .../namenode/TestAuditLoggerWithCommands.java   | 52 ++++++++++++++++++++
 3 files changed, 103 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9938/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 47e1c0d..7776dc2 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
@@ -36,6 +36,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetAddress;
@@ -70,6 +72,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HttpConfig;
@@ -80,6 +83,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ToolRunner;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -1570,4 +1574,22 @@ public class DFSUtil {
         .createKeyProviderCryptoExtension(keyProvider);
     return cryptoProvider;
   }
+
+  /**
+   * Decodes an HDFS delegation token to its identifier.
+   *
+   * @param token the token
+   * @return the decoded identifier.
+   * @throws IOException
+   */
+  public static DelegationTokenIdentifier decodeDelegationToken(
+      final Token<DelegationTokenIdentifier> token) throws IOException {
+    final DelegationTokenIdentifier id = new DelegationTokenIdentifier();
+    final ByteArrayInputStream buf =
+        new ByteArrayInputStream(token.getIdentifier());
+    try (DataInputStream in = new DataInputStream(buf)) {
+      id.readFields(in);
+    }
+    return id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9938/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 12d96d8..346f046 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -101,9 +101,7 @@ import static org.apache.hadoop.util.Time.monotonicNow;
 import static org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics.TOPMETRICS_METRICS_SOURCE_NAME;
 
 import java.io.BufferedWriter;
-import java.io.ByteArrayInputStream;
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -5399,6 +5397,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
+    final String operationName = "getDelegationToken";
+    final boolean success;
+    final String tokenId;
     Token<DelegationTokenIdentifier> token;
     checkOperation(OperationCategory.WRITE);
     writeLock();
@@ -5427,10 +5428,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dtId, dtSecretManager);
       long expiryTime = dtSecretManager.getTokenExpiryTime(dtId);
       getEditLog().logGetDelegationToken(dtId, expiryTime);
+      tokenId = dtId.toStringStable();
+      success = true;
     } finally {
       writeUnlock("getDelegationToken");
     }
     getEditLog().logSync();
+    logAuditEvent(success, operationName, tokenId);
     return token;
   }
 
@@ -5443,6 +5447,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
+    final String operationName = "renewDelegationToken";
+    boolean success = false;
+    String tokenId;
     long expiryTime;
     checkOperation(OperationCategory.WRITE);
     writeLock();
@@ -5456,15 +5463,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       String renewer = getRemoteUser().getShortUserName();
       expiryTime = dtSecretManager.renewToken(token, renewer);
-      DelegationTokenIdentifier id = new DelegationTokenIdentifier();
-      ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
-      DataInputStream in = new DataInputStream(buf);
-      id.readFields(in);
+      final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
       getEditLog().logRenewDelegationToken(id, expiryTime);
+      tokenId = id.toStringStable();
+      success = true;
+    } catch (AccessControlException ace) {
+      final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
+      tokenId = id.toStringStable();
+      logAuditEvent(success, operationName, tokenId);
+      throw ace;
     } finally {
       writeUnlock("renewDelegationToken");
     }
     getEditLog().logSync();
+    logAuditEvent(success, operationName, tokenId);
     return expiryTime;
   }
 
@@ -5475,6 +5487,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
+    final String operationName = "cancelDelegationToken";
+    boolean success = false;
+    String tokenId;
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -5485,10 +5500,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       DelegationTokenIdentifier id = dtSecretManager
         .cancelToken(token, canceller);
       getEditLog().logCancelDelegationToken(id);
+      tokenId = id.toStringStable();
+      success = true;
+    } catch (AccessControlException ace) {
+      final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
+      tokenId = id.toStringStable();
+      logAuditEvent(success, operationName, tokenId);
+      throw ace;
     } finally {
       writeUnlock("cancelDelegationToken");
     }
     getEditLog().logSync();
+    logAuditEvent(success, operationName, tokenId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9938/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
index 8b06b0b..2adf470 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
@@ -34,13 +34,16 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 import org.junit.BeforeClass;
 import org.junit.AfterClass;
 import org.junit.Ignore;
 import org.junit.Test;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import org.mockito.Mock;
@@ -68,6 +71,7 @@ public class TestAuditLoggerWithCommands {
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,true);
+    conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
     cluster.waitActive();
@@ -566,6 +570,54 @@ public class TestAuditLoggerWithCommands {
     cluster.getNamesystem().setFSDirectory(dir);
   }
 
+  @Test
+  public void testDelegationTokens() throws Exception {
+    Token dt = fs.getDelegationToken("foo");
+    final String getDT =
+        ".*src=HDFS_DELEGATION_TOKEN token 1.*with renewer foo.*";
+    verifyAuditLogs(true, ".*cmd=getDelegationToken" + getDT);
+
+    // renew
+    final UserGroupInformation foo =
+        UserGroupInformation.createUserForTesting("foo", new String[] {});
+    foo.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        dt.renew(conf);
+        return null;
+      }
+    });
+    verifyAuditLogs(true, ".*cmd=renewDelegationToken" + getDT);
+    try {
+      dt.renew(conf);
+      fail("Renewing a token with non-renewer should fail");
+    } catch (AccessControlException expected) {
+    }
+    verifyAuditLogs(false, ".*cmd=renewDelegationToken" + getDT);
+
+    // cancel
+    final UserGroupInformation bar =
+        UserGroupInformation.createUserForTesting("bar", new String[] {});
+    try {
+      bar.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          dt.cancel(conf);
+          return null;
+        }
+      });
+      fail("Canceling a token with non-renewer should fail");
+    } catch (AccessControlException expected) {
+    }
+    verifyAuditLogs(false, ".*cmd=cancelDelegationToken" + getDT);
+    dt.cancel(conf);
+    verifyAuditLogs(true, ".*cmd=cancelDelegationToken" + getDT);
+  }
+
+  private int verifyAuditLogs(final boolean allowed, final String pattern) {
+    return verifyAuditLogs(".*allowed=" + allowed + pattern);
+  }
+
   private int verifyAuditLogs(String pattern) {
     int length = auditlog.getOutput().split("\n").length;
     String lastAudit = auditlog.getOutput().split("\n")[length - 1];


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


[44/48] 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/8c2b7aea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c2b7aea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c2b7aea

Branch: refs/heads/HDFS-10467
Commit: 8c2b7aeaff07b79ecaccc5bbd72c18aa0ec79405
Parents: f514ba4
Author: Inigo Goiri <in...@apache.org>
Authored: Mon Aug 21 11:40:41 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:10 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/8c2b7aea/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/8c2b7aea/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/8c2b7aea/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/8c2b7aea/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/8c2b7aea/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


[24/48] hadoop git commit: Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 658387b..100d38c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -18,8 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import java.io.IOException;
-import java.io.StringWriter;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -33,8 +32,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.gson.stream.JsonWriter;
-
 /**
  * This is a run length encoded sparse data structure that maintains resource
  * allocations over time.
@@ -44,12 +41,14 @@ public class RLESparseResourceAllocation {
   private static final int THRESHOLD = 100;
   private static final Resource ZERO_RESOURCE = Resources.none();
 
-  private NavigableMap<Long, Resource> cumulativeCapacity =
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected NavigableMap<Long, Resource> cumulativeCapacity =
       new TreeMap<Long, Resource>();
 
   private final ReentrantReadWriteLock readWriteLock =
       new ReentrantReadWriteLock();
-  private final Lock readLock = readWriteLock.readLock();
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected final Lock readLock = readWriteLock.readLock();
   private final Lock writeLock = readWriteLock.writeLock();
 
   private final ResourceCalculator resourceCalculator;
@@ -236,34 +235,6 @@ public class RLESparseResourceAllocation {
   }
 
   /**
-   * Returns the JSON string representation of the current resources allocated
-   * over time.
-   *
-   * @return the JSON string representation of the current resources allocated
-   *         over time
-   */
-  public String toMemJSONString() {
-    StringWriter json = new StringWriter();
-    JsonWriter jsonWriter = new JsonWriter(json);
-    readLock.lock();
-    try {
-      jsonWriter.beginObject();
-      // jsonWriter.name("timestamp").value("resource");
-      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
-        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
-      }
-      jsonWriter.endObject();
-      jsonWriter.close();
-      return json.toString();
-    } catch (IOException e) {
-      // This should not happen
-      return "";
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  /**
    * Returns the representation of the current resources allocated over time as
    * an interval map (in the defined non-null range).
    *
@@ -304,7 +275,7 @@ public class RLESparseResourceAllocation {
   public NavigableMap<Long, Resource> getCumulative() {
     readLock.lock();
     try {
-      return cumulativeCapacity;
+      return Collections.unmodifiableNavigableMap(cumulativeCapacity);
     } finally {
       readLock.unlock();
     }
@@ -437,8 +408,8 @@ public class RLESparseResourceAllocation {
       Resource val = Resources.negate(e.getValue());
       // test for negative value and throws
       if (operator == RLEOperator.subtractTestNonNegative
-          && (Resources.fitsIn(val, ZERO_RESOURCE) &&
-              !Resources.equals(val, ZERO_RESOURCE))) {
+          && (Resources.fitsIn(val, ZERO_RESOURCE)
+              && !Resources.equals(val, ZERO_RESOURCE))) {
         throw new PlanningException(
             "RLESparseResourceAllocation: merge failed as the "
                 + "resulting RLESparseResourceAllocation would be negative");
@@ -504,22 +475,29 @@ public class RLESparseResourceAllocation {
 
   }
 
+  /**
+   * Get a {@link RLESparseResourceAllocation} view of the {@link Resource}
+   * allocations between the specified start and end times.
+   *
+   * @param start the time from which the {@link Resource} allocations are
+   *          required
+   * @param end the time upto which the {@link Resource} allocations are
+   *          required
+   * @return the overlapping allocations
+   */
   public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
     readLock.lock();
     try {
       NavigableMap<Long, Resource> a = this.getCumulative();
-
       if (a != null && !a.isEmpty()) {
         // include the portion of previous entry that overlaps start
         if (start > a.firstKey()) {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
-
         if (end < a.lastKey()) {
           a = a.headMap(end, true);
         }
-
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);
@@ -527,7 +505,33 @@ public class RLESparseResourceAllocation {
     } finally {
       readLock.unlock();
     }
+  }
 
+  /**
+   * This method shifts all the timestamp of the {@link Resource} entries by the
+   * specified "delta".
+   *
+   * @param delta the time by which to shift the {@link Resource} allocations
+   */
+  public void shift(long delta) {
+    writeLock.lock();
+    try {
+      TreeMap<Long, Resource> newCum = new TreeMap<>();
+      long start;
+      for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+        if (delta > 0) {
+          start = (entry.getKey() == Long.MAX_VALUE) ? Long.MAX_VALUE
+              : entry.getKey() + delta;
+        } else {
+          start = (entry.getKey() == Long.MIN_VALUE) ? Long.MIN_VALUE
+              : entry.getKey() + delta;
+        }
+        newCum.put(start, entry.getValue());
+      }
+      cumulativeCapacity = newCum;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   /**
@@ -541,8 +545,8 @@ public class RLESparseResourceAllocation {
   /**
    * Get the maximum capacity across specified time instances. The search-space
    * is specified using the starting value, tick, and the periodic interval for
-   * search. Maximum resource allocation across tick, tick + period,
-   * tick + 2 * period,..., tick + n * period .. is returned.
+   * search. Maximum resource allocation across tick, tick + period, tick + 2 *
+   * period,..., tick + n * period .. is returned.
    *
    * @param tick the starting time instance
    * @param period interval at which capacity is evaluated
@@ -550,14 +554,19 @@ public class RLESparseResourceAllocation {
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxCapacity = ZERO_RESOURCE;
-    if (!cumulativeCapacity.isEmpty()) {
-      Long lastKey = cumulativeCapacity.lastKey();
-      for (long t = tick; t <= lastKey; t = t + period) {
-        maxCapacity = Resources.componentwiseMax(maxCapacity,
-            cumulativeCapacity.floorEntry(t).getValue());
+    readLock.lock();
+    try {
+      if (!cumulativeCapacity.isEmpty()) {
+        Long lastKey = cumulativeCapacity.lastKey();
+        for (long t = tick; t <= lastKey; t = t + period) {
+          maxCapacity = Resources.componentwiseMax(maxCapacity,
+              cumulativeCapacity.floorEntry(t).getValue());
+        }
       }
+      return maxCapacity;
+    } finally {
+      readLock.unlock();
     }
-    return maxCapacity;
   }
 
   /**
@@ -567,17 +576,17 @@ public class RLESparseResourceAllocation {
    * @return minimum resource allocation
    */
   public Resource getMinimumCapacityInInterval(ReservationInterval interval) {
-    Resource minCapacity = Resource.newInstance(
-        Integer.MAX_VALUE, Integer.MAX_VALUE);
+    Resource minCapacity =
+        Resource.newInstance(Integer.MAX_VALUE, Integer.MAX_VALUE);
     long start = interval.getStartTime();
     long end = interval.getEndTime();
     NavigableMap<Long, Resource> capacityRange =
-        this.getRangeOverlapping(start, end).getCumulative();
+        getRangeOverlapping(start, end).getCumulative();
     if (!capacityRange.isEmpty()) {
       for (Map.Entry<Long, Resource> entry : capacityRange.entrySet()) {
         if (entry.getValue() != null) {
-          minCapacity = Resources.componentwiseMin(minCapacity,
-              entry.getValue());
+          minCapacity =
+              Resources.componentwiseMin(minCapacity, entry.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index 0da95ac..bb4a7fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -24,14 +24,16 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A ReservationAllocation represents a concrete allocation of resources over
  * time that satisfy a certain {@link ReservationDefinition}. This is used
  * internally by a {@link Plan} to store information about how each of the
  * accepted {@link ReservationDefinition} have been allocated.
  */
-public interface ReservationAllocation extends
-    Comparable<ReservationAllocation> {
+public interface ReservationAllocation
+    extends Comparable<ReservationAllocation> {
 
   /**
    * Returns the unique identifier {@link ReservationId} that represents the
@@ -40,28 +42,28 @@ public interface ReservationAllocation extends
    * @return reservationId the unique identifier {@link ReservationId} that
    *         represents the reservation
    */
-  public ReservationId getReservationId();
+  ReservationId getReservationId();
 
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
    * @return the {@link ReservationDefinition} submitted by the client
    */
-  public ReservationDefinition getReservationDefinition();
+  ReservationDefinition getReservationDefinition();
 
   /**
    * Returns the time at which the reservation is activated.
    * 
    * @return the time at which the reservation is activated
    */
-  public long getStartTime();
+  long getStartTime();
 
   /**
    * Returns the time at which the reservation terminates.
    * 
    * @return the time at which the reservation terminates
    */
-  public long getEndTime();
+  long getEndTime();
 
   /**
    * Returns the map of resources requested against the time interval for which
@@ -70,28 +72,28 @@ public interface ReservationAllocation extends
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  public Map<ReservationInterval, Resource> getAllocationRequests();
+  Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs
    * 
    * @return the plan to which the reservation belongs
    */
-  public String getPlanName();
+  String getPlanName();
 
   /**
    * Returns the user who requested the reservation
    * 
    * @return the user who requested the reservation
    */
-  public String getUser();
+  String getUser();
 
   /**
    * Returns whether the reservation has gang semantics or not
    * 
    * @return true if there is a gang request, false otherwise
    */
-  public boolean containsGangs();
+  boolean containsGangs();
 
   /**
    * Sets the time at which the reservation was accepted by the system
@@ -99,14 +101,14 @@ public interface ReservationAllocation extends
    * @param acceptedAt the time at which the reservation was accepted by the
    *          system
    */
-  public void setAcceptanceTimestamp(long acceptedAt);
+  void setAcceptanceTimestamp(long acceptedAt);
 
   /**
    * Returns the time at which the reservation was accepted by the system
    * 
    * @return the time at which the reservation was accepted by the system
    */
-  public long getAcceptanceTime();
+  long getAcceptanceTime();
 
   /**
    * Returns the capacity represented by cumulative resources reserved by the
@@ -116,12 +118,42 @@ public interface ReservationAllocation extends
    *          requested
    * @return the resources reserved at the specified time
    */
-  public Resource getResourcesAtTime(long tick);
+  Resource getResourcesAtTime(long tick);
+
+  /**
+   * Return a RLE representation of used resources.
+   *
+   * @return a RLE encoding of resources allocated over time.
+   */
+  RLESparseResourceAllocation getResourcesOverTime();
+
 
   /**
    * Return a RLE representation of used resources.
+   *
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return a RLE encoding of resources allocated over time.
    */
-  public RLESparseResourceAllocation getResourcesOverTime();
+  RLESparseResourceAllocation getResourcesOverTime(long start, long end);
+
+  /**
+   * Get the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @return periodicity of this reservation
+   */
+  long getPeriodicity();
+
+  /**
+   * Set the periodicity of this reservation representing the time period of the
+   * periodic job. Period is represented in milliseconds for periodic jobs.
+   * Period is 0 for non-periodic jobs.
+   *
+   * @param period periodicity of this reservation
+   */
+  @VisibleForTesting
+  void setPeriodicity(long period);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 027d066..a66d222 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -44,6 +44,8 @@ public class ReservationInputValidator {
 
   /**
    * Utility class to validate reservation requests.
+   *
+   * @param clock the {@link Clock} to use
    */
   public ReservationInputValidator(Clock clock) {
     this.clock = clock;
@@ -53,22 +55,21 @@ public class ReservationInputValidator {
       ReservationId reservationId, String auditConstant) throws YarnException {
     // check if the reservation id is valid
     if (reservationId == null) {
-      String message =
-          "Missing reservation id."
-              + " Please try again by specifying a reservation id.";
+      String message = "Missing reservation id."
+          + " Please try again by specifying a reservation id.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     String queue = reservationSystem.getQueueForReservation(reservationId);
     String nullQueueErrorMessage =
-            "The specified reservation with ID: " + reservationId
-                    + " is unknown. Please try again with a valid reservation.";
+        "The specified reservation with ID: " + reservationId
+            + " is unknown. Please try again with a valid reservation.";
     String nullPlanErrorMessage = "The specified reservation: " + reservationId
-                            + " is not associated with any valid plan."
-                            + " Please try again with a valid reservation.";
+        + " is not associated with any valid plan."
+        + " Please try again with a valid reservation.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
   private void validateReservationDefinition(ReservationId reservationId,
@@ -77,17 +78,15 @@ public class ReservationInputValidator {
     String message = "";
     // check if deadline is in the past
     if (contract == null) {
-      message =
-          "Missing reservation definition."
-              + " Please try again by specifying a reservation definition.";
+      message = "Missing reservation definition."
+          + " Please try again by specifying a reservation definition.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     if (contract.getDeadline() <= clock.getTime()) {
-      message =
-          "The specified deadline: " + contract.getDeadline()
-              + " is the past. Please try again with deadline in the future.";
+      message = "The specified deadline: " + contract.getDeadline()
+          + " is the past. Please try again with deadline in the future.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -95,18 +94,16 @@ public class ReservationInputValidator {
     // Check if at least one RR has been specified
     ReservationRequests resReqs = contract.getReservationRequests();
     if (resReqs == null) {
-      message =
-          "No resources have been specified to reserve."
-              + "Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + "Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
     }
     List<ReservationRequest> resReq = resReqs.getReservationResources();
     if (resReq == null || resReq.isEmpty()) {
-      message =
-          "No resources have been specified to reserve."
-              + " Please try again by specifying the resources to reserve.";
+      message = "No resources have been specified to reserve."
+          + " Please try again by specifying the resources to reserve.";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -123,22 +120,18 @@ public class ReservationInputValidator {
       } else {
         minDuration += rr.getDuration();
       }
-      maxGangSize =
-          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
-              maxGangSize,
-              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+      maxGangSize = Resources.max(plan.getResourceCalculator(),
+          plan.getTotalCapacity(), maxGangSize,
+          Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
     long duration = contract.getDeadline() - contract.getArrival();
-    if (duration < minDuration
-        && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The time difference ("
-              + (duration)
-              + ") between arrival (" + contract.getArrival() + ") "
-              + "and deadline (" + contract.getDeadline() + ") must "
-              + " be greater or equal to the minimum resource duration ("
-              + minDuration + ")";
+    if (duration < minDuration && type != ReservationRequestInterpreter.R_ANY) {
+      message = "The time difference (" + (duration) + ") between arrival ("
+          + contract.getArrival() + ") " + "and deadline ("
+          + contract.getDeadline() + ") must "
+          + " be greater or equal to the minimum resource duration ("
+          + minDuration + ")";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -148,10 +141,9 @@ public class ReservationInputValidator {
     if (Resources.greaterThan(plan.getResourceCalculator(),
         plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
         && type != ReservationRequestInterpreter.R_ANY) {
-      message =
-          "The size of the largest gang in the reservation definition ("
-              + maxGangSize + ") exceed the capacity available ("
-              + plan.getTotalCapacity() + " )";
+      message = "The size of the largest gang in the reservation definition ("
+          + maxGangSize + ") exceed the capacity available ("
+          + plan.getTotalCapacity() + " )";
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
           "validate reservation input definition", "ClientRMService", message);
       throw RPCUtil.getRemoteException(message);
@@ -179,32 +171,32 @@ public class ReservationInputValidator {
     }
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant) throws YarnException {
     String nullQueueErrorMessage = "The queue is not specified."
-            + " Please try again with a valid reservable queue.";
+        + " Please try again with a valid reservable queue.";
     String nullPlanErrorMessage = "The specified queue: " + queue
-            + " is not managed by reservation system."
-            + " Please try again with a valid reservable queue.";
+        + " is not managed by reservation system."
+        + " Please try again with a valid reservable queue.";
     return getPlanFromQueue(reservationSystem, queue, auditConstant,
-            nullQueueErrorMessage, nullPlanErrorMessage);
+        nullQueueErrorMessage, nullPlanErrorMessage);
   }
 
-  private Plan getPlanFromQueue(ReservationSystem reservationSystem, String
-          queue, String auditConstant, String nullQueueErrorMessage,
-          String nullPlanErrorMessage) throws YarnException {
+  private Plan getPlanFromQueue(ReservationSystem reservationSystem,
+      String queue, String auditConstant, String nullQueueErrorMessage,
+      String nullPlanErrorMessage) throws YarnException {
     if (queue == null || queue.isEmpty()) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullQueueErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullQueueErrorMessage);
       throw RPCUtil.getRemoteException(nullQueueErrorMessage);
     }
     // check if the associated plan is valid
     Plan plan = reservationSystem.getPlan(queue);
     if (plan == null) {
       RMAuditLogger.logFailure("UNKNOWN", auditConstant,
-              "validate reservation input", "ClientRMService",
-              nullPlanErrorMessage);
+          "validate reservation input", "ClientRMService",
+          nullPlanErrorMessage);
       throw RPCUtil.getRemoteException(nullPlanErrorMessage);
     }
     return plan;
@@ -222,22 +214,21 @@ public class ReservationInputValidator {
    * @param reservationId the {@link ReservationId} associated with the current
    *          request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationSubmissionRequest(
-      ReservationSystem reservationSystem,
-      ReservationSubmissionRequest request, ReservationId reservationId)
-      throws YarnException {
+      ReservationSystem reservationSystem, ReservationSubmissionRequest request,
+      ReservationId reservationId) throws YarnException {
     String message;
     if (reservationId == null) {
-      message = "Reservation id cannot be null. Please try again " +
-        "specifying a valid reservation id by creating a new reservation id.";
+      message = "Reservation id cannot be null. Please try again specifying "
+          + " a valid reservation id by creating a new reservation id.";
       throw RPCUtil.getRemoteException(message);
     }
     // Check if it is a managed queue
     String queue = request.getQueue();
     Plan plan = getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.SUBMIT_RESERVATION_REQUEST);
+        AuditConstants.SUBMIT_RESERVATION_REQUEST);
 
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
@@ -255,15 +246,14 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationUpdateRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationUpdateRequest(
       ReservationSystem reservationSystem, ReservationUpdateRequest request)
       throws YarnException {
     ReservationId reservationId = request.getReservationId();
-    Plan plan =
-        validateReservation(reservationSystem, reservationId,
-            AuditConstants.UPDATE_RESERVATION_REQUEST);
+    Plan plan = validateReservation(reservationSystem, reservationId,
+        AuditConstants.UPDATE_RESERVATION_REQUEST);
     validateReservationDefinition(reservationId,
         request.getReservationDefinition(), plan,
         AuditConstants.UPDATE_RESERVATION_REQUEST);
@@ -278,28 +268,26 @@ public class ReservationInputValidator {
    *
    * @param reservationSystem the {@link ReservationSystem} to validate against
    * @param request the {@link ReservationListRequest} defining search
-   *                parameters for reservations in the {@link ReservationSystem}
-   *                that is being validated against.
+   *          parameters for reservations in the {@link ReservationSystem} that
+   *          is being validated against.
    * @return the {@link Plan} to list reservations of.
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationListRequest(
-      ReservationSystem reservationSystem,
-      ReservationListRequest request)
+      ReservationSystem reservationSystem, ReservationListRequest request)
       throws YarnException {
     String queue = request.getQueue();
     if (request.getEndTime() < request.getStartTime()) {
-      String errorMessage = "The specified end time must be greater than " +
-              "the specified start time.";
+      String errorMessage = "The specified end time must be greater than "
+          + "the specified start time.";
       RMAuditLogger.logFailure("UNKNOWN",
-              AuditConstants.LIST_RESERVATION_REQUEST,
-              "validate list reservation input", "ClientRMService",
-              errorMessage);
+          AuditConstants.LIST_RESERVATION_REQUEST,
+          "validate list reservation input", "ClientRMService", errorMessage);
       throw RPCUtil.getRemoteException(errorMessage);
     }
     // Check if it is a managed queue
     return getPlanFromQueue(reservationSystem, queue,
-            AuditConstants.LIST_RESERVATION_REQUEST);
+        AuditConstants.LIST_RESERVATION_REQUEST);
   }
 
   /**
@@ -312,7 +300,7 @@ public class ReservationInputValidator {
    * @param request the {@link ReservationDeleteRequest} defining the resources
    *          required over time for the request
    * @return the {@link Plan} to submit the request to
-   * @throws YarnException
+   * @throws YarnException if validation fails
    */
   public Plan validateReservationDeleteRequest(
       ReservationSystem reservationSystem, ReservationDeleteRequest request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
index 8b62972..a6c8fcf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -29,8 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ReservationsACLsManager;
 
-import java.util.Map;
-
 /**
  * This interface is the one implemented by any system that wants to support
  * Reservations i.e. make {@code Resource} allocations in future. Implementors
@@ -57,7 +57,7 @@ public interface ReservationSystem extends Recoverable {
    * 
    * @param conf configuration
    * @param rmContext current context of the {@code ResourceManager}
-   * @throws YarnException
+   * @throws YarnException if initialization of the configured plan fails
    */
   void reinitialize(Configuration conf, RMContext rmContext)
       throws YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index e458055..cbf0f38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -38,7 +38,7 @@ public interface SharingPolicy {
    * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  public void init(String planQueuePath, ReservationSchedulerConfiguration conf);
+  void init(String planQueuePath, ReservationSchedulerConfiguration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on
@@ -51,7 +51,7 @@ public interface SharingPolicy {
    * @throws PlanningException if the policy is respected if we add this
    *           {@link ReservationAllocation} to the {@link Plan}
    */
-  public void validate(Plan plan, ReservationAllocation newAllocation)
+  void validate(Plan plan, ReservationAllocation newAllocation)
       throws PlanningException;
 
   /**
@@ -68,9 +68,13 @@ public interface SharingPolicy {
    * @param start the start time for the range we are querying
    * @param end the end time for the range we are querying
    * @param oldId (optional) the id of a reservation being updated
+   *
+   * @return the available resources expressed as a
+   *         {@link RLESparseResourceAllocation}
+   *
    * @throws PlanningException throws if the request is not valid
    */
-  public RLESparseResourceAllocation availableResources(
+  RLESparseResourceAllocation availableResources(
       RLESparseResourceAllocation available, Plan plan, String user,
       ReservationId oldId, long start, long end) throws PlanningException;
 
@@ -82,7 +86,6 @@ public interface SharingPolicy {
    * 
    * @return validWindow the window of validity considered by the policy.
    */
-  public long getValidWindow();
-
+  long getValidWindow();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
index abac6ac..af0e712 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/Planner.java
@@ -34,7 +34,7 @@ public interface Planner {
    *
    * @param plan the {@link Plan} to replan
    * @param contracts the list of reservation requests
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   public void plan(Plan plan, List<ReservationDefinition> contracts)
       throws PlanningException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
index 199bfa5..bbbf0d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/PlanningAlgorithm.java
@@ -50,7 +50,7 @@ public abstract class PlanningAlgorithm implements ReservationAgent {
    * @return whether the allocateUser function was successful or not
    *
    * @throws PlanningException if the session cannot be fitted into the plan
-   * @throws ContractValidationException
+   * @throws ContractValidationException if validation fails
    */
   protected boolean allocateUser(ReservationId reservationId, String user,
       Plan plan, ReservationDefinition contract,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
index ec6d9c0..8934b0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocator.java
@@ -50,7 +50,7 @@ public interface StageAllocator {
    *
    * @return The computed allocation (or null if the stage could not be
    *         allocated)
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
   Map<ReservationInterval, Resource> computeStageAllocation(Plan plan,
       RLESparseResourceAllocation planLoads,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
index da04336..d107487 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedy.java
@@ -69,7 +69,7 @@ public class StageAllocatorGreedy implements StageAllocator {
 
     RLESparseResourceAllocation netAvailable =
         plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+            stageDeadline, 0);
 
     netAvailable =
         RLESparseResourceAllocation.merge(plan.getResourceCalculator(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index ec83e02..ae7d91a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -83,9 +83,8 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
     int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
 
     // get available resources from plan
-    RLESparseResourceAllocation netRLERes =
-        plan.getAvailableResourceOverTime(user, oldId, stageEarliestStart,
-            stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageEarliestStart, stageDeadline, 0);
 
     // remove plan modifications
     netRLERes =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
index e45f58c..c014549 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorLowCostAligned.java
@@ -77,8 +77,8 @@ public class StageAllocatorLowCostAligned implements StageAllocator {
     ResourceCalculator resCalc = plan.getResourceCalculator();
     Resource capacity = plan.getTotalCapacity();
 
-    RLESparseResourceAllocation netRLERes = plan
-        .getAvailableResourceOverTime(user, oldId, stageArrival, stageDeadline);
+    RLESparseResourceAllocation netRLERes = plan.getAvailableResourceOverTime(
+        user, oldId, stageArrival, stageDeadline, 0);
 
     long step = plan.getStep();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index e99842e..5337e06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anySetOf;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import java.io.FileWriter;
 import java.io.IOException;
@@ -76,7 +79,8 @@ public class ReservationSystemTestUtil {
       String reservationQ, long timeWindow, float instConstraint,
       float avgConstraint) {
 
-    ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration();
+    ReservationSchedulerConfiguration realConf =
+        new CapacitySchedulerConfiguration();
     ReservationSchedulerConfiguration conf = spy(realConf);
     when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
     when(conf.getInstantaneousMaxCapacity(reservationQ))
@@ -168,7 +172,6 @@ public class ReservationSystemTestUtil {
     scheduler.start();
     scheduler.reinitialize(conf, rmContext);
 
-
     Resource resource =
         ReservationSystemTestUtil.calculateClusterResource(numContainers);
     RMNode node1 = MockNodes.newNodeInfo(1, resource, 1, "127.0.0.1");
@@ -184,10 +187,16 @@ public class ReservationSystemTestUtil {
 
   public static ReservationDefinition createSimpleReservationDefinition(
       long arrival, long deadline, long duration, int parallelism) {
+    return createSimpleReservationDefinition(arrival, deadline, duration,
+        parallelism, null);
+  }
+
+  public static ReservationDefinition createSimpleReservationDefinition(
+      long arrival, long deadline, long duration, int parallelism,
+      String recurrenceExpression) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            parallelism, parallelism, duration);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), parallelism, parallelism, duration);
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationRequests reqs = new ReservationRequestsPBImpl();
     reqs.setReservationResources(Collections.singletonList(r));
@@ -195,32 +204,31 @@ public class ReservationSystemTestUtil {
     rDef.setReservationRequests(reqs);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    if (recurrenceExpression != null) {
+      rDef.setRecurrenceExpression(recurrenceExpression);
+    }
     return rDef;
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
-    return createSimpleReservationRequest(reservationId, numContainers,
-        arrival, deadline, duration, Priority.UNDEFINED);
+    return createSimpleReservationRequest(reservationId, numContainers, arrival,
+        deadline, duration, Priority.UNDEFINED);
   }
 
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            numContainers, 1, duration);
-    ReservationRequests reqs =
-        ReservationRequests.newInstance(Collections.singletonList(r),
-            ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef =
-        ReservationDefinition.newInstance(arrival, deadline, reqs,
-            "testClientRMService#reservation", "0", priority);
-    ReservationSubmissionRequest request =
-        ReservationSubmissionRequest.newInstance(rDef,
-            reservationQ, reservationId);
+    ReservationRequest r = ReservationRequest
+        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
+    ReservationRequests reqs = ReservationRequests.newInstance(
+        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
+        deadline, reqs, "testClientRMService#reservation", "0", priority);
+    ReservationSubmissionRequest request = ReservationSubmissionRequest
+        .newInstance(rDef, reservationQ, reservationId);
     return request;
   }
 
@@ -252,9 +260,9 @@ public class ReservationSystemTestUtil {
     return cs;
   }
 
-  @SuppressWarnings("rawtypes") public static void initializeRMContext(
-      int numContainers, AbstractYarnScheduler scheduler,
-      RMContext mockRMContext) {
+  @SuppressWarnings("rawtypes")
+  public static void initializeRMContext(int numContainers,
+      AbstractYarnScheduler scheduler, RMContext mockRMContext) {
 
     when(mockRMContext.getScheduler()).thenReturn(scheduler);
     Resource r = calculateClusterResource(numContainers);
@@ -262,26 +270,25 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(
-        new RMContextImpl(null, null, null, null, null, null,
-            new RMContainerTokenSecretManager(conf),
-            new NMTokenSecretManagerInRM(conf),
-            new ClientToAMTokenSecretManagerInRM(), null));
+    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
+        null, null, null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null));
 
     RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
     when(nlm.getQueueResource(any(String.class), anySetOf(String.class),
-            any(Resource.class))).thenAnswer(new Answer<Resource>() {
-      @Override public Resource answer(InvocationOnMock invocation)
-          throws Throwable {
-        Object[] args = invocation.getArguments();
-        return (Resource) args[2];
-      }
-    });
+        any(Resource.class))).thenAnswer(new Answer<Resource>() {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
+            Object[] args = invocation.getArguments();
+            return (Resource) args[2];
+          }
+        });
 
     when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
         .thenAnswer(new Answer<Resource>() {
-          @Override public Resource answer(InvocationOnMock invocation)
-              throws Throwable {
+          @Override
+          public Resource answer(InvocationOnMock invocation) throws Throwable {
             Object[] args = invocation.getArguments();
             return (Resource) args[1];
           }
@@ -304,9 +311,8 @@ public class ReservationSystemTestUtil {
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
 
-    final String dedicated =
-        CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT
-            + reservationQ;
+    final String dedicated = CapacitySchedulerConfiguration.ROOT
+        + CapacitySchedulerConfiguration.DOT + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
     conf.setReservable(dedicated, true);
@@ -405,26 +411,55 @@ public class ReservationSystemTestUtil {
 
   public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
+    return generateAllocation(startTime, step, alloc, null);
+  }
+
+  public static Map<ReservationInterval, Resource> generateAllocation(
+      long startTime, long step, int[] alloc, String recurrenceExpression) {
     Map<ReservationInterval, Resource> req = new TreeMap<>();
-    for (int i = 0; i < alloc.length; i++) {
-      req.put(new ReservationInterval(startTime + i * step,
-          startTime + (i + 1) * step), ReservationSystemUtil.toResource(
-          ReservationRequest
-              .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+    long period = 0;
+    if (recurrenceExpression != null) {
+      period = Long.parseLong(recurrenceExpression);
+    }
+
+    long rStart;
+    long rEnd;
+    for (int j = 0; j < 86400000; j += period) {
+      for (int i = 0; i < alloc.length; i++) {
+        rStart = (startTime + i * step) + j * period;
+        rEnd = (startTime + (i + 1) * step) + j * period;
+        if (period > 0) {
+          rStart = rStart % period + j * period;
+          rEnd = rEnd % period + j * period;
+          if (rStart > rEnd) {
+            // skip wrap-around entry
+            continue;
+          }
+        }
+
+        req.put(new ReservationInterval(rStart, rEnd),
+            ReservationSystemUtil.toResource(ReservationRequest
+                .newInstance(Resource.newInstance(1024, 1), alloc[i])));
+
+      }
+      // execute only once if non-periodic
+      if (period == 0) {
+        break;
+      }
     }
     return req;
   }
 
-  public static RLESparseResourceAllocation
-      generateRLESparseResourceAllocation(int[] alloc, long[] timeSteps) {
+  public static RLESparseResourceAllocation generateRLESparseResourceAllocation(
+      int[] alloc, long[] timeSteps) {
     TreeMap<Long, Resource> allocationsMap = new TreeMap<>();
     for (int i = 0; i < alloc.length; i++) {
       allocationsMap.put(timeSteps[i],
           Resource.newInstance(alloc[i], alloc[i]));
     }
-    RLESparseResourceAllocation rleVector =
-        new RLESparseResourceAllocation(allocationsMap,
-            new DefaultResourceCalculator());
+    RLESparseResourceAllocation rleVector = new RLESparseResourceAllocation(
+        allocationsMap, new DefaultResourceCalculator());
     return rleVector;
   }
 


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


[28/48] 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/cd09291f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cd09291f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cd09291f

Branch: refs/heads/HDFS-10467
Commit: cd09291fc57e9ebedfec7d1867064e0a97ecbe67
Parents: 275980b
Author: Inigo <in...@apache.org>
Authored: Tue Mar 28 14:30:59 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:07 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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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


[48/48] 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/f514ba4b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f514ba4b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f514ba4b

Branch: refs/heads/HDFS-10467
Commit: f514ba4b6a98af231dcbc2204dd26df6f087fd1f
Parents: e91ecb6
Author: Inigo Goiri <in...@apache.org>
Authored: Wed Aug 16 17:31:37 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:10 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/f514ba4b/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/f514ba4b/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


[39/48] 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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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


[46/48] 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/37449af2/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


[26/48] hadoop git commit: YARN-7075. Better styling for donut charts in new YARN UI. Contributed by Da Ding.

Posted by in...@apache.org.
YARN-7075. Better styling for donut charts in new YARN UI. Contributed by Da Ding.


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

Branch: refs/heads/HDFS-10467
Commit: 275980bb1e3ae2dd5c64673e3cb384eb104a947d
Parents: 7996eca
Author: Sunil G <su...@apache.org>
Authored: Sat Sep 2 08:08:59 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Sep 2 08:08:59 2017 +0530

----------------------------------------------------------------------
 .../main/webapp/app/components/donut-chart.js   |  5 +++-
 .../src/main/webapp/app/styles/app.css          | 24 ++++++++++++++++++--
 2 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/275980bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
index 82d2d46..b1e6ecf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
@@ -53,7 +53,10 @@ export default BaseChartComponent.extend({
 
     // 50 is for title
     var outerRadius = (h - 50 - 2 * layout.margin) / 2;
-    var innerRadius = outerRadius * 0.618;
+
+    // Ratio of inner radius to outer radius
+    var radiusRatio = 0.75;
+    var innerRadius = outerRadius * radiusRatio;
 
     var arc = d3.svg.arc()
       .innerRadius(innerRadius)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/275980bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
index 8b8ea56..38e25e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
@@ -20,6 +20,8 @@ body, html, body > .ember-view {
   height: 100%;
   overflow: visible;
   color: @text-color;
+  background: #f6f6f8;
+  font-family: "Open Sans","Helvetica Neue", sans-serif!important;
 }
 body, html {
   min-width: 1024px;
@@ -29,11 +31,11 @@ body, html {
  Over all style
  */
 text {
-  font: 16px sans-serif;
+  font-size: 14px;
 }
 
 text.small {
-  font: 8px sans-serif;
+  font-size: 8px;
 }
 
 html, body
@@ -184,6 +186,7 @@ table.dataTable thead .sorting_desc_disabled {
 
 .breadcrumb {
   padding-bottom: 3px;
+  background-color: #f6f6f8;
 }
 
 .navbar-default .navbar-nav > li > a {
@@ -268,12 +271,29 @@ td {
   margin: 0 auto -40px; // Must be same as footer & footer-frame
 }
 
+.panel {
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-radius: 15px;
+  -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);
+  box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);
+  overflow: hidden;
+}
+
 .panel-default .container-fluid {
   margin-top: -45px !important;
   margin-bottom: -10px !important;
 }
 
+.panel-default > .panel-heading {
+  background-image: none;
+  background-color: #f5f5f5 !important;
+  border-radius: 15px 15px 0px 0px;
+}
+
+
 .panel-heading {
+  background-color: rgba(0, 0, 0, 0) !important;
   font-weight: bold;
 }
 


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


[38/48] 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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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


[42/48] 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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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


[41/48] 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/a8d935b4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a8d935b4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a8d935b4

Branch: refs/heads/HDFS-10467
Commit: a8d935b45536db54e66f3f30e1a61966bb1cfaef
Parents: 02c53b9
Author: Inigo Goiri <in...@apache.org>
Authored: Tue Aug 1 14:40:27 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:09 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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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/a8d935b4/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


[05/48] hadoop git commit: YARN-7141. Move logging APIs to slf4j in timelineservice after ATSv2 merge. Contributed by Varun Saxena

Posted by in...@apache.org.
YARN-7141. Move logging APIs to slf4j in timelineservice after ATSv2 merge. Contributed by Varun Saxena


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

Branch: refs/heads/HDFS-10467
Commit: dcd0bedcc8113dd5e1d852c242ae9511d1b3d58e
Parents: 1fbb662
Author: bibinchundatt <bi...@apache.org>
Authored: Fri Sep 1 11:29:16 2017 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Fri Sep 1 11:29:16 2017 +0530

----------------------------------------------------------------------
 .../resourcemanager/TestRMHATimelineCollectors.java |  4 ----
 .../storage/reader/EntityTypeReader.java            | 16 ++++++----------
 .../storage/subapplication/SubApplicationTable.java |  6 +++---
 .../collector/AppLevelTimelineCollectorWithAgg.java | 15 ++++++---------
 .../TimelineReaderWhitelistAuthorizationFilter.java |  8 ++++----
 ...melineV2DelegationTokenSecretManagerService.java |  8 ++++----
 6 files changed, 23 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcd0bedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java
index fa0d318..a0cd048 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
@@ -40,8 +38,6 @@ import static org.junit.Assert.assertEquals;
  * transition.
  */
 public class TestRMHATimelineCollectors extends RMHATestBase {
-  public static final Log LOG = LogFactory
-      .getLog(TestSubmitApplicationWithRMHA.class);
 
   @Before
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcd0bedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
index 05570f1..3f72334 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
@@ -36,6 +34,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -49,7 +49,8 @@ import java.util.TreeSet;
  */
 public final class EntityTypeReader extends AbstractTimelineStorageReader {
 
-  private static final Log LOG = LogFactory.getLog(EntityTypeReader.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EntityTypeReader.class);
   private static final EntityTable ENTITY_TABLE = new EntityTable();
 
   public EntityTypeReader(TimelineReaderContext context) {
@@ -85,9 +86,7 @@ public final class EntityTypeReader extends AbstractTimelineStorageReader {
     typeFilterList.addFilter(new FirstKeyOnlyFilter());
     typeFilterList.addFilter(new KeyOnlyFilter());
     typeFilterList.addFilter(new PageFilter(1));
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("FilterList created for scan is - " + typeFilterList);
-    }
+    LOG.debug("FilterList created for scan is - {}", typeFilterList);
 
     int counter = 0;
     while (true) {
@@ -110,10 +109,7 @@ public final class EntityTypeReader extends AbstractTimelineStorageReader {
         currRowKey = getNextRowKey(prefix.getRowKeyPrefix(), currType);
       }
     }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Scanned " + counter + "records for "
-          + types.size() + "types");
-    }
+    LOG.debug("Scanned {} records for {} types", counter, types.size());
     return types;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcd0bedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java
index 334bab6..785a243 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.subapplication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The sub application table has column families:
@@ -103,7 +103,7 @@ public class SubApplicationTable extends BaseTable<SubApplicationTable> {
   /** default max number of versions. */
   private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000;
 
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       SubApplicationTable.class);
 
   public SubApplicationTable() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcd0bedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java
index 6c0d693..d7f47c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java
@@ -19,14 +19,14 @@
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 
 import java.util.HashSet;
@@ -46,7 +46,8 @@ import java.util.concurrent.TimeUnit;
 @Unstable
 public class AppLevelTimelineCollectorWithAgg
     extends AppLevelTimelineCollector {
-  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollector.class);
 
   private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1;
   private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15;
@@ -111,9 +112,7 @@ public class AppLevelTimelineCollectorWithAgg
   private class AppLevelAggregator implements Runnable {
 
     private void aggregate() {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("App-level real-time aggregating");
-      }
+      LOG.debug("App-level real-time aggregating");
       if (!isReadyToAggregate()) {
         LOG.warn("App-level collector is not ready, skip aggregation. ");
         return;
@@ -136,9 +135,7 @@ public class AppLevelTimelineCollectorWithAgg
       } catch (Exception e) {
         LOG.error("Error aggregating timeline metrics", e);
       }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("App-level real-time aggregation complete");
-      }
+      LOG.debug("App-level real-time aggregation complete");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcd0bedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
index b22ea3f..8093fcf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
@@ -31,13 +31,13 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderWebServicesUtils;
 
 /**
@@ -48,8 +48,8 @@ public class TimelineReaderWhitelistAuthorizationFilter implements Filter {
 
   public static final String EMPTY_STRING = "";
 
-  private static final Log LOG =
-      LogFactory.getLog(TimelineReaderWhitelistAuthorizationFilter.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineReaderWhitelistAuthorizationFilter.class);
 
   private boolean isWhitelistReadAuthEnabled = false;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcd0bedc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java
index de5ccdc..1a8bf09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.yarn.server.timelineservice.security;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.io.Text;
@@ -30,6 +28,8 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineDelgationTokenSecretManagerService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The service wrapper of {@link TimelineV2DelegationTokenSecretManager}.
@@ -75,8 +75,8 @@ public class TimelineV2DelegationTokenSecretManagerService extends
   public static class TimelineV2DelegationTokenSecretManager extends
       AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
 
-    private static final Log LOG =
-        LogFactory.getLog(TimelineV2DelegationTokenSecretManager.class);
+    private static final Logger LOG =
+        LoggerFactory.getLogger(TimelineV2DelegationTokenSecretManager.class);
 
     /**
      * Create a timeline v2 secret manager.


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


[22/48] hadoop git commit: YARN-7023. Incorrect ReservationId.compareTo() implementation.

Posted by in...@apache.org.
YARN-7023. Incorrect ReservationId.compareTo() implementation.

This closes #262

Signed-off-by: Akira Ajisaka <aa...@apache.org>


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

Branch: refs/heads/HDFS-10467
Commit: c5281a85e108c7cab6e84dfe351843483d720b9b
Parents: 0d419c9
Author: Oleg Danilov <ol...@olegd.com>
Authored: Wed Aug 16 15:19:26 2017 +0300
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Sep 1 21:33:31 2017 +0200

----------------------------------------------------------------------
 .../apache/hadoop/yarn/api/records/ReservationId.java    | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5281a85/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
index a0c209d..fd01115 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.util.Records;
  * {@link ReservationId} represents the <em>globally unique</em> identifier for
  * a reservation.
  * </p>
- * 
+ *
  * <p>
  * The globally unique nature of the identifier is achieved by using the
  * <em>cluster timestamp</em> i.e. start-time of the {@code ResourceManager}
@@ -62,7 +62,7 @@ public abstract class ReservationId implements Comparable<ReservationId> {
    * Get the long identifier of the {@link ReservationId} which is unique for
    * all Reservations started by a particular instance of the
    * {@code ResourceManager}.
-   * 
+   *
    * @return long identifier of the {@link ReservationId}
    */
   @Public
@@ -76,7 +76,7 @@ public abstract class ReservationId implements Comparable<ReservationId> {
   /**
    * Get the <em>start time</em> of the {@code ResourceManager} which is used to
    * generate globally unique {@link ReservationId}.
-   * 
+   *
    * @return <em>start time</em> of the {@code ResourceManager}
    */
   @Public
@@ -94,10 +94,9 @@ public abstract class ReservationId implements Comparable<ReservationId> {
   @Override
   public int compareTo(ReservationId other) {
     if (this.getClusterTimestamp() - other.getClusterTimestamp() == 0) {
-      return getId() > getId() ? 1 : getId() < getId() ? -1 : 0;
+      return Long.compare(getId(), other.getId());
     } else {
-      return this.getClusterTimestamp() > other.getClusterTimestamp() ? 1
-          : this.getClusterTimestamp() < other.getClusterTimestamp() ? -1 : 0;
+      return Long.compare(getClusterTimestamp(), other.getClusterTimestamp());
     }
   }
 


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


[15/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 3fbdcb0..f846689 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -79,6 +79,9 @@ class S3ABlockOutputStream extends OutputStream {
   /** Size of all blocks. */
   private final int blockSize;
 
+  /** Total bytes for uploads submitted so far. */
+  private long bytesSubmitted;
+
   /** Callback for progress. */
   private final ProgressListener progressListener;
   private final ListeningExecutorService executorService;
@@ -302,6 +305,7 @@ class S3ABlockOutputStream extends OutputStream {
     }
     try {
       multiPartUpload.uploadBlockAsync(getActiveBlock());
+      bytesSubmitted += getActiveBlock().dataSize();
     } finally {
       // set the block to null, so the next write will create a new block.
       clearActiveBlock();
@@ -330,13 +334,14 @@ class S3ABlockOutputStream extends OutputStream {
         this,
         blockCount,
         hasBlock ? block : "(none)");
+    long bytes = 0;
     try {
       if (multiPartUpload == null) {
         if (hasBlock) {
           // no uploads of data have taken place, put the single block up.
           // This must happen even if there is no data, so that 0 byte files
           // are created.
-          putObject();
+          bytes = putObject();
         }
       } else {
         // there has already been at least one block scheduled for upload;
@@ -350,6 +355,7 @@ class S3ABlockOutputStream extends OutputStream {
             multiPartUpload.waitForAllPartUploads();
         // then complete the operation
         multiPartUpload.complete(partETags);
+        bytes = bytesSubmitted;
       }
       LOG.debug("Upload complete for {}", writeOperationHelper);
     } catch (IOException ioe) {
@@ -362,7 +368,7 @@ class S3ABlockOutputStream extends OutputStream {
       clearActiveBlock();
     }
     // All end of write operations, including deleting fake parent directories
-    writeOperationHelper.writeSuccessful();
+    writeOperationHelper.writeSuccessful(bytes);
   }
 
   /**
@@ -370,8 +376,11 @@ class S3ABlockOutputStream extends OutputStream {
    * is empty a 0-byte PUT will be invoked, as it is needed to create an
    * entry at the far end.
    * @throws IOException any problem.
+   * @return number of bytes uploaded. If thread was interrupted while
+   * waiting for upload to complete, returns zero with interrupted flag set
+   * on this thread.
    */
-  private void putObject() throws IOException {
+  private int putObject() throws IOException {
     LOG.debug("Executing regular upload for {}", writeOperationHelper);
 
     final S3ADataBlocks.DataBlock block = getActiveBlock();
@@ -405,9 +414,11 @@ class S3ABlockOutputStream extends OutputStream {
     //wait for completion
     try {
       putObjectResult.get();
+      return size;
     } catch (InterruptedException ie) {
       LOG.warn("Interrupted object upload", ie);
       Thread.currentThread().interrupt();
+      return 0;
     } catch (ExecutionException ee) {
       throw extractException("regular upload", key, ee);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
index b0f08e3..be08afe 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.Path;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AFileStatus extends FileStatus {
-  private boolean isEmptyDirectory;
+  private Tristate isEmptyDirectory;
 
   /**
    * Create a directory status.
@@ -42,6 +42,18 @@ public class S3AFileStatus extends FileStatus {
   public S3AFileStatus(boolean isemptydir,
       Path path,
       String owner) {
+    this(Tristate.fromBool(isemptydir), path, owner);
+  }
+
+  /**
+   * Create a directory status.
+   * @param isemptydir is this an empty directory?
+   * @param path the path
+   * @param owner the owner
+   */
+  public S3AFileStatus(Tristate isemptydir,
+      Path path,
+      String owner) {
     super(0, true, 1, 0, 0, path);
     isEmptyDirectory = isemptydir;
     setOwner(owner);
@@ -59,12 +71,37 @@ public class S3AFileStatus extends FileStatus {
   public S3AFileStatus(long length, long modification_time, Path path,
       long blockSize, String owner) {
     super(length, false, 1, blockSize, modification_time, path);
-    isEmptyDirectory = false;
+    isEmptyDirectory = Tristate.FALSE;
     setOwner(owner);
     setGroup(owner);
   }
 
-  public boolean isEmptyDirectory() {
+  /**
+   * Convenience constructor for creating from a vanilla FileStatus plus
+   * an isEmptyDirectory flag.
+   * @param source FileStatus to convert to S3AFileStatus
+   * @param isEmptyDirectory TRUE/FALSE if known to be / not be an empty
+   *     directory, UNKNOWN if that information was not computed.
+   * @return a new S3AFileStatus
+   */
+  public static S3AFileStatus fromFileStatus(FileStatus source,
+      Tristate isEmptyDirectory) {
+    if (source.isDirectory()) {
+      return new S3AFileStatus(isEmptyDirectory, source.getPath(),
+          source.getOwner());
+    } else {
+      return new S3AFileStatus(source.getLen(), source.getModificationTime(),
+          source.getPath(), source.getBlockSize(), source.getOwner());
+    }
+  }
+
+
+  /**
+   * @return FALSE if status is not a directory, or its a dir, but known to
+   * not be empty.  TRUE if it is an empty directory.  UNKNOWN if it is a
+   * directory, but we have not computed whether or not it is empty.
+   */
+  public Tristate isEmptyDirectory() {
     return isEmptyDirectory;
   }
 
@@ -110,7 +147,7 @@ public class S3AFileStatus extends FileStatus {
   @Override
   public String toString() {
     return super.toString() +
-        String.format(" isEmptyDirectory=%s", isEmptyDirectory());
+        String.format(" isEmptyDirectory=%s", isEmptyDirectory().name());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 872dd5f..c22383a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -25,12 +25,16 @@ import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ExecutorService;
+import java.util.Set;
 import java.util.Objects;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -92,6 +96,11 @@ import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
@@ -149,6 +158,8 @@ public class S3AFileSystem extends FileSystem {
   private long readAhead;
   private S3AInputPolicy inputPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
+  private MetadataStore metadataStore;
+  private boolean allowAuthoritative;
 
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
@@ -277,6 +288,10 @@ public class S3AFileSystem extends FileSystem {
       } else {
         LOG.debug("Using S3AOutputStream");
       }
+
+      metadataStore = S3Guard.getMetadataStore(this);
+      allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
+          DEFAULT_METADATASTORE_AUTHORITATIVE);
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
     }
@@ -388,12 +403,35 @@ public class S3AFileSystem extends FileSystem {
    * Returns the S3 client used by this filesystem.
    * @return AmazonS3Client
    */
-  @VisibleForTesting
   AmazonS3 getAmazonS3Client() {
     return s3;
   }
 
   /**
+   * Get the region of a bucket.
+   * @return the region in which a bucket is located
+   * @throws IOException on any failure.
+   */
+  public String getBucketLocation() throws IOException {
+    return getBucketLocation(bucket);
+  }
+
+  /**
+   * Get the region of a bucket.
+   * @param bucketName the name of the bucket
+   * @return the region in which a bucket is located
+   * @throws IOException on any failure.
+   */
+  public String getBucketLocation(String bucketName) throws IOException {
+    try {
+      return s3.getBucketLocation(bucketName);
+    } catch (AmazonClientException e) {
+      throw translateException("getBucketLocation()",
+          bucketName, e);
+    }
+  }
+
+  /**
    * Returns the read ahead range value used by this filesystem
    * @return
    */
@@ -457,7 +495,7 @@ public class S3AFileSystem extends FileSystem {
    * @return a key excluding the leading "/", or, if it is the root path, ""
    */
   @VisibleForTesting
-  String pathToKey(Path path) {
+  public String pathToKey(Path path) {
     if (!path.isAbsolute()) {
       path = new Path(workingDir, path);
     }
@@ -508,7 +546,7 @@ public class S3AFileSystem extends FileSystem {
    * @param path path to qualify
    * @return a qualified path.
    */
-  Path qualify(Path path) {
+  public Path qualify(Path path) {
     return path.makeQualified(uri, workingDir);
   }
 
@@ -578,7 +616,7 @@ public class S3AFileSystem extends FileSystem {
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
     String key = pathToKey(f);
-    S3AFileStatus status = null;
+    FileStatus status = null;
     try {
       // get the status or throw an FNFE
       status = getFileStatus(f);
@@ -706,8 +744,8 @@ public class S3AFileSystem extends FileSystem {
    * the description of the operation.
    * This operation throws an exception on any failure which needs to be
    * reported and downgraded to a failure. That is: if a rename
-   * @param src path to be renamed
-   * @param dst new path after rename
+   * @param source path to be renamed
+   * @param dest new path after rename
    * @throws RenameFailedException if some criteria for a state changing
    * rename was not met. This means work didn't happen; it's not something
    * which is reported upstream to the FileSystem APIs, for which the semantics
@@ -716,9 +754,12 @@ public class S3AFileSystem extends FileSystem {
    * @throws IOException on IO failure.
    * @throws AmazonClientException on failures inside the AWS SDK
    */
-  private boolean innerRename(Path src, Path dst)
+  private boolean innerRename(Path source, Path dest)
       throws RenameFailedException, FileNotFoundException, IOException,
         AmazonClientException {
+    Path src = qualify(source);
+    Path dst = qualify(dest);
+
     LOG.debug("Rename path {} to {}", src, dst);
     incrementStatistic(INVOCATION_RENAME);
 
@@ -734,7 +775,7 @@ public class S3AFileSystem extends FileSystem {
 
     // get the source file status; this raises a FNFE if there is no source
     // file.
-    S3AFileStatus srcStatus = getFileStatus(src);
+    S3AFileStatus srcStatus = innerGetFileStatus(src, true);
 
     if (srcKey.equals(dstKey)) {
       LOG.debug("rename: src and dest refer to the same file or directory: {}",
@@ -746,7 +787,7 @@ public class S3AFileSystem extends FileSystem {
 
     S3AFileStatus dstStatus = null;
     try {
-      dstStatus = getFileStatus(dst);
+      dstStatus = innerGetFileStatus(dst, true);
       // if there is no destination entry, an exception is raised.
       // hence this code sequence can assume that there is something
       // at the end of the path; the only detail being what it is and
@@ -756,7 +797,7 @@ public class S3AFileSystem extends FileSystem {
           throw new RenameFailedException(src, dst,
               "source is a directory and dest is a file")
               .withExitCode(srcStatus.isFile());
-        } else if (!dstStatus.isEmptyDirectory()) {
+        } else if (dstStatus.isEmptyDirectory() != Tristate.TRUE) {
           throw new RenameFailedException(src, dst,
               "Destination is a non-empty directory")
               .withExitCode(false);
@@ -778,7 +819,8 @@ public class S3AFileSystem extends FileSystem {
       Path parent = dst.getParent();
       if (!pathToKey(parent).isEmpty()) {
         try {
-          S3AFileStatus dstParentStatus = getFileStatus(dst.getParent());
+          S3AFileStatus dstParentStatus = innerGetFileStatus(dst.getParent(),
+              false);
           if (!dstParentStatus.isDirectory()) {
             throw new RenameFailedException(src, dst,
                 "destination parent is not a directory");
@@ -790,9 +832,20 @@ public class S3AFileSystem extends FileSystem {
       }
     }
 
+    // If we have a MetadataStore, track deletions/creations.
+    Collection<Path> srcPaths = null;
+    List<PathMetadata> dstMetas = null;
+    if (hasMetadataStore()) {
+      srcPaths = new HashSet<>(); // srcPaths need fast look up before put
+      dstMetas = new ArrayList<>();
+    }
+    // TODO S3Guard HADOOP-13761: retries when source paths are not visible yet
+    // TODO S3Guard: performance: mark destination dirs as authoritative
+
     // Ok! Time to start
     if (srcStatus.isFile()) {
       LOG.debug("rename: renaming file {} to {}", src, dst);
+      long length = srcStatus.getLen();
       if (dstStatus != null && dstStatus.isDirectory()) {
         String newDstKey = dstKey;
         if (!newDstKey.endsWith("/")) {
@@ -801,9 +854,14 @@ public class S3AFileSystem extends FileSystem {
         String filename =
             srcKey.substring(pathToKey(src.getParent()).length()+1);
         newDstKey = newDstKey + filename;
-        copyFile(srcKey, newDstKey, srcStatus.getLen());
+        copyFile(srcKey, newDstKey, length);
+        S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src,
+            keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst),
+            username);
       } else {
         copyFile(srcKey, dstKey, srcStatus.getLen());
+        S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst,
+            length, getDefaultBlockSize(dst), username);
       }
       innerDelete(srcStatus, false);
     } else {
@@ -825,42 +883,66 @@ public class S3AFileSystem extends FileSystem {
       }
 
       List<DeleteObjectsRequest.KeyVersion> keysToDelete = new ArrayList<>();
-      if (dstStatus != null && dstStatus.isEmptyDirectory()) {
+      if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) {
         // delete unnecessary fake directory.
         keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey));
       }
 
-      ListObjectsRequest request = new ListObjectsRequest();
-      request.setBucketName(bucket);
-      request.setPrefix(srcKey);
-      request.setMaxKeys(maxKeys);
-
-      ObjectListing objects = listObjects(request);
-
-      while (true) {
-        for (S3ObjectSummary summary : objects.getObjectSummaries()) {
-          keysToDelete.add(
-              new DeleteObjectsRequest.KeyVersion(summary.getKey()));
-          String newDstKey =
-              dstKey + summary.getKey().substring(srcKey.length());
-          copyFile(summary.getKey(), newDstKey, summary.getSize());
-
-          if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
-            removeKeys(keysToDelete, true, false);
+      Path parentPath = keyToPath(srcKey);
+      RemoteIterator<LocatedFileStatus> iterator = listFilesAndEmptyDirectories(
+          parentPath, true);
+      while (iterator.hasNext()) {
+        LocatedFileStatus status = iterator.next();
+        long length = status.getLen();
+        String key = pathToKey(status.getPath());
+        if (status.isDirectory() && !key.endsWith("/")) {
+          key += "/";
+        }
+        keysToDelete
+            .add(new DeleteObjectsRequest.KeyVersion(key));
+        String newDstKey =
+            dstKey + key.substring(srcKey.length());
+        copyFile(key, newDstKey, length);
+
+        if (hasMetadataStore()) {
+          // with a metadata store, the object entries need to be updated,
+          // including, potentially, the ancestors
+          Path childSrc = keyToQualifiedPath(key);
+          Path childDst = keyToQualifiedPath(newDstKey);
+          if (objectRepresentsDirectory(key, length)) {
+            S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, childSrc,
+                childDst, username);
+          } else {
+            S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, childSrc,
+                childDst, length, getDefaultBlockSize(childDst), username);
           }
+          // Ancestor directories may not be listed, so we explicitly add them
+          S3Guard.addMoveAncestors(metadataStore, srcPaths, dstMetas,
+              keyToQualifiedPath(srcKey), childSrc, childDst, username);
         }
 
-        if (objects.isTruncated()) {
-          objects = continueListObjects(objects);
-        } else {
-          if (!keysToDelete.isEmpty()) {
-            removeKeys(keysToDelete, false, false);
-          }
-          break;
+        if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
+          removeKeys(keysToDelete, true, false);
         }
       }
+      if (!keysToDelete.isEmpty()) {
+        removeKeys(keysToDelete, false, false);
+      }
+
+      // We moved all the children, now move the top-level dir
+      // Empty directory should have been added as the object summary
+      if (hasMetadataStore()
+          && srcPaths != null
+          && !srcPaths.contains(src)) {
+        LOG.debug("To move the non-empty top-level dir src={} and dst={}",
+            src, dst);
+        S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, src, dst,
+            username);
+      }
     }
 
+    metadataStore.move(srcPaths, dstMetas);
+
     if (src.getParent() != dst.getParent()) {
       deleteUnnecessaryFakeDirectories(dst.getParent());
       createFakeDirectoryIfNecessary(src.getParent());
@@ -880,6 +962,31 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Does this Filesystem have a metadata store?
+   * @return true iff the FS has been instantiated with a metadata store
+   */
+  public boolean hasMetadataStore() {
+    return !S3Guard.isNullMetadataStore(metadataStore);
+  }
+
+  /**
+   * Get the metadata store.
+   * This will always be non-null, but may be bound to the
+   * {@code NullMetadataStore}.
+   * @return the metadata store of this FS instance
+   */
+  @VisibleForTesting
+  MetadataStore getMetadataStore() {
+    return metadataStore;
+  }
+
+  /** For testing only.  See ITestS3GuardEmptyDirs. */
+  @VisibleForTesting
+  void setMetadataStore(MetadataStore ms) {
+    metadataStore = ms;
+  }
+
+  /**
    * Increment a statistic by 1.
    * @param statistic The operation to increment
    */
@@ -1063,8 +1170,9 @@ public class S3AFileSystem extends FileSystem {
    * @param inputStream source data.
    * @return the request
    */
-  private PutObjectRequest newPutObjectRequest(String key,
-      ObjectMetadata metadata, InputStream inputStream) {
+  PutObjectRequest newPutObjectRequest(String key,
+      ObjectMetadata metadata,
+      InputStream inputStream) {
     Preconditions.checkNotNull(inputStream);
     PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
         inputStream, metadata);
@@ -1115,7 +1223,7 @@ public class S3AFileSystem extends FileSystem {
    * @param putObjectRequest the request
    * @return the upload initiated
    */
-  public Upload putObject(PutObjectRequest putObjectRequest) {
+  public UploadInfo putObject(PutObjectRequest putObjectRequest) {
     long len;
     if (putObjectRequest.getFile() != null) {
       len = putObjectRequest.getFile().length();
@@ -1126,7 +1234,7 @@ public class S3AFileSystem extends FileSystem {
     try {
       Upload upload = transfers.upload(putObjectRequest);
       incrementPutCompletedStatistics(true, len);
-      return upload;
+      return new UploadInfo(upload, len);
     } catch (AmazonClientException e) {
       incrementPutCompletedStatistics(false, len);
       throw e;
@@ -1142,14 +1250,10 @@ public class S3AFileSystem extends FileSystem {
    * @return the upload initiated
    * @throws AmazonClientException on problems
    */
-  public PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
+  PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
       throws AmazonClientException {
-    long len;
-    if (putObjectRequest.getFile() != null) {
-      len = putObjectRequest.getFile().length();
-    } else {
-      len = putObjectRequest.getMetadata().getContentLength();
-    }
+    long len = getPutRequestLength(putObjectRequest);
+    LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey());
     incrementPutStartStatistics(len);
     try {
       PutObjectResult result = s3.putObject(putObjectRequest);
@@ -1162,6 +1266,23 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Get the length of the PUT, verifying that the length is known.
+   * @param putObjectRequest a request bound to a file or a stream.
+   * @return the request length
+   * @throws IllegalArgumentException if the length is negative
+   */
+  private long getPutRequestLength(PutObjectRequest putObjectRequest) {
+    long len;
+    if (putObjectRequest.getFile() != null) {
+      len = putObjectRequest.getFile().length();
+    } else {
+      len = putObjectRequest.getMetadata().getContentLength();
+    }
+    Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length");
+    return len;
+  }
+
+  /**
    * Upload part of a multi-partition file.
    * Increments the write and put counters.
    * <i>Important: this call does not close any input stream in the request.</i>
@@ -1288,7 +1409,7 @@ public class S3AFileSystem extends FileSystem {
    */
   public boolean delete(Path f, boolean recursive) throws IOException {
     try {
-      return innerDelete(getFileStatus(f), recursive);
+      return innerDelete(innerGetFileStatus(f, true), recursive);
     } catch (FileNotFoundException e) {
       LOG.debug("Couldn't delete {} - does not exist", f);
       instrumentation.errorIgnored();
@@ -1318,6 +1439,9 @@ public class S3AFileSystem extends FileSystem {
 
     if (status.isDirectory()) {
       LOG.debug("delete: Path is a directory: {}", f);
+      Preconditions.checkArgument(
+          status.isEmptyDirectory() != Tristate.UNKNOWN,
+          "File status must have directory emptiness computed");
 
       if (!key.endsWith("/")) {
         key = key + "/";
@@ -1327,13 +1451,15 @@ public class S3AFileSystem extends FileSystem {
         return rejectRootDirectoryDelete(status, recursive);
       }
 
-      if (!recursive && !status.isEmptyDirectory()) {
+      if (!recursive && status.isEmptyDirectory() == Tristate.FALSE) {
         throw new PathIsNotEmptyDirectoryException(f.toString());
       }
 
-      if (status.isEmptyDirectory()) {
+      if (status.isEmptyDirectory() == Tristate.TRUE) {
         LOG.debug("Deleting fake empty directory {}", key);
+        // HADOOP-13761 S3Guard: retries here
         deleteObject(key);
+        metadataStore.delete(f);
         instrumentation.directoryDeleted();
       } else {
         LOG.debug("Getting objects for directory prefix {} to delete", key);
@@ -1349,6 +1475,7 @@ public class S3AFileSystem extends FileSystem {
             LOG.debug("Got object to delete {}", summary.getKey());
 
             if (keys.size() == MAX_ENTRIES_TO_DELETE) {
+              // TODO: HADOOP-13761 S3Guard: retries
               removeKeys(keys, true, false);
             }
           }
@@ -1357,16 +1484,19 @@ public class S3AFileSystem extends FileSystem {
             objects = continueListObjects(objects);
           } else {
             if (!keys.isEmpty()) {
+              // TODO: HADOOP-13761 S3Guard: retries
               removeKeys(keys, false, false);
             }
             break;
           }
         }
       }
+      metadataStore.deleteSubtree(f);
     } else {
       LOG.debug("delete: Path is a file");
       instrumentation.fileDeleted(1);
       deleteObject(key);
+      metadataStore.delete(f);
     }
 
     Path parent = f.getParent();
@@ -1390,7 +1520,7 @@ public class S3AFileSystem extends FileSystem {
   private boolean rejectRootDirectoryDelete(S3AFileStatus status,
       boolean recursive) throws IOException {
     LOG.info("s3a delete the {} root directory of {}", bucket, recursive);
-    boolean emptyRoot = status.isEmptyDirectory();
+    boolean emptyRoot = status.isEmptyDirectory() == Tristate.TRUE;
     if (emptyRoot) {
       return true;
     }
@@ -1405,7 +1535,7 @@ public class S3AFileSystem extends FileSystem {
   private void createFakeDirectoryIfNecessary(Path f)
       throws IOException, AmazonClientException {
     String key = pathToKey(f);
-    if (!key.isEmpty() && !exists(f)) {
+    if (!key.isEmpty() && !s3Exists(f)) {
       LOG.debug("Creating new fake directory at {}", f);
       createFakeDirectory(key);
     }
@@ -1454,6 +1584,11 @@ public class S3AFileSystem extends FileSystem {
         key = key + '/';
       }
 
+      DirListingMetadata dirMeta = metadataStore.listChildren(path);
+      if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
+        return S3Guard.dirMetaToStatuses(dirMeta);
+      }
+
       ListObjectsRequest request = createListObjectsRequest(key, "/");
       LOG.debug("listStatus: doing listObjects for directory {}", key);
 
@@ -1466,7 +1601,8 @@ public class S3AFileSystem extends FileSystem {
       while (files.hasNext()) {
         result.add(files.next());
       }
-      return result.toArray(new FileStatus[result.size()]);
+      return S3Guard.dirListingUnion(metadataStore, path, result, dirMeta,
+          allowAuthoritative);
     } else {
       LOG.debug("Adding: rd (not a dir): {}", path);
       FileStatus[] stats = new FileStatus[1];
@@ -1482,7 +1618,8 @@ public class S3AFileSystem extends FileSystem {
    * @param delimiter any delimiter
    * @return the request
    */
-  private ListObjectsRequest createListObjectsRequest(String key,
+  @VisibleForTesting
+  ListObjectsRequest createListObjectsRequest(String key,
       String delimiter) {
     ListObjectsRequest request = new ListObjectsRequest();
     request.setBucketName(bucket);
@@ -1541,23 +1678,30 @@ public class S3AFileSystem extends FileSystem {
       throw translateException("innerMkdirs", path, e);
     }
   }
+
   /**
    *
    * Make the given path and all non-existent parents into
    * directories.
    * See {@link #mkdirs(Path, FsPermission)}
-   * @param f path to create
+   * @param p path to create
    * @param permission to apply to f
-   * @return true if a directory was created
+   * @return true if a directory was created or already existed
    * @throws FileAlreadyExistsException there is a file at the path specified
    * @throws IOException other IO problems
    * @throws AmazonClientException on failures inside the AWS SDK
    */
-  private boolean innerMkdirs(Path f, FsPermission permission)
+  private boolean innerMkdirs(Path p, FsPermission permission)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
+    Path f = qualify(p);
     LOG.debug("Making directory: {}", f);
     incrementStatistic(INVOCATION_MKDIRS);
     FileStatus fileStatus;
+    List<Path> metadataStoreDirs = null;
+    if (hasMetadataStore()) {
+      metadataStoreDirs = new ArrayList<>();
+    }
+
     try {
       fileStatus = getFileStatus(f);
 
@@ -1567,8 +1711,12 @@ public class S3AFileSystem extends FileSystem {
         throw new FileAlreadyExistsException("Path is a file: " + f);
       }
     } catch (FileNotFoundException e) {
+      // Walk path to root, ensuring closest ancestor is a directory, not file
       Path fPart = f.getParent();
-      do {
+      if (metadataStoreDirs != null) {
+        metadataStoreDirs.add(f);
+      }
+      while (fPart != null) {
         try {
           fileStatus = getFileStatus(fPart);
           if (fileStatus.isDirectory()) {
@@ -1581,12 +1729,17 @@ public class S3AFileSystem extends FileSystem {
           }
         } catch (FileNotFoundException fnfe) {
           instrumentation.errorIgnored();
+          // We create all missing directories in MetadataStore; it does not
+          // infer directories exist by prefix like S3.
+          if (metadataStoreDirs != null) {
+            metadataStoreDirs.add(fPart);
+          }
         }
         fPart = fPart.getParent();
-      } while (fPart != null);
-
+      }
       String key = pathToKey(f);
       createFakeDirectory(key);
+      S3Guard.makeDirsOrdered(metadataStore, metadataStoreDirs, username, true);
       // this is complicated because getParent(a/b/c/) returns a/b/c, but
       // we want a/b. See HADOOP-14428 for more details.
       deleteUnnecessaryFakeDirectories(new Path(f.toString()).getParent());
@@ -1598,21 +1751,93 @@ public class S3AFileSystem extends FileSystem {
    * Return a file status object that represents the path.
    * @param f The path we want information from
    * @return a FileStatus object
-   * @throws java.io.FileNotFoundException when the path does not exist;
+   * @throws FileNotFoundException when the path does not exist
    * @throws IOException on other problems.
    */
-  public S3AFileStatus getFileStatus(final Path f) throws IOException {
+  public FileStatus getFileStatus(final Path f) throws IOException {
+    return innerGetFileStatus(f, false);
+  }
+
+  /**
+   * Internal version of {@link #getFileStatus(Path)}.
+   * @param f The path we want information from
+   * @param needEmptyDirectoryFlag if true, implementation will calculate
+   *        a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()}
+   * @return a S3AFileStatus object
+   * @throws FileNotFoundException when the path does not exist
+   * @throws IOException on other problems.
+   */
+  @VisibleForTesting
+  S3AFileStatus innerGetFileStatus(final Path f,
+      boolean needEmptyDirectoryFlag) throws IOException {
     incrementStatistic(INVOCATION_GET_FILE_STATUS);
     final Path path = qualify(f);
     String key = pathToKey(path);
-    LOG.debug("Getting path status for {}  ({})", path , key);
+    LOG.debug("Getting path status for {}  ({})", path, key);
+
+    // Check MetadataStore, if any.
+    PathMetadata pm = metadataStore.get(path, needEmptyDirectoryFlag);
+    Set<Path> tombstones = Collections.EMPTY_SET;
+    if (pm != null) {
+      if (pm.isDeleted()) {
+        throw new FileNotFoundException("Path " + f + " is recorded as " +
+            "deleted by S3Guard");
+      }
+
+      FileStatus msStatus = pm.getFileStatus();
+      if (needEmptyDirectoryFlag && msStatus.isDirectory()) {
+        if (pm.isEmptyDirectory() != Tristate.UNKNOWN) {
+          // We have a definitive true / false from MetadataStore, we are done.
+          return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory());
+        } else {
+          DirListingMetadata children = metadataStore.listChildren(path);
+          if (children != null) {
+            tombstones = children.listTombstones();
+          }
+          LOG.debug("MetadataStore doesn't know if dir is empty, using S3.");
+        }
+      } else {
+        // Either this is not a directory, or we don't care if it is empty
+        return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory());
+      }
+
+      // If the metadata store has no children for it and it's not listed in
+      // S3 yet, we'll assume the empty directory is true;
+      S3AFileStatus s3FileStatus;
+      try {
+        s3FileStatus = s3GetFileStatus(path, key, tombstones);
+      } catch (FileNotFoundException e) {
+        return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE);
+      }
+      // entry was found, save in S3Guard
+      return S3Guard.putAndReturn(metadataStore, s3FileStatus, instrumentation);
+    } else {
+      // there was no entry in S3Guard
+      // retrieve the data and update the metadata store in the process.
+      return S3Guard.putAndReturn(metadataStore,
+          s3GetFileStatus(path, key, tombstones), instrumentation);
+    }
+  }
+
+  /**
+   * Raw {@code getFileStatus} that talks direct to S3.
+   * Used to implement {@link #innerGetFileStatus(Path, boolean)},
+   * and for direct management of empty directory blobs.
+   * @param path Qualified path
+   * @param key  Key string for the path
+   * @return Status
+   * @throws FileNotFoundException when the path does not exist
+   * @throws IOException on other problems.
+   */
+  private S3AFileStatus s3GetFileStatus(final Path path, String key,
+      Set<Path> tombstones) throws IOException {
     if (!key.isEmpty()) {
       try {
         ObjectMetadata meta = getObjectMetadata(key);
 
         if (objectRepresentsDirectory(key, meta.getContentLength())) {
           LOG.debug("Found exact file: fake directory");
-          return new S3AFileStatus(true, path, username);
+          return new S3AFileStatus(Tristate.TRUE, path, username);
         } else {
           LOG.debug("Found exact file: normal file");
           return new S3AFileStatus(meta.getContentLength(),
@@ -1637,16 +1862,16 @@ public class S3AFileSystem extends FileSystem {
 
           if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
             LOG.debug("Found file (with /): fake directory");
-            return new S3AFileStatus(true, path, username);
+            return new S3AFileStatus(Tristate.TRUE, path, username);
           } else {
             LOG.warn("Found file (with /): real file? should not happen: {}",
                 key);
 
             return new S3AFileStatus(meta.getContentLength(),
-                dateToLong(meta.getLastModified()),
-                path,
-                getDefaultBlockSize(path),
-                username);
+                    dateToLong(meta.getLastModified()),
+                    path,
+                    getDefaultBlockSize(path),
+                    username);
           }
         } catch (AmazonServiceException e) {
           if (e.getStatusCode() != 404) {
@@ -1668,25 +1893,26 @@ public class S3AFileSystem extends FileSystem {
 
       ObjectListing objects = listObjects(request);
 
-      if (!objects.getCommonPrefixes().isEmpty()
-          || !objects.getObjectSummaries().isEmpty()) {
+      Collection<String> prefixes = objects.getCommonPrefixes();
+      Collection<S3ObjectSummary> summaries = objects.getObjectSummaries();
+      if (!isEmptyOfKeys(prefixes, tombstones) ||
+          !isEmptyOfObjects(summaries, tombstones)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Found path as directory (with /): {}/{}",
-              objects.getCommonPrefixes().size() ,
-              objects.getObjectSummaries().size());
+              prefixes.size(), summaries.size());
 
-          for (S3ObjectSummary summary : objects.getObjectSummaries()) {
+          for (S3ObjectSummary summary : summaries) {
             LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize());
           }
-          for (String prefix : objects.getCommonPrefixes()) {
+          for (String prefix : prefixes) {
             LOG.debug("Prefix: {}", prefix);
           }
         }
 
-        return new S3AFileStatus(false, path, username);
+        return new S3AFileStatus(Tristate.FALSE, path, username);
       } else if (key.isEmpty()) {
         LOG.debug("Found root directory");
-        return new S3AFileStatus(true, path, username);
+        return new S3AFileStatus(Tristate.TRUE, path, username);
       }
     } catch (AmazonServiceException e) {
       if (e.getStatusCode() != 404) {
@@ -1701,6 +1927,64 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Helper function to determine if a collection of paths is empty
+   * after accounting for tombstone markers (if provided).
+   * @param keys Collection of path (prefixes / directories or keys).
+   * @param tombstones Set of tombstone markers, or null if not applicable.
+   * @return false if summaries contains objects not accounted for by
+   * tombstones.
+   */
+  private boolean isEmptyOfKeys(Collection<String> keys, Set<Path>
+      tombstones) {
+    if (tombstones == null) {
+      return keys.isEmpty();
+    }
+    for (String key : keys) {
+      Path qualified = keyToQualifiedPath(key);
+      if (!tombstones.contains(qualified)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Helper function to determine if a collection of object summaries is empty
+   * after accounting for tombstone markers (if provided).
+   * @param summaries Collection of objects as returned by listObjects.
+   * @param tombstones Set of tombstone markers, or null if not applicable.
+   * @return false if summaries contains objects not accounted for by
+   * tombstones.
+   */
+  private boolean isEmptyOfObjects(Collection<S3ObjectSummary> summaries,
+      Set<Path> tombstones) {
+    if (tombstones == null) {
+      return summaries.isEmpty();
+    }
+    Collection<String> stringCollection = new ArrayList<>(summaries.size());
+    for (S3ObjectSummary summary : summaries) {
+      stringCollection.add(summary.getKey());
+    }
+    return isEmptyOfKeys(stringCollection, tombstones);
+  }
+
+  /**
+   * Raw version of {@link FileSystem#exists(Path)} which uses S3 only:
+   * S3Guard MetadataStore, if any, will be skipped.
+   * @return true if path exists in S3
+   */
+  private boolean s3Exists(final Path f) throws IOException {
+    Path path = qualify(f);
+    String key = pathToKey(path);
+    try {
+      s3GetFileStatus(path, key, null);
+      return true;
+    } catch (FileNotFoundException e) {
+      return false;
+    }
+  }
+
+  /**
    * The src file is on the local disk.  Add it to FS at
    * the given dst name.
    *
@@ -1777,12 +2061,13 @@ public class S3AFileSystem extends FileSystem {
     final String key = pathToKey(dst);
     final ObjectMetadata om = newObjectMetadata(srcfile.length());
     PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
-    Upload up = putObject(putObjectRequest);
+    UploadInfo info = putObject(putObjectRequest);
+    Upload upload = info.getUpload();
     ProgressableProgressListener listener = new ProgressableProgressListener(
-        this, key, up, null);
-    up.addProgressListener(listener);
+        this, key, upload, null);
+    upload.addProgressListener(listener);
     try {
-      up.waitForUploadResult();
+      upload.waitForUploadResult();
     } catch (InterruptedException e) {
       throw new InterruptedIOException("Interrupted copying " + src
           + " to "  + dst + ", cancelling");
@@ -1790,7 +2075,7 @@ public class S3AFileSystem extends FileSystem {
     listener.uploadCompleted();
 
     // This will delete unnecessary fake parent directories
-    finishedWrite(key);
+    finishedWrite(key, info.getLength());
 
     if (delSrc) {
       local.delete(src, false);
@@ -1814,6 +2099,10 @@ public class S3AFileSystem extends FileSystem {
         transfers.shutdownNow(true);
         transfers = null;
       }
+      if (metadataStore != null) {
+        metadataStore.close();
+        metadataStore = null;
+      }
     }
   }
 
@@ -1956,11 +2245,38 @@ public class S3AFileSystem extends FileSystem {
 
   /**
    * Perform post-write actions.
+   * This operation MUST be called after any PUT/multipart PUT completes
+   * successfully.
+   * This includes
+   * <ol>
+   *   <li>Calling {@link #deleteUnnecessaryFakeDirectories(Path)}</li>
+   *   <li>Updating any metadata store with details on the newly created
+   *   object.</li>
+   * </ol>
    * @param key key written to
+   * @param length  total length of file written
    */
-  public void finishedWrite(String key) {
-    LOG.debug("Finished write to {}", key);
-    deleteUnnecessaryFakeDirectories(keyToPath(key).getParent());
+  @InterfaceAudience.Private
+  void finishedWrite(String key, long length) {
+    LOG.debug("Finished write to {}, len {}", key, length);
+    Path p = keyToQualifiedPath(key);
+    deleteUnnecessaryFakeDirectories(p.getParent());
+    Preconditions.checkArgument(length >= 0, "content length is negative");
+
+    // See note about failure semantics in S3Guard documentation
+    try {
+      if (hasMetadataStore()) {
+        S3Guard.addAncestors(metadataStore, p, username);
+        S3AFileStatus status = createUploadFileStatus(p,
+            S3AUtils.objectRepresentsDirectory(key, length), length,
+            getDefaultBlockSize(p), username);
+        S3Guard.putAndReturn(metadataStore, status, instrumentation);
+      }
+    } catch (IOException e) {
+      LOG.error("S3Guard: Error updating MetadataStore for write to {}:",
+          key, e);
+      instrumentation.errorIgnored();
+    }
   }
 
   /**
@@ -2015,9 +2331,9 @@ public class S3AFileSystem extends FileSystem {
     PutObjectRequest putObjectRequest = newPutObjectRequest(objectName,
         newObjectMetadata(0L),
         im);
-    Upload upload = putObject(putObjectRequest);
+    UploadInfo info = putObject(putObjectRequest);
     try {
-      upload.waitForUploadResult();
+      info.getUpload().waitForUploadResult();
     } catch (InterruptedException e) {
       throw new InterruptedIOException("Interrupted creating " + objectName);
     }
@@ -2123,6 +2439,8 @@ public class S3AFileSystem extends FileSystem {
     if (blockFactory != null) {
       sb.append(", blockFactory=").append(blockFactory);
     }
+    sb.append(", metastore=").append(metadataStore);
+    sb.append(", authoritative=").append(allowAuthoritative);
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", statistics {")
@@ -2241,6 +2559,18 @@ public class S3AFileSystem extends FileSystem {
   @Override
   public RemoteIterator<LocatedFileStatus> listFiles(Path f,
       boolean recursive) throws FileNotFoundException, IOException {
+    return innerListFiles(f, recursive,
+        new Listing.AcceptFilesOnly(qualify(f)));
+  }
+
+  public RemoteIterator<LocatedFileStatus> listFilesAndEmptyDirectories(Path f,
+      boolean recursive) throws IOException {
+    return innerListFiles(f, recursive,
+        new Listing.AcceptAllButS3nDirs());
+  }
+
+  private RemoteIterator<LocatedFileStatus> innerListFiles(Path f, boolean
+      recursive, Listing.FileStatusAcceptor acceptor) throws IOException {
     incrementStatistic(INVOCATION_LIST_FILES);
     Path path = qualify(f);
     LOG.debug("listFiles({}, {})", path, recursive);
@@ -2258,13 +2588,42 @@ public class S3AFileSystem extends FileSystem {
         String delimiter = recursive ? null : "/";
         LOG.debug("Requesting all entries under {} with delimiter '{}'",
             key, delimiter);
-        return listing.createLocatedFileStatusIterator(
-            listing.createFileStatusListingIterator(path,
-                createListObjectsRequest(key, delimiter),
-                ACCEPT_ALL,
-                new Listing.AcceptFilesOnly(path)));
+        final RemoteIterator<FileStatus> cachedFilesIterator;
+        final Set<Path> tombstones;
+        if (recursive) {
+          final PathMetadata pm = metadataStore.get(path, true);
+          // shouldn't need to check pm.isDeleted() because that will have
+          // been caught by getFileStatus above.
+          MetadataStoreListFilesIterator metadataStoreListFilesIterator =
+              new MetadataStoreListFilesIterator(metadataStore, pm,
+                  allowAuthoritative);
+          tombstones = metadataStoreListFilesIterator.listTombstones();
+          cachedFilesIterator = metadataStoreListFilesIterator;
+        } else {
+          DirListingMetadata meta = metadataStore.listChildren(path);
+          if (meta != null) {
+            tombstones = meta.listTombstones();
+          } else {
+            tombstones = null;
+          }
+          cachedFilesIterator = listing.createProvidedFileStatusIterator(
+              S3Guard.dirMetaToStatuses(meta), ACCEPT_ALL, acceptor);
+          if (allowAuthoritative && meta != null && meta.isAuthoritative()) {
+            // metadata listing is authoritative, so return it directly
+            return listing.createLocatedFileStatusIterator(cachedFilesIterator);
+          }
+        }
+        return listing.createTombstoneReconcilingIterator(
+            listing.createLocatedFileStatusIterator(
+                listing.createFileStatusListingIterator(path,
+                    createListObjectsRequest(key, delimiter),
+                    ACCEPT_ALL,
+                    acceptor,
+                    cachedFilesIterator)),
+            tombstones);
       }
     } catch (AmazonClientException e) {
+      // TODO S3Guard: retry on file not found exception
       throw translateException("listFiles", path, e);
     }
   }
@@ -2309,12 +2668,21 @@ public class S3AFileSystem extends FileSystem {
             filter.accept(path) ? toLocatedFileStatus(fileStatus) : null);
       } else {
         // directory: trigger a lookup
-        String key = maybeAddTrailingSlash(pathToKey(path));
-        return listing.createLocatedFileStatusIterator(
-            listing.createFileStatusListingIterator(path,
-                createListObjectsRequest(key, "/"),
-                filter,
-                new Listing.AcceptAllButSelfAndS3nDirs(path)));
+        final String key = maybeAddTrailingSlash(pathToKey(path));
+        final Listing.FileStatusAcceptor acceptor =
+            new Listing.AcceptAllButSelfAndS3nDirs(path);
+        DirListingMetadata meta = metadataStore.listChildren(path);
+        final RemoteIterator<FileStatus> cachedFileStatusIterator =
+            listing.createProvidedFileStatusIterator(
+                S3Guard.dirMetaToStatuses(meta), filter, acceptor);
+        return (allowAuthoritative && meta != null && meta.isAuthoritative())
+            ? listing.createLocatedFileStatusIterator(cachedFileStatusIterator)
+            : listing.createLocatedFileStatusIterator(
+                listing.createFileStatusListingIterator(path,
+                    createListObjectsRequest(key, "/"),
+                    filter,
+                    acceptor,
+                    cachedFileStatusIterator));
       }
     } catch (AmazonClientException e) {
       throw translateException("listLocatedStatus", path, e);
@@ -2389,8 +2757,8 @@ public class S3AFileSystem extends FileSystem {
     /**
      * Callback on a successful write.
      */
-    void writeSuccessful() {
-      finishedWrite(key);
+    void writeSuccessful(long length) {
+      finishedWrite(key, length);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index d2e7a88..da1fc5a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -23,6 +23,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.Interns;
@@ -30,6 +31,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableMetric;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 
 import java.io.Closeable;
 import java.net.URI;
@@ -38,7 +40,6 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import org.apache.hadoop.fs.FileSystem.Statistics;
 
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 
@@ -90,6 +91,10 @@ public class S3AInstrumentation {
   private final Map<String, MutableCounterLong> streamMetrics =
       new HashMap<>(30);
 
+  /** Instantiate this without caring whether or not S3Guard is enabled. */
+  private final S3GuardInstrumentation s3GuardInstrumentation
+      = new S3GuardInstrumentation();
+
   private static final Statistic[] COUNTERS_TO_CREATE = {
       INVOCATION_COPY_FROM_LOCAL_FILE,
       INVOCATION_EXISTS,
@@ -117,6 +122,8 @@ public class S3AInstrumentation {
       STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
       STREAM_WRITE_TOTAL_TIME,
       STREAM_WRITE_TOTAL_DATA,
+      S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
+      S3GUARD_METADATASTORE_INITIALIZATION
   };
 
 
@@ -171,6 +178,9 @@ public class S3AInstrumentation {
     for (Statistic statistic : GAUGES_TO_CREATE) {
       gauge(statistic.getSymbol(), statistic.getDescription());
     }
+    //todo need a config for the quantiles interval?
+    quantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
+        "ops", "latency", 1);
   }
 
   /**
@@ -227,6 +237,22 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Create a quantiles in the registry.
+   * @param op  statistic to collect
+   * @param sampleName sample name of the quantiles
+   * @param valueName value name of the quantiles
+   * @param interval interval of the quantiles in seconds
+   * @return the created quantiles metric
+   */
+  protected final MutableQuantiles quantiles(Statistic op,
+      String sampleName,
+      String valueName,
+      int interval) {
+    return registry.newQuantiles(op.getSymbol(), op.getDescription(),
+        sampleName, valueName, interval);
+  }
+
+  /**
    * Get the metrics registry.
    * @return the registry
    */
@@ -311,6 +337,20 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Look up a quantiles.
+   * @param name quantiles name
+   * @return the quantiles or null
+   * @throws ClassCastException if the metric is not a Quantiles.
+   */
+  public MutableQuantiles lookupQuantiles(String name) {
+    MutableMetric metric = lookupMetric(name);
+    if (metric == null) {
+      LOG.debug("No quantiles {}", name);
+    }
+    return (MutableQuantiles) metric;
+  }
+
+  /**
    * Look up a metric from both the registered set and the lighter weight
    * stream entries.
    * @param name metric name
@@ -391,6 +431,21 @@ public class S3AInstrumentation {
       counter.incr(count);
     }
   }
+
+  /**
+   * Add a value to a quantiles statistic. No-op if the quantile
+   * isn't found.
+   * @param op operation to look up.
+   * @param value value to add.
+   * @throws ClassCastException if the metric is not a Quantiles.
+   */
+  public void addValueToQuantiles(Statistic op, long value) {
+    MutableQuantiles quantiles = lookupQuantiles(op.getSymbol());
+    if (quantiles != null) {
+      quantiles.add(value);
+    }
+  }
+
   /**
    * Increment a specific counter.
    * No-op if not defined.
@@ -442,6 +497,15 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Create a S3Guard instrumentation instance.
+   * There's likely to be at most one instance of this per FS instance.
+   * @return the S3Guard instrumentation point.
+   */
+  public S3GuardInstrumentation getS3GuardInstrumentation() {
+    return s3GuardInstrumentation;
+  }
+
+  /**
    * Merge in the statistics of a single input stream into
    * the filesystem-wide statistics.
    * @param statistics stream statistics
@@ -840,4 +904,19 @@ public class S3AInstrumentation {
       return sb.toString();
     }
   }
+
+  /**
+   * Instrumentation exported to S3Guard.
+   */
+  public final class S3GuardInstrumentation {
+
+    /** Initialized event. */
+    public void initialized() {
+      incrementCounter(S3GUARD_METADATASTORE_INITIALIZATION, 1);
+    }
+
+    public void storeClosed() {
+
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
index 6ebc9e4..e723b75 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.transfer.Upload;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -101,19 +100,20 @@ public class S3AOutputStream extends OutputStream {
 
     try {
       final ObjectMetadata om = fs.newObjectMetadata(backupFile.length());
-      Upload upload = fs.putObject(
+      UploadInfo info = fs.putObject(
           fs.newPutObjectRequest(
               key,
               om,
               backupFile));
       ProgressableProgressListener listener =
-          new ProgressableProgressListener(fs, key, upload, progress);
-      upload.addProgressListener(listener);
+          new ProgressableProgressListener(fs, key, info.getUpload(), progress);
+      info.getUpload().addProgressListener(listener);
 
-      upload.waitForUploadResult();
+      info.getUpload().waitForUploadResult();
       listener.uploadCompleted();
-      // This will delete unnecessary fake parent directories
-      fs.finishedWrite(key);
+      // This will delete unnecessary fake parent directories, update any
+      // MetadataStore
+      fs.finishedWrite(key, info.getLength());
     } catch (InterruptedException e) {
       throw (InterruptedIOException) new InterruptedIOException(e.toString())
           .initCause(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 27406b6..9dd5def 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -294,12 +294,38 @@ public final class S3AUtils {
       S3ObjectSummary summary,
       long blockSize,
       String owner) {
-    if (objectRepresentsDirectory(summary.getKey(), summary.getSize())) {
-      return new S3AFileStatus(true, keyPath, owner);
+    long size = summary.getSize();
+    return createFileStatus(keyPath,
+        objectRepresentsDirectory(summary.getKey(), size),
+        size, summary.getLastModified(), blockSize, owner);
+  }
+
+  /**
+   * Create a file status for object we just uploaded.  For files, we use
+   * current time as modification time, since s3a uses S3's service-based
+   * modification time, which will not be available until we do a
+   * getFileStatus() later on.
+   * @param keyPath path for created object
+   * @param isDir true iff directory
+   * @param size file length
+   * @param blockSize block size for file status
+   * @param owner Hadoop username
+   * @return a status entry
+   */
+  public static S3AFileStatus createUploadFileStatus(Path keyPath,
+      boolean isDir, long size, long blockSize, String owner) {
+    Date date = isDir ? null : new Date();
+    return createFileStatus(keyPath, isDir, size, date, blockSize, owner);
+  }
+
+  /* Date 'modified' is ignored when isDir is true. */
+  private static S3AFileStatus createFileStatus(Path keyPath, boolean isDir,
+      long size, Date modified, long blockSize, String owner) {
+    if (isDir) {
+      return new S3AFileStatus(Tristate.UNKNOWN, keyPath, owner);
     } else {
-      return new S3AFileStatus(summary.getSize(),
-          dateToLong(summary.getLastModified()), keyPath,
-          blockSize, owner);
+      return new S3AFileStatus(size, dateToLong(modified), keyPath, blockSize,
+          owner);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index d4e09e3..e7603d9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -18,33 +18,20 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-
 import java.io.IOException;
 import java.net.URI;
 
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
-import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.S3ClientOptions;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.util.VersionInfo;
-
-import org.slf4j.Logger;
 
 /**
- * Factory for creation of S3 client instances to be used by {@link S3Store}.
+ * Factory for creation of {@link AmazonS3} client instances.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-interface S3ClientFactory {
+public interface S3ClientFactory {
 
   /**
    * Creates a new {@link AmazonS3} client.  This method accepts the S3A file
@@ -57,177 +44,4 @@ interface S3ClientFactory {
    */
   AmazonS3 createS3Client(URI name) throws IOException;
 
-  /**
-   * The default factory implementation, which calls the AWS SDK to configure
-   * and create an {@link AmazonS3Client} that communicates with the S3 service.
-   */
-  static class DefaultS3ClientFactory extends Configured
-      implements S3ClientFactory {
-
-    private static final Logger LOG = S3AFileSystem.LOG;
-
-    @Override
-    public AmazonS3 createS3Client(URI name) throws IOException {
-      Configuration conf = getConf();
-      AWSCredentialsProvider credentials =
-          createAWSCredentialProviderSet(name, conf);
-      ClientConfiguration awsConf = new ClientConfiguration();
-      initConnectionSettings(conf, awsConf);
-      initProxySupport(conf, awsConf);
-      initUserAgent(conf, awsConf);
-      return createAmazonS3Client(conf, credentials, awsConf);
-    }
-
-    /**
-     * Initializes all AWS SDK settings related to connection management.
-     *
-     * @param conf Hadoop configuration
-     * @param awsConf AWS SDK configuration
-     */
-    private static void initConnectionSettings(Configuration conf,
-        ClientConfiguration awsConf) {
-      awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
-          DEFAULT_MAXIMUM_CONNECTIONS, 1));
-      boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
-          DEFAULT_SECURE_CONNECTIONS);
-      awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
-      awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
-          DEFAULT_MAX_ERROR_RETRIES, 0));
-      awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
-          DEFAULT_ESTABLISH_TIMEOUT, 0));
-      awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
-          DEFAULT_SOCKET_TIMEOUT, 0));
-      int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
-          DEFAULT_SOCKET_SEND_BUFFER, 2048);
-      int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
-          DEFAULT_SOCKET_RECV_BUFFER, 2048);
-      awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
-      String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
-      if (!signerOverride.isEmpty()) {
-        LOG.debug("Signer override = {}", signerOverride);
-        awsConf.setSignerOverride(signerOverride);
-      }
-    }
-
-    /**
-     * Initializes AWS SDK proxy support if configured.
-     *
-     * @param conf Hadoop configuration
-     * @param awsConf AWS SDK configuration
-     * @throws IllegalArgumentException if misconfigured
-     */
-    private static void initProxySupport(Configuration conf,
-        ClientConfiguration awsConf) throws IllegalArgumentException {
-      String proxyHost = conf.getTrimmed(PROXY_HOST, "");
-      int proxyPort = conf.getInt(PROXY_PORT, -1);
-      if (!proxyHost.isEmpty()) {
-        awsConf.setProxyHost(proxyHost);
-        if (proxyPort >= 0) {
-          awsConf.setProxyPort(proxyPort);
-        } else {
-          if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
-            LOG.warn("Proxy host set without port. Using HTTPS default 443");
-            awsConf.setProxyPort(443);
-          } else {
-            LOG.warn("Proxy host set without port. Using HTTP default 80");
-            awsConf.setProxyPort(80);
-          }
-        }
-        String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
-        String proxyPassword = conf.getTrimmed(PROXY_PASSWORD);
-        if ((proxyUsername == null) != (proxyPassword == null)) {
-          String msg = "Proxy error: " + PROXY_USERNAME + " or " +
-              PROXY_PASSWORD + " set without the other.";
-          LOG.error(msg);
-          throw new IllegalArgumentException(msg);
-        }
-        awsConf.setProxyUsername(proxyUsername);
-        awsConf.setProxyPassword(proxyPassword);
-        awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
-        awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
-                  "domain {} as workstation {}", awsConf.getProxyHost(),
-              awsConf.getProxyPort(),
-              String.valueOf(awsConf.getProxyUsername()),
-              awsConf.getProxyPassword(), awsConf.getProxyDomain(),
-              awsConf.getProxyWorkstation());
-        }
-      } else if (proxyPort >= 0) {
-        String msg =
-            "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
-        LOG.error(msg);
-        throw new IllegalArgumentException(msg);
-      }
-    }
-
-    /**
-     * Initializes the User-Agent header to send in HTTP requests to the S3
-     * back-end.  We always include the Hadoop version number.  The user also
-     * may set an optional custom prefix to put in front of the Hadoop version
-     * number.  The AWS SDK interally appends its own information, which seems
-     * to include the AWS SDK version, OS and JVM version.
-     *
-     * @param conf Hadoop configuration
-     * @param awsConf AWS SDK configuration
-     */
-    private static void initUserAgent(Configuration conf,
-        ClientConfiguration awsConf) {
-      String userAgent = "Hadoop " + VersionInfo.getVersion();
-      String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
-      if (!userAgentPrefix.isEmpty()) {
-        userAgent = userAgentPrefix + ", " + userAgent;
-      }
-      LOG.debug("Using User-Agent: {}", userAgent);
-      awsConf.setUserAgentPrefix(userAgent);
-    }
-
-    /**
-     * Creates an {@link AmazonS3Client} from the established configuration.
-     *
-     * @param conf Hadoop configuration
-     * @param credentials AWS credentials
-     * @param awsConf AWS SDK configuration
-     * @return S3 client
-     * @throws IllegalArgumentException if misconfigured
-     */
-    private static AmazonS3 createAmazonS3Client(Configuration conf,
-        AWSCredentialsProvider credentials, ClientConfiguration awsConf)
-        throws IllegalArgumentException {
-      AmazonS3 s3 = new AmazonS3Client(credentials, awsConf);
-      String endPoint = conf.getTrimmed(ENDPOINT, "");
-      if (!endPoint.isEmpty()) {
-        try {
-          s3.setEndpoint(endPoint);
-        } catch (IllegalArgumentException e) {
-          String msg = "Incorrect endpoint: "  + e.getMessage();
-          LOG.error(msg);
-          throw new IllegalArgumentException(msg, e);
-        }
-      }
-      enablePathStyleAccessIfRequired(s3, conf);
-      return s3;
-    }
-
-    /**
-     * Enables path-style access to S3 buckets if configured.  By default, the
-     * behavior is to use virtual hosted-style access with URIs of the form
-     * http://bucketname.s3.amazonaws.com.  Enabling path-style access and a
-     * region-specific endpoint switches the behavior to use URIs of the form
-     * http://s3-eu-west-1.amazonaws.com/bucketname.
-     *
-     * @param s3 S3 client
-     * @param conf Hadoop configuration
-     */
-    private static void enablePathStyleAccessIfRequired(AmazonS3 s3,
-        Configuration conf) {
-      final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
-      if (pathStyleAccess) {
-        LOG.debug("Enabling path style access!");
-        s3.setS3ClientOptions(S3ClientOptions.builder()
-            .setPathStyleAccess(true)
-            .build());
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 789c6d7..777c161 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -140,7 +140,18 @@ public enum Statistic {
   STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
       "Count of total data uploaded in block output"),
   STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
-      "Total queue duration of all block uploads");
+      "Total queue duration of all block uploads"),
+
+  // S3Guard stats
+  S3GUARD_METADATASTORE_PUT_PATH_REQUEST(
+      "s3guard_metadatastore_put_path_request",
+      "s3guard metadata store put one metadata path request"),
+  S3GUARD_METADATASTORE_PUT_PATH_LATENCY(
+      "s3guard_metadatastore_put_path_latency",
+      "s3guard metadata store put one metadata path lantency"),
+  S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
+      "s3guard metadata store initialization times");
+
 
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java
new file mode 100644
index 0000000..0462ccf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java
@@ -0,0 +1,32 @@
+/*
+ * 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.s3a;
+
+/**
+ * Simple enum to express {true, false, don't know}.
+ */
+public enum Tristate {
+  // Do not add additional values here.  Logic will assume there are exactly
+  // three possibilities.
+  TRUE, FALSE, UNKNOWN;
+
+  public static Tristate fromBool(boolean v) {
+    return v ? TRUE : FALSE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
new file mode 100644
index 0000000..238cd97
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.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.fs.s3a;
+
+import com.amazonaws.services.s3.transfer.Upload;
+
+/**
+ * Simple struct that contains information about a S3 upload.
+ */
+public class UploadInfo {
+  private final Upload upload;
+  private final long length;
+
+  public UploadInfo(Upload upload, long length) {
+    this.upload = upload;
+    this.length = length;
+  }
+
+  public Upload getUpload() {
+    return upload;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
new file mode 100644
index 0000000..dcee358
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
@@ -0,0 +1,142 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * {@code DescendantsIterator} is a {@link RemoteIterator} that implements
+ * pre-ordering breadth-first traversal (BFS) of a path and all of its
+ * descendants recursively.  After visiting each path, that path's direct
+ * children are discovered by calling {@link MetadataStore#listChildren(Path)}.
+ * Each iteration returns the next direct child, and if that child is a
+ * directory, also pushes it onto a queue to discover its children later.
+ *
+ * For example, assume the consistent store contains metadata representing this
+ * file system structure:
+ *
+ * <pre>
+ * /dir1
+ * |-- dir2
+ * |   |-- file1
+ * |   `-- file2
+ * `-- dir3
+ *     |-- dir4
+ *     |   `-- file3
+ *     |-- dir5
+ *     |   `-- file4
+ *     `-- dir6
+ * </pre>
+ *
+ * Consider this code sample:
+ * <pre>
+ * final PathMetadata dir1 = get(new Path("/dir1"));
+ * for (DescendantsIterator descendants = new DescendantsIterator(dir1);
+ *     descendants.hasNext(); ) {
+ *   final FileStatus status = descendants.next().getFileStatus();
+ *   System.out.printf("%s %s%n", status.isDirectory() ? 'D' : 'F',
+ *       status.getPath());
+ * }
+ * </pre>
+ *
+ * The output is:
+ * <pre>
+ * D /dir1
+ * D /dir1/dir2
+ * D /dir1/dir3
+ * F /dir1/dir2/file1
+ * F /dir1/dir2/file2
+ * D /dir1/dir3/dir4
+ * D /dir1/dir3/dir5
+ * F /dir1/dir3/dir4/file3
+ * F /dir1/dir3/dir5/file4
+ * D /dir1/dir3/dir6
+ * </pre>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DescendantsIterator implements RemoteIterator<FileStatus> {
+
+  private final MetadataStore metadataStore;
+  private final Queue<PathMetadata> queue = new LinkedList<>();
+
+  /**
+   * Creates a new {@code DescendantsIterator}.
+   *
+   * @param ms the associated {@link MetadataStore}
+   * @param meta base path for descendants iteration, which will be the first
+   *     returned during iteration (except root). Null makes empty iterator.
+   * @throws IOException if errors happen during metadata store listing
+   */
+  public DescendantsIterator(MetadataStore ms, PathMetadata meta)
+      throws IOException {
+    Preconditions.checkNotNull(ms);
+    this.metadataStore = ms;
+
+    if (meta != null) {
+      final Path path = meta.getFileStatus().getPath();
+      if (path.isRoot()) {
+        DirListingMetadata rootListing = ms.listChildren(path);
+        if (rootListing != null) {
+          rootListing = rootListing.withoutTombstones();
+          queue.addAll(rootListing.getListing());
+        }
+      } else {
+        queue.add(meta);
+      }
+    }
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    return !queue.isEmpty();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!hasNext()) {
+      throw new NoSuchElementException("No more descendants.");
+    }
+    PathMetadata next;
+    next = queue.poll();
+    if (next.getFileStatus().isDirectory()) {
+      final Path path = next.getFileStatus().getPath();
+      DirListingMetadata meta = metadataStore.listChildren(path);
+      if (meta != null) {
+        Collection<PathMetadata> more = meta.withoutTombstones().getListing();
+        if (!more.isEmpty()) {
+          queue.addAll(more);
+        }
+      }
+    }
+    return next.getFileStatus();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
new file mode 100644
index 0000000..e5b4fb5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
@@ -0,0 +1,322 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * {@code DirListingMetadata} models a directory listing stored in a
+ * {@link MetadataStore}.  Instances of this class are mutable and thread-safe.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DirListingMetadata {
+
+  /**
+   * Convenience parameter for passing into constructor.
+   */
+  public static final Collection<PathMetadata> EMPTY_DIR =
+      Collections.emptyList();
+
+  private final Path path;
+
+  /** Using a map for fast find / remove with large directories. */
+  private Map<Path, PathMetadata> listMap = new ConcurrentHashMap<>();
+
+  private boolean isAuthoritative;
+
+  /**
+   * Create a directory listing metadata container.
+   *
+   * @param path Path of the directory. If this path has a host component, then
+   *     all paths added later via {@link #put(FileStatus)} must also have
+   *     the same host.
+   * @param listing Entries in the directory.
+   * @param isAuthoritative true iff listing is the full contents of the
+   *     directory, and the calling client reports that this may be cached as
+   *     the full and authoritative listing of all files in the directory.
+   */
+  public DirListingMetadata(Path path, Collection<PathMetadata> listing,
+      boolean isAuthoritative) {
+
+    checkPathAbsolute(path);
+    this.path = path;
+
+    if (listing != null) {
+      for (PathMetadata entry : listing) {
+        Path childPath = entry.getFileStatus().getPath();
+        checkChildPath(childPath);
+        listMap.put(childPath, entry);
+      }
+    }
+    this.isAuthoritative  = isAuthoritative;
+  }
+
+  /**
+   * Copy constructor.
+   * @param d the existing {@link DirListingMetadata} object.
+   */
+  public DirListingMetadata(DirListingMetadata d) {
+    path = d.path;
+    isAuthoritative = d.isAuthoritative;
+    listMap = new ConcurrentHashMap<>(d.listMap);
+  }
+
+  /**
+   * @return {@code Path} of the directory that contains this listing.
+   */
+  public Path getPath() {
+    return path;
+  }
+
+  /**
+   * @return entries in the directory
+   */
+  public Collection<PathMetadata> getListing() {
+    return Collections.unmodifiableCollection(listMap.values());
+  }
+
+  public Set<Path> listTombstones() {
+    Set<Path> tombstones = new HashSet<>();
+    for (PathMetadata meta : listMap.values()) {
+      if (meta.isDeleted()) {
+        tombstones.add(meta.getFileStatus().getPath());
+      }
+    }
+    return tombstones;
+  }
+
+  public DirListingMetadata withoutTombstones() {
+    Collection<PathMetadata> filteredList = new ArrayList<>();
+    for (PathMetadata meta : listMap.values()) {
+      if (!meta.isDeleted()) {
+        filteredList.add(meta);
+      }
+    }
+    return new DirListingMetadata(path, filteredList, isAuthoritative);
+  }
+
+  /**
+   * @return number of entries tracked.  This is not the same as the number
+   * of entries in the actual directory unless {@link #isAuthoritative()} is
+   * true.
+   */
+  public int numEntries() {
+    return listMap.size();
+  }
+
+  /**
+   * @return true iff this directory listing is full and authoritative within
+   * the scope of the {@code MetadataStore} that returned it.
+   */
+  public boolean isAuthoritative() {
+    return isAuthoritative;
+  }
+
+
+  /**
+   * Is the underlying directory known to be empty?
+   * @return FALSE if directory is known to have a child entry, TRUE if
+   * directory is known to be empty, UNKNOWN otherwise.
+   */
+  public Tristate isEmpty() {
+    if (getListing().isEmpty()) {
+      if (isAuthoritative()) {
+        return Tristate.TRUE;
+      } else {
+        // This listing is empty, but may not be full list of underlying dir.
+        return Tristate.UNKNOWN;
+      }
+    } else { // not empty listing
+      // There exists at least one child, dir not empty.
+      return Tristate.FALSE;
+    }
+  }
+
+  /**
+   * Marks this directory listing as full and authoritative.
+   * @param authoritative see {@link #isAuthoritative()}.
+   */
+  public void setAuthoritative(boolean authoritative) {
+    this.isAuthoritative = authoritative;
+  }
+
+  /**
+   * Lookup entry within this directory listing.  This may return null if the
+   * {@code MetadataStore} only tracks a partial set of the directory entries.
+   * In the case where {@link #isAuthoritative()} is true, however, this
+   * function returns null iff the directory is known not to contain the listing
+   * at given path (within the scope of the {@code MetadataStore} that returned
+   * it).
+   *
+   * @param childPath path of entry to look for.
+   * @return entry, or null if it is not present or not being tracked.
+   */
+  public PathMetadata get(Path childPath) {
+    checkChildPath(childPath);
+    return listMap.get(childPath);
+  }
+
+  /**
+   * Replace an entry with a tombstone.
+   * @param childPath path of entry to replace.
+   */
+  public void markDeleted(Path childPath) {
+    checkChildPath(childPath);
+    listMap.put(childPath, PathMetadata.tombstone(childPath));
+  }
+
+  /**
+   * Remove entry from this directory.
+   *
+   * @param childPath path of entry to remove.
+   */
+  public void remove(Path childPath) {
+    checkChildPath(childPath);
+    listMap.remove(childPath);
+  }
+
+  /**
+   * Add an entry to the directory listing.  If this listing already contains a
+   * {@code FileStatus} with the same path, it will be replaced.
+   *
+   * @param childFileStatus entry to add to this directory listing.
+   * @return true if the status was added or replaced with a new value. False
+   * if the same FileStatus value was already present.
+   */
+  public boolean put(FileStatus childFileStatus) {
+    Preconditions.checkNotNull(childFileStatus,
+        "childFileStatus must be non-null");
+    Path childPath = childStatusToPathKey(childFileStatus);
+    PathMetadata newValue = new PathMetadata(childFileStatus);
+    PathMetadata oldValue = listMap.put(childPath, newValue);
+    return oldValue == null || !oldValue.equals(newValue);
+  }
+
+  @Override
+  public String toString() {
+    return "DirListingMetadata{" +
+        "path=" + path +
+        ", listMap=" + listMap +
+        ", isAuthoritative=" + isAuthoritative +
+        '}';
+  }
+
+  /**
+   * Log contents to supplied StringBuilder in a pretty fashion.
+   * @param sb target StringBuilder
+   */
+  public void prettyPrint(StringBuilder sb) {
+    sb.append(String.format("DirMeta %-20s %-18s",
+        path.toString(),
+        isAuthoritative ? "Authoritative" : "Not Authoritative"));
+    for (Map.Entry<Path, PathMetadata> entry : listMap.entrySet()) {
+      sb.append("\n   key: ").append(entry.getKey()).append(": ");
+      entry.getValue().prettyPrint(sb);
+    }
+    sb.append("\n");
+  }
+
+  public String prettyPrint() {
+    StringBuilder sb = new StringBuilder();
+    prettyPrint(sb);
+    return sb.toString();
+  }
+
+  /**
+   * Checks that child path is valid.
+   * @param childPath path to check.
+   */
+  private void checkChildPath(Path childPath) {
+    checkPathAbsolute(childPath);
+
+    // If this dir's path has host (and thus scheme), so must its children
+    URI parentUri = path.toUri();
+    if (parentUri.getHost() != null) {
+      URI childUri = childPath.toUri();
+      Preconditions.checkNotNull(childUri.getHost(), "Expected non-null URI " +
+          "host: %s", childUri);
+      Preconditions.checkArgument(
+          childUri.getHost().equals(parentUri.getHost()),
+          "childUri %s and parentUri %s must have the same host",
+          childUri, parentUri);
+      Preconditions.checkNotNull(childUri.getScheme(), "No scheme in path %s",
+          childUri);
+    }
+    Preconditions.checkArgument(!childPath.isRoot(),
+        "childPath cannot be the root path: %s", childPath);
+    Preconditions.checkArgument(childPath.getParent().equals(path),
+        "childPath %s must be a child of %s", childPath, path);
+  }
+
+  /**
+   * For Paths that are handed in directly, we assert they are in consistent
+   * format with checkPath().  For paths that are supplied embedded in
+   * FileStatus, we attempt to fill in missing scheme and host, when this
+   * DirListingMetadata is associated with one.
+   *
+   * @return Path suitable for consistent hashtable lookups
+   * @throws NullPointerException null status argument
+   * @throws IllegalArgumentException bad status values or failure to
+   *                                  create a URI.
+   */
+  private Path childStatusToPathKey(FileStatus status) {
+    Path p = status.getPath();
+    Preconditions.checkNotNull(p, "Child status' path cannot be null");
+    Preconditions.checkArgument(!p.isRoot(),
+        "childPath cannot be the root path: %s", p);
+    Preconditions.checkArgument(p.getParent().equals(path),
+        "childPath %s must be a child of %s", p, path);
+    URI uri = p.toUri();
+    URI parentUri = path.toUri();
+    // If FileStatus' path is missing host, but should have one, add it.
+    if (uri.getHost() == null && parentUri.getHost() != null) {
+      try {
+        return new Path(new URI(parentUri.getScheme(), parentUri.getHost(),
+            uri.getPath(), uri.getFragment()));
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException("FileStatus path invalid with" +
+            " added " + parentUri.getScheme() + "://" + parentUri.getHost() +
+            " added", e);
+      }
+    }
+    return p;
+  }
+
+  private void checkPathAbsolute(Path p) {
+    Preconditions.checkNotNull(p, "path must be non-null");
+    Preconditions.checkArgument(p.isAbsolute(), "path must be absolute: %s", p);
+  }
+}


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


[20/48] hadoop git commit: HADOOP-14674. Correct javadoc for getRandomizedTempPath. Contributed by Mukul Kumar Singh.

Posted by in...@apache.org.
HADOOP-14674. Correct javadoc for getRandomizedTempPath. Contributed by Mukul Kumar Singh.


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

Branch: refs/heads/HDFS-10467
Commit: 063b6d0c93d700a57a7c6c29fdd1bcdecd0b9dc0
Parents: 99a7f5d
Author: Jitendra Pandey <ji...@apache.org>
Authored: Thu Aug 31 21:26:31 2017 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Fri Sep 1 10:17:12 2017 -0700

----------------------------------------------------------------------
 .../src/test/java/org/apache/hadoop/test/GenericTestUtils.java      | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/063b6d0c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index 9291bb0..4cb9f8b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -274,7 +274,6 @@ public abstract class GenericTestUtils {
    * Get a temp path. This may or may not be relative; it depends on what the
    * {@link #SYSPROP_TEST_DATA_DIR} is set to. If unset, it returns a path
    * under the relative path {@link #DEFAULT_TEST_DATA_PATH}
-   * @param subpath sub path, with no leading "/" character
    * @return a string to use in paths
    */
   public static String getRandomizedTempPath() {


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


[03/48] hadoop git commit: HADOOP-14824. Update ADLS SDK to 2.2.2 for MSI fix. Contributed by Atul Sikaria.

Posted by in...@apache.org.
HADOOP-14824. Update ADLS SDK to 2.2.2 for MSI fix. Contributed by Atul Sikaria.


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

Branch: refs/heads/HDFS-10467
Commit: 7ecc6dbed62c80397f71949bee41dcd03065755c
Parents: 27359b7
Author: John Zhuge <jz...@apache.org>
Authored: Thu Aug 31 21:04:12 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Thu Aug 31 21:13:22 2017 -0700

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure-datalake/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ecc6dbe/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index 47f12df..f699464 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -110,7 +110,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.2.1</version>
+      <version>2.2.2</version>
     </dependency>
     <!--  ENDS HERE-->
     <dependency>


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


[35/48] 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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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


[12/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/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 3b83f1f..3b9b5c4 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
@@ -107,6 +107,10 @@ 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
 
+*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
+at a time.
+
 The contents of each bucket will be destroyed during the test process:
 do not use the bucket for any purpose other than testing. Furthermore, for
 s3a, all in-progress multi-part uploads to the bucket will be aborted at the
@@ -691,7 +695,7 @@ use requires the presence of secret credentials, where tests may be slow,
 and where finding out why something failed from nothing but the test output
 is critical.
 
-#### Subclasses Existing Shared Base Blasses
+#### Subclasses Existing Shared Base Classes
 
 Extend `AbstractS3ATestBase` or `AbstractSTestS3AHugeFiles` unless justifiable.
 These set things up for testing against the object stores, provide good threadnames,
@@ -798,7 +802,7 @@ We really appreciate this &mdash; you will too.
 
 ### How to keep your credentials really safe
 
-Although the `auth-keys.xml` file is marged as ignored in git and subversion,
+Although the `auth-keys.xml` file is marked as ignored in git and subversion,
 it is still in your source tree, and there's always that risk that it may
 creep out.
 
@@ -813,3 +817,283 @@ using an absolute XInclude reference to it.
 
 </configuration>
 ```
+
+# Failure Injection
+
+**Warning do not enable any type of failure injection in production.  The
+following settings are for testing only.**
+
+One of the challenges with S3A integration tests is the fact that S3 is an
+eventually-consistent storage system.  In practice, we rarely see delays in
+visibility of recently created objects both in listings (`listStatus()`) and
+when getting a single file's metadata (`getFileStatus()`). Since this behavior
+is rare and non-deterministic, thorough integration testing is challenging.
+
+To address this, S3A supports a shim layer on top of the `AmazonS3Client`
+class which artificially delays certain paths from appearing in listings.
+This is implemented in the class `InconsistentAmazonS3Client`.
+
+## Simulating List Inconsistencies
+
+### Enabling the InconsistentAmazonS3CClient
+
+There are two ways of enabling the `InconsistentAmazonS3Client`: at
+config-time, or programmatically. For an example of programmatic test usage,
+see `ITestS3GuardListConsistency`.
+
+To enable the fault-injecting client via configuration, switch the
+S3A client to use the "Inconsistent S3 Client Factory" when connecting to
+S3:
+
+```xml
+<property>
+  <name>fs.s3a.s3.client.factory.impl</name>
+  <value>org.apache.hadoop.fs.s3a.InconsistentS3ClientFactory</value>
+</property>
+```
+
+The inconsistent client works by:
+
+1. Choosing which objects will be "inconsistent" at the time the object is
+created or deleted.
+2. When `listObjects()` is called, any keys that we have marked as
+inconsistent above will not be returned in the results (until the
+configured delay has elapsed). Similarly, deleted items may be *added* to
+missing results to delay the visibility of the delete.
+
+There are two ways of choosing which keys (filenames) will be affected: By
+substring, and by random probability.
+
+```xml
+<property>
+  <name>fs.s3a.failinject.inconsistency.key.substring</name>
+  <value>DELAY_LISTING_ME</value>
+</property>
+
+<property>
+  <name>fs.s3a.failinject.inconsistency.probability</name>
+  <value>1.0</value>
+</property>
+```
+
+By default, any object which has the substring "DELAY_LISTING_ME" in its key
+will subject to delayed visibility. For example, the path
+`s3a://my-bucket/test/DELAY_LISTING_ME/file.txt` would match this condition.
+To match all keys use the value "\*" (a single asterisk). This is a special
+value: *We don't support arbitrary wildcards.*
+
+The default probability of delaying an object is 1.0. This means that *all*
+keys that match the substring will get delayed visibility. Note that we take
+the logical *and* of the two conditions (substring matches *and* probability
+random chance occurs). Here are some example configurations:
+
+```
+| substring | probability |  behavior                                  |
+|-----------|-------------|--------------------------------------------|
+|           | 0.001       | An empty <value> tag in .xml config will   |
+|           |             | be interpreted as unset and revert to the  |
+|           |             | default value, "DELAY_LISTING_ME"          |
+|           |             |                                            |
+| *         | 0.001       | 1/1000 chance of *any* key being delayed.  |
+|           |             |                                            |
+| delay     | 0.01        | 1/100 chance of any key containing "delay" |
+|           |             |                                            |
+| delay     | 1.0         | All keys containing substring "delay" ..   |
+```
+
+You can also configure how long you want the delay in visibility to last.
+The default is 5000 milliseconds (five seconds).
+
+```xml
+<property>
+  <name>fs.s3a.failinject.inconsistency.msec</name>
+  <value>5000</value>
+</property>
+```
+
+Future versions of this client will introduce new failure modes,
+with simulation of S3 throttling exceptions the next feature under
+development.
+
+### Limitations of Inconsistency Injection
+
+Although `InconsistentAmazonS3Client` can delay the visibility of an object
+or parent directory, it does not prevent the key of that object from
+appearing in all prefix searches. For example, if we create the following
+object with the default configuration above, in an otherwise empty bucket:
+
+```
+s3a://bucket/a/b/c/DELAY_LISTING_ME
+```
+
+Then the following paths will still be visible as directories (ignoring
+possible real-world inconsistencies):
+
+```
+s3a://bucket/a
+s3a://bucket/a/b
+```
+
+Whereas `getFileStatus()` on the following *will* be subject to delayed
+visibility (`FileNotFoundException` until delay has elapsed):
+
+```
+s3a://bucket/a/b/c
+s3a://bucket/a/b/c/DELAY_LISTING_ME
+```
+
+In real-life S3 inconsistency, however, we expect that all the above paths
+(including `a` and `b`) will be subject to delayed visiblity.
+
+### Using the `InconsistentAmazonS3CClient` in downstream integration tests
+
+The inconsistent client is shipped in the `hadoop-aws` JAR, so it can
+be used in applications which work with S3 to see how they handle
+inconsistent directory listings.
+
+## Testing S3Guard
+
+The basic strategy for testing S3Guard correctness consists of:
+
+1. MetadataStore Contract tests.
+
+    The MetadataStore contract tests are inspired by the Hadoop FileSystem and
+    `FileContext` contract tests.  Each implementation of the `MetadataStore` interface
+    subclasses the `MetadataStoreTestBase` class and customizes it to initialize
+    their MetadataStore.  This test ensures that the different implementations
+    all satisfy the semantics of the MetadataStore API.
+
+2. Running existing S3A unit and integration tests with S3Guard enabled.
+
+    You can run the S3A integration tests on top of S3Guard by configuring your
+    `MetadataStore` in your
+    `hadoop-tools/hadoop-aws/src/test/resources/core-site.xml` or
+    `hadoop-tools/hadoop-aws/src/test/resources/auth-keys.xml` files.
+    Next run the S3A integration tests as outlined in the *Running the Tests* section
+    of the [S3A documentation](./index.html)
+
+3. Running fault-injection tests that test S3Guard's consistency features.
+
+    The `ITestS3GuardListConsistency` uses failure injection to ensure
+    that list consistency logic is correct even when the underlying storage is
+    eventually consistent.
+
+    The integration test adds a shim above the Amazon S3 Client layer that injects
+    delays in object visibility.
+
+    All of these tests will be run if you follow the steps listed in step 2 above.
+
+    No charges are incurred for using this store, and its consistency
+    guarantees are that of the underlying object store instance. <!-- :) -->
+
+## Testing S3A with S3Guard Enabled
+
+All the S3A tests which work with a private repository can be configured to
+run with S3Guard by using the `s3guard` profile. When set, this will run
+all the tests with local memory for the metadata set to "non-authoritative" mode.
+
+```bash
+mvn -T 1C verify -Dparallel-tests -DtestsThreadCount=6 -Ds3guard
+```
+
+When the `s3guard` profile is enabled, following profiles can be specified:
+
+* `dynamo`: use an AWS-hosted DynamoDB table; creating the table if it does
+  not exist. You will have to pay the bills for DynamoDB web service.
+* `dynamodblocal`: use an in-memory DynamoDBLocal server instead of real AWS
+  DynamoDB web service; launch the server and creating the table.
+  You won't be charged bills for using DynamoDB in test. As it runs in-JVM,
+  the table isn't shared across other tests running in parallel.
+* `non-auth`: treat the S3Guard metadata as authorative.
+
+```bash
+mvn -T 1C verify -Dparallel-tests -DtestsThreadCount=6 -Ds3guard -Ddynamo -Dauth
+```
+
+When experimenting with options, it is usually best to run a single test suite
+at a time until the operations appear to be working.
+
+```bash
+mvn -T 1C verify -Dtest=skip -Dit.test=ITestS3AMiscOperations -Ds3guard -Ddynamo
+```
+
+### Notes
+
+1. If the `s3guard` profile is not set, then the S3Guard properties are those
+of the test configuration set in `contract-test-options.xml` or `auth-keys.xml`
+
+If the `s3guard` profile *is* set,
+1. The S3Guard options from maven (the dynamo and authoritative flags)
+  overwrite any previously set in the configuration files.
+1. DynamoDB will be configured to create any missing tables.
+
+### Warning About Concurrent Tests
+
+You must not run S3A and S3N tests in parallel on the same bucket.  This is
+especially true when S3Guard is enabled.  S3Guard requires that all clients
+that are modifying the bucket have S3Guard enabled, so having S3N
+integration tests running in parallel with S3A tests will cause strange
+failures.
+
+### Scale Testing MetadataStore Directly
+
+There are some scale tests that exercise Metadata Store implementations
+directly. These ensure that S3Guard is are robust to things like DynamoDB
+throttling, and compare performance for different implementations. These
+are included in the scale tests executed when `-Dscale` is passed to
+the maven command line.
+
+The two S3Guard scale testse are `ITestDynamoDBMetadataStoreScale` and
+`ITestLocalMetadataStoreScale`.  To run the DynamoDB test, you will need to
+define your table name and region in your test configuration.  For example,
+the following settings allow us to run `ITestDynamoDBMetadataStoreScale` with
+artificially low read and write capacity provisioned, so we can judge the
+effects of being throttled by the DynamoDB service:
+
+```xml
+<property>
+  <name>scale.test.operation.count</name>
+  <value>10</value>
+</property>
+<property>
+  <name>scale.test.directory.count</name>
+  <value>3</value>
+</property>
+<property>
+  <name>fs.s3a.scale.test.enabled</name>
+  <value>true</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table</name>
+  <value>my-scale-test</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.region</name>
+  <value>us-west-2</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table.create</name>
+  <value>true</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.read</name>
+  <value>10</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.write</name>
+  <value>10</value>
+</property>
+```
+
+### Testing only: Local Metadata Store
+
+There is an in-memory Metadata Store for testing.
+
+```xml
+<property>
+  <name>fs.s3a.metadatastore.impl</name>
+  <value>org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore</value>
+</property>
+```
+
+This is not for use in production.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
index d2a858f..fd9497b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests creating files.
  */
 public class ITestS3AContractCreate extends AbstractContractCreateTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
index a47dcaef..95ea410 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests covering deletes.
  */
 public class ITestS3AContractDelete extends AbstractContractDeleteTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
index 50ce0c2..587dbbc 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.contract.s3a;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.SCALE_TEST_TIMEOUT_MILLIS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
@@ -38,12 +39,18 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
     return SCALE_TEST_TIMEOUT_MILLIS;
   }
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
   @Override
   protected Configuration createConfiguration() {
     Configuration newConf = super.createConfiguration();
     newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
     newConf.setBoolean(FAST_UPLOAD, true);
     newConf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_DISK);
+    // patch in S3Guard options
+    maybeEnableS3Guard(newConf);
     return newConf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
index c7ed5a3..cb9819c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests covering getFileStatus.
  */
@@ -46,6 +48,8 @@ public class ITestS3AContractGetFileStatus
     S3ATestUtils.disableFilesystemCaching(conf);
     // aggressively low page size forces tests to go multipage
     conf.setInt(Constants.MAX_PAGING_KEYS, 2);
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
     return conf;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
index d953e7e..dba52e1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * Test dir operations on S3A.
  */
 public class ITestS3AContractMkdir extends AbstractContractMkdirTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
index a7bdc0d..8e338b7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests opening files.
  */
 public class ITestS3AContractOpen extends AbstractContractOpenTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
index 5dba03d..4339649 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
@@ -26,12 +26,25 @@ import org.apache.hadoop.fs.Path;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 /**
  * S3A contract tests covering rename.
  */
 public class ITestS3AContractRename extends AbstractContractRenameTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
index 8383a77..5c2e2cd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * root dir operations against an S3 bucket.
  */
@@ -37,6 +39,18 @@ public class ITestS3AContractRootDir extends
   private static final Logger LOG =
       LoggerFactory.getLogger(ITestS3AContractRootDir.class);
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
index 1572fbc..379ace8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests covering file seek.
  */
 public class ITestS3AContractSeek extends AbstractContractSeekTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
index 6734947..0c7f7df 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
@@ -26,6 +26,8 @@ import com.amazonaws.services.s3.AmazonS3;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
 
 import org.junit.After;
 import org.junit.Before;
@@ -33,7 +35,8 @@ import org.junit.Rule;
 import org.junit.rules.ExpectedException;
 
 /**
- * Abstract base class for S3A unit tests using a mock S3 client.
+ * Abstract base class for S3A unit tests using a mock S3 client and a null
+ * metadata store.
  */
 public abstract class AbstractS3AMockTest {
 
@@ -55,6 +58,10 @@ public abstract class AbstractS3AMockTest {
     Configuration conf = new Configuration();
     conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class,
         S3ClientFactory.class);
+    // We explicitly disable MetadataStore even if it's configured. For unit
+    // test we don't issue request to AWS DynamoDB service.
+    conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+        MetadataStore.class);
     fs = new S3AFileSystem();
     URI uri = URI.create(FS_S3A + "://" + BUCKET);
     fs.initialize(uri, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
index c19b72c..f0c389d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -33,6 +33,7 @@ import java.io.IOException;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 /**
  * An extension of the contract test base set up for S3A tests.
@@ -65,6 +66,18 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
     return S3A_TEST_TIMEOUT;
   }
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   protected Configuration getConfiguration() {
     return getContract().getConf();
   }
@@ -99,10 +112,21 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
    */
   protected Path writeThenReadFile(String name, int len) throws IOException {
     Path path = path(name);
+    writeThenReadFile(path, len);
+    return path;
+  }
+
+  /**
+   * Write a file, read it back, validate the dataset. Overwrites the file
+   * if it is present
+   * @param path path to file
+   * @param len length of file
+   * @throws IOException any IO problem
+   */
+  protected void writeThenReadFile(Path path, int len) throws IOException {
     byte[] data = dataset(len, 'a', 'z');
     writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, true);
     ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
-    return path;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
index 22c4f7e..6601233 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
@@ -140,6 +140,10 @@ public class ITestS3AAWSCredentialsProvider {
       createFailingFS(conf);
     } catch (AccessDeniedException e) {
       // expected
+    } catch (AWSServiceIOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "UnrecognizedClientException", e);
+      // expected
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index dd75cb6..b9fe0fd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -25,6 +25,7 @@ import com.amazonaws.services.s3.S3ClientOptions;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.reflect.FieldUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
@@ -483,7 +484,7 @@ public class ITestS3AConfiguration {
       }
     });
     assertEquals("username", alice, fs.getUsername());
-    S3AFileStatus status = fs.getFileStatus(new Path("/"));
+    FileStatus status = fs.getFileStatus(new Path("/"));
     assertEquals("owner in " + status, alice, status.getOwner());
     assertEquals("group in " + status, alice, status.getGroup());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
index 71776ac..7dc286d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
@@ -29,6 +29,7 @@ import org.apache.commons.io.Charsets;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -63,7 +64,7 @@ public class ITestS3ACopyFromLocalFile extends AbstractS3ATestBase {
     Path dest = upload(file, true);
     assertPathExists("uploaded file not found", dest);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(dest);
+    FileStatus status = fs.getFileStatus(dest);
     assertEquals("File length of " + status,
         message.getBytes(ASCII).length, status.getLen());
     assertFileTextEquals(dest, message);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
index b3d7abf..95d44cc 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
@@ -37,6 +38,7 @@ import java.net.URLEncoder;
 import java.nio.file.AccessDeniedException;
 
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3GuardState;
 
 /**
  * Tests that credentials can go into the URL. This includes a valid
@@ -63,6 +65,11 @@ public class ITestS3ACredentialsInURL extends Assert {
   public void testInstantiateFromURL() throws Throwable {
 
     Configuration conf = new Configuration();
+
+    // Skip in the case of S3Guard with DynamoDB because it cannot get
+    // credentials for its own use if they're only in S3 URLs
+    assumeS3GuardState(false, conf);
+
     String accessKey = conf.get(Constants.ACCESS_KEY);
     String secretKey = conf.get(Constants.SECRET_KEY);
     String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
@@ -84,6 +91,7 @@ public class ITestS3ACredentialsInURL extends Assert {
     conf.unset(Constants.ACCESS_KEY);
     conf.unset(Constants.SECRET_KEY);
     fs = S3ATestUtils.createTestFileSystem(conf);
+
     String fsURI = fs.getUri().toString();
     assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
     assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
@@ -119,13 +127,14 @@ public class ITestS3ACredentialsInURL extends Assert {
     Configuration conf = new Configuration();
     String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
     Assume.assumeNotNull(fsname);
+    assumeS3GuardState(false, conf);
     URI original = new URI(fsname);
     URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
 
     conf.set(TEST_FS_S3A_NAME, testURI.toString());
-    fs = S3ATestUtils.createTestFileSystem(conf);
     try {
-      S3AFileStatus status = fs.getFileStatus(new Path("/"));
+      fs = S3ATestUtils.createTestFileSystem(conf);
+      FileStatus status = fs.getFileStatus(new Path("/"));
       fail("Expected an AccessDeniedException, got " + status);
     } catch (AccessDeniedException e) {
       // expected

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java
new file mode 100644
index 0000000..7abd474
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.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.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.concurrent.Callable;
+
+/**
+ * Tests behavior of a FileNotFound error that happens after open(), i.e. on
+ * the first read.
+ */
+public class ITestS3ADelayedFNF extends AbstractS3ATestBase {
+
+
+  /**
+   * See debugging documentation
+   * <a href="https://cwiki.apache.org/confluence/display/HADOOP/S3A%3A+FileNotFound+Exception+on+Read">here</a>.
+   * @throws Exception
+   */
+  @Test
+  public void testNotFoundFirstRead() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path p = path("some-file");
+    ContractTestUtils.createFile(fs, p, false, new byte[] {20, 21, 22});
+
+    final FSDataInputStream in = fs.open(p);
+    assertDeleted(p, false);
+
+    // This should fail since we deleted after the open.
+    LambdaTestUtils.intercept(FileNotFoundException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return in.read();
+          }
+        });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java
new file mode 100644
index 0000000..c55be5b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Tests which exercise treatment of empty/non-empty directories.
+ */
+public class ITestS3AEmptyDirectory extends AbstractS3ATestBase {
+
+  @Test
+  public void testDirectoryBecomesEmpty() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+
+    // 1. set up non-empty dir
+    Path dir = path("testEmptyDir");
+    Path child = path("testEmptyDir/dir2");
+    mkdirs(child);
+
+    S3AFileStatus status = getS3AFileStatus(fs, dir);
+    assertEmptyDirectory(false, status);
+
+    // 2. Make testEmptyDir empty
+    assertDeleted(child, false);
+    status = getS3AFileStatus(fs, dir);
+
+    assertEmptyDirectory(true, status);
+  }
+
+  private static void assertEmptyDirectory(boolean isEmpty, S3AFileStatus s) {
+    String msg = "dir is empty";
+    // Should *not* be Tristate.UNKNOWN since we request a definitive value
+    // in getS3AFileStatus() below
+    Tristate expected = Tristate.fromBool(isEmpty);
+    assertEquals(msg, expected, s.isEmptyDirectory());
+  }
+
+  @Test
+  public void testDirectoryBecomesNonEmpty() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+
+    // 1. create empty dir
+    Path dir = path("testEmptyDir");
+    mkdirs(dir);
+
+    S3AFileStatus status = getS3AFileStatus(fs, dir);
+    assertEmptyDirectory(true, status);
+
+    // 2. Make testEmptyDir non-empty
+
+    ContractTestUtils.touch(fs, path("testEmptyDir/file1"));
+    status = getS3AFileStatus(fs, dir);
+
+    assertEmptyDirectory(false, status);
+  }
+
+  private S3AFileStatus getS3AFileStatus(S3AFileSystem fs, Path p) throws
+      IOException {
+    return fs.innerGetFileStatus(p, true /* want isEmptyDirectory value */);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
index 91be8b9..8b7e031 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
@@ -18,19 +18,21 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionTestsDisabled;
-import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-
 import java.io.IOException;
+import java.nio.file.AccessDeniedException;
+
+import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
-import org.junit.Test;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Concrete class that extends {@link AbstractTestS3AEncryption}
@@ -38,17 +40,39 @@ import org.junit.Test;
  */
 public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
 
+  private static final String SERVICE_AMAZON_S3_STATUS_CODE_403
+      = "Service: Amazon S3; Status Code: 403;";
+  private static final String KEY_1
+      = "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=";
+  private static final String KEY_2
+      = "G61nz31Q7+zpjJWbakxfTOZW4VS0UmQWAq2YXhcTXoo=";
+  private static final String KEY_3
+      = "NTx0dUPrxoo9+LbNiT/gqf3z9jILqL6ilismFmJO50U=";
+  private static final String KEY_4
+      = "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=";
+  private static final int TEST_FILE_LEN = 2048;
+
+  /**
+   * Filesystem created with a different key.
+   */
+  private FileSystem fsKeyB;
+
   @Override
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
-    S3ATestUtils.disableFilesystemCaching(conf);
+    disableFilesystemCaching(conf);
     conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
         getSSEAlgorithm().getMethod());
-    conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY,
-        "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=");
+    conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, KEY_1);
     return conf;
   }
 
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.closeStream(fsKeyB);
+  }
+
   /**
    * This will create and write to a file using encryption key A, then attempt
    * to read from it again with encryption key B.  This will not work as it
@@ -64,26 +88,25 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
 
-    final Path[] path = new Path[1];
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;", () -> {
-
-        int len = 2048;
-        describe("Create an encrypted file of size " + len);
-        String src = createFilename(len);
-        path[0] = writeThenReadFile(src, len);
-
-        //extract the test FS
-        FileSystem fileSystem = createNewFileSystemWithSSECKey(
-            "kX7SdwVc/1VXJr76kfKnkQ3ONYhxianyL2+C3rPVT9s=");
-        byte[] data = dataset(len, 'a', 'z');
-        ContractTestUtils.verifyFileContents(fileSystem, path[0], data);
-        throw new Exception("Fail");
-      });
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          int len = TEST_FILE_LEN;
+          describe("Create an encrypted file of size " + len);
+          Path src = path("testCreateFileAndReadWithDifferentEncryptionKey");
+          writeThenReadFile(src, len);
+
+          //extract the test FS
+          fsKeyB = createNewFileSystemWithSSECKey(
+              "kX7SdwVc/1VXJr76kfKnkQ3ONYhxianyL2+C3rPVT9s=");
+          byte[] data = dataset(len, 'a', 'z');
+          ContractTestUtils.verifyFileContents(fsKeyB, src, data);
+          return fsKeyB.getFileStatus(src);
+        });
   }
 
   /**
-   * While each object has it's own key and should be distinct, this verifies
+   * While each object has its own key and should be distinct, this verifies
    * that hadoop treats object keys as a filesystem path.  So if a top level
    * dir is encrypted with keyA, a sublevel dir cannot be accessed with a
    * different keyB.
@@ -96,25 +119,20 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testCreateSubdirWithDifferentKey() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
-
-    final Path[] path = new Path[1];
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;", () -> {
-
-        path[0] = S3ATestUtils.createTestPath(
-          new Path(createFilename("dir/"))
-        );
-        Path nestedDirectory = S3ATestUtils.createTestPath(
-            new Path(createFilename("dir/nestedDir/"))
-        );
-        FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-            "G61nz31Q7+zpjJWbakxfTOZW4VS0UmQWAq2YXhcTXoo=");
-        getFileSystem().mkdirs(path[0]);
-        fsKeyB.mkdirs(nestedDirectory);
-
-        throw new Exception("Exception should be thrown.");
-      });
-    rm(getFileSystem(), path[0], true, false);
+    assumeS3GuardState(false, getConfiguration());
+
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          Path base = path("testCreateSubdirWithDifferentKey");
+          Path nestedDirectory = new Path(base, "nestedDir");
+          fsKeyB = createNewFileSystemWithSSECKey(
+              KEY_2);
+          getFileSystem().mkdirs(base);
+          fsKeyB.mkdirs(nestedDirectory);
+          // expected to fail
+          return fsKeyB.getFileStatus(nestedDirectory);
+        });
   }
 
   /**
@@ -130,20 +148,17 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
 
-    final Path[] path = new Path[1];
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;", () -> {
-
-        int len = 2048;
-        String src = createFilename(len);
-        path[0] = writeThenReadFile(src, len);
-
-        FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-            "NTx0dUPrxoo9+LbNiT/gqf3z9jILqL6ilismFmJO50U=");
-        fsKeyB.rename(path[0], new Path(createFilename("different-path.txt")));
-
-        throw new Exception("Exception should be thrown.");
-      });
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          int len = TEST_FILE_LEN;
+          Path src = path(createFilename(len));
+          writeThenReadFile(src, len);
+          fsKeyB = createNewFileSystemWithSSECKey(KEY_3);
+          Path dest = path(createFilename("different-path.txt"));
+          getFileSystem().mkdirs(dest.getParent());
+          return fsKeyB.rename(src, dest);
+        });
   }
 
   /**
@@ -157,11 +172,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
 
-    String src = createFilename("original-path.txt");
-    Path path = writeThenReadFile(src, 2048);
-    Path newPath = path(createFilename("different-path.txt"));
-    getFileSystem().rename(path, newPath);
-    byte[] data = dataset(2048, 'a', 'z');
+    Path src = path("original-path.txt");
+    writeThenReadFile(src, TEST_FILE_LEN);
+    Path newPath = path("different-path.txt");
+    getFileSystem().rename(src, newPath);
+    byte[] data = dataset(TEST_FILE_LEN, 'a', 'z');
     ContractTestUtils.verifyFileContents(getFileSystem(), newPath, data);
   }
 
@@ -175,30 +190,26 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testListEncryptedDir() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
+
+    Path pathABC = path("testListEncryptedDir/a/b/c/");
+    Path pathAB = pathABC.getParent();
+    Path pathA = pathAB.getParent();
 
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-         path(createFilename("/a/b/c/"))
-    );
+    Path nestedDirectory = createTestPath(pathABC);
     assertTrue(getFileSystem().mkdirs(nestedDirectory));
 
-    FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
-    fsKeyB.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))
-    ), true);
-    fsKeyB.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))
-    ), true);
+    fsKeyB.listFiles(pathA, true);
+    fsKeyB.listFiles(pathAB, true);
 
     //Until this point, no exception is thrown about access
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;", () -> {
-        fsKeyB.listFiles(S3ATestUtils.createTestPath(
-            path(createFilename("/a/b/c/"))
-        ), false);
-        throw new Exception("Exception should be thrown.");
-      });
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          fsKeyB.listFiles(pathABC, false);
+        });
 
     Configuration conf = this.createConfiguration();
     conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM);
@@ -209,22 +220,13 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     FileSystem unencryptedFileSystem = contract.getTestFileSystem();
 
     //unencrypted can access until the final directory
-    unencryptedFileSystem.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))
-    ), true);
-    unencryptedFileSystem.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))
-    ), true);
-    intercept(org.apache.hadoop.fs.s3a.AWSS3IOException.class,
-        "Bad Request (Service: Amazon S3; Status Code: 400; Error" +
-          " Code: 400 Bad Request;", () -> {
-
-        unencryptedFileSystem.listFiles(S3ATestUtils.createTestPath(
-            path(createFilename("/a/b/c/"))
-        ), false);
-        throw new Exception("Exception should be thrown.");
-      });
-    rm(getFileSystem(), path(createFilename("/")), true, false);
+    unencryptedFileSystem.listFiles(pathA, true);
+    unencryptedFileSystem.listFiles(pathAB, true);
+    AWSS3IOException ex = intercept(AWSS3IOException.class,
+        () -> {
+          unencryptedFileSystem.listFiles(pathABC, false);
+        });
+    assertStatusCode(ex, 400);
   }
 
   /**
@@ -236,31 +238,27 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testListStatusEncryptedDir() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
 
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-         path(createFilename("/a/b/c/"))
-    );
-    assertTrue(getFileSystem().mkdirs(nestedDirectory));
+    Path pathABC = path("testListStatusEncryptedDir/a/b/c/");
+    Path pathAB = pathABC.getParent();
+    Path pathA = pathAB.getParent();
+    assertTrue(getFileSystem().mkdirs(pathABC));
 
-    FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
-    fsKeyB.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))));
-    fsKeyB.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))));
+    fsKeyB.listStatus(pathA);
+    fsKeyB.listStatus(pathAB);
 
     //Until this point, no exception is thrown about access
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;", () -> {
-        fsKeyB.listStatus(S3ATestUtils.createTestPath(
-            path(createFilename("/a/b/c/"))));
-
-        throw new Exception("Exception should be thrown.");
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          fsKeyB.listStatus(pathABC);
         });
 
     //Now try it with an unencrypted filesystem.
-    Configuration conf = this.createConfiguration();
+    Configuration conf = createConfiguration();
     conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM);
     conf.unset(Constants.SERVER_SIDE_ENCRYPTION_KEY);
 
@@ -269,20 +267,14 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     FileSystem unencryptedFileSystem = contract.getTestFileSystem();
 
     //unencrypted can access until the final directory
-    unencryptedFileSystem.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))));
-    unencryptedFileSystem.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))));
-
-    intercept(org.apache.hadoop.fs.s3a.AWSS3IOException.class,
-        "Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400" +
-        " Bad Request;", () -> {
-
-        unencryptedFileSystem.listStatus(S3ATestUtils.createTestPath(
-            path(createFilename("/a/b/c/"))));
-        throw new Exception("Exception should be thrown.");
+    unencryptedFileSystem.listStatus(pathA);
+    unencryptedFileSystem.listStatus(pathAB);
+
+    AWSS3IOException ex = intercept(AWSS3IOException.class,
+        () -> {
+          unencryptedFileSystem.listStatus(pathABC);
         });
-    rm(getFileSystem(), path(createFilename("/")), true, false);
+    assertStatusCode(ex, 400);
   }
 
   /**
@@ -294,31 +286,24 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testListStatusEncryptedFile() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
+    Path pathABC = path("testListStatusEncryptedFile/a/b/c/");
+    assertTrue(getFileSystem().mkdirs(pathABC));
 
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/c/"))
-    );
-    assertTrue(getFileSystem().mkdirs(nestedDirectory));
-
-    String src = createFilename("/a/b/c/fileToStat.txt");
-    Path fileToStat =  writeThenReadFile(src, 2048);
+    Path fileToStat = new Path(pathABC, "fileToStat.txt");
+    writeThenReadFile(fileToStat, TEST_FILE_LEN);
 
-    FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
     //Until this point, no exception is thrown about access
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;", () -> {
-        fsKeyB.listStatus(S3ATestUtils.createTestPath(fileToStat));
-
-        throw new Exception("Exception should be thrown.");
-      });
-    rm(getFileSystem(), path(createFilename("/")), true, false);
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          fsKeyB.listStatus(fileToStat);
+        });
   }
 
 
-
-
   /**
    * It is possible to delete directories without the proper encryption key and
    * the hierarchy above it.
@@ -329,31 +314,26 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testDeleteEncryptedObjectWithDifferentKey() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
-
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/c/"))
-    );
-    assertTrue(getFileSystem().mkdirs(nestedDirectory));
-    String src = createFilename("/a/b/c/filetobedeleted.txt");
-    Path fileToDelete =  writeThenReadFile(src, 2048);
-
-    FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Forbidden (Service: Amazon S3; Status Code: 403; Error Code: " +
-        "403 Forbidden", () -> {
-
-        fsKeyB.delete(fileToDelete, false);
-        throw new Exception("Exception should be thrown.");
-      });
+    assumeS3GuardState(false, getConfiguration());
+    Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/");
+
+    Path pathAB = pathABC.getParent();
+    Path pathA = pathAB.getParent();
+    assertTrue(getFileSystem().mkdirs(pathABC));
+    Path fileToDelete = new Path(pathABC, "filetobedeleted.txt");
+    writeThenReadFile(fileToDelete, TEST_FILE_LEN);
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> {
+          fsKeyB.delete(fileToDelete, false);
+        });
 
     //This is possible
-    fsKeyB.delete(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/c/"))), true);
-    fsKeyB.delete(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))), true);
-    fsKeyB.delete(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))), true);
+    fsKeyB.delete(pathABC, true);
+    fsKeyB.delete(pathAB, true);
+    fsKeyB.delete(pathA, true);
+    assertPathDoesNotExist("expected recursive delete", fileToDelete);
   }
 
   private FileSystem createNewFileSystemWithSSECKey(String sseCKey) throws
@@ -371,4 +351,5 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   protected S3AEncryptionMethods getSSEAlgorithm() {
     return S3AEncryptionMethods.SSE_C;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
index 00171f0..3e293f7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
@@ -32,8 +33,8 @@ import java.net.URI;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
 import static org.apache.hadoop.test.GenericTestUtils.getTestDir;
+import static org.junit.Assume.assumeFalse;
 
 /**
  * Use metrics to assert about the cost of file status queries.
@@ -62,9 +63,11 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     S3AFileSystem fs = getFileSystem();
     touch(fs, simpleFile);
     resetMetricDiffs();
-    S3AFileStatus status = fs.getFileStatus(simpleFile);
+    FileStatus status = fs.getFileStatus(simpleFile);
     assertTrue("not a file: " + status, status.isFile());
-    metadataRequests.assertDiffEquals(1);
+    if (!fs.hasMetadataStore()) {
+      metadataRequests.assertDiffEquals(1);
+    }
     listRequests.assertDiffEquals(0);
   }
 
@@ -79,9 +82,13 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path dir = path("empty");
     fs.mkdirs(dir);
     resetMetricDiffs();
-    S3AFileStatus status = fs.getFileStatus(dir);
-    assertTrue("not empty: " + status, status.isEmptyDirectory());
-    metadataRequests.assertDiffEquals(2);
+    S3AFileStatus status = fs.innerGetFileStatus(dir, true);
+    assertTrue("not empty: " + status,
+        status.isEmptyDirectory() == Tristate.TRUE);
+
+    if (!fs.hasMetadataStore()) {
+      metadataRequests.assertDiffEquals(2);
+    }
     listRequests.assertDiffEquals(0);
   }
 
@@ -92,7 +99,7 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path path = path("missing");
     resetMetricDiffs();
     try {
-      S3AFileStatus status = fs.getFileStatus(path);
+      FileStatus status = fs.getFileStatus(path);
       fail("Got a status back from a missing file path " + status);
     } catch (FileNotFoundException expected) {
       // expected
@@ -108,7 +115,7 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path path = path("missingdir/missingpath");
     resetMetricDiffs();
     try {
-      S3AFileStatus status = fs.getFileStatus(path);
+      FileStatus status = fs.getFileStatus(path);
       fail("Got a status back from a missing file path " + status);
     } catch (FileNotFoundException expected) {
       // expected
@@ -126,16 +133,18 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path simpleFile = new Path(dir, "simple.txt");
     touch(fs, simpleFile);
     resetMetricDiffs();
-    S3AFileStatus status = fs.getFileStatus(dir);
-    if (status.isEmptyDirectory()) {
+    S3AFileStatus status = fs.innerGetFileStatus(dir, true);
+    if (status.isEmptyDirectory() == Tristate.TRUE) {
       // erroneous state
       String fsState = fs.toString();
       fail("FileStatus says directory isempty: " + status
           + "\n" + ContractTestUtils.ls(fs, dir)
           + "\n" + fsState);
     }
-    metadataRequests.assertDiffEquals(2);
-    listRequests.assertDiffEquals(1);
+    if (!fs.hasMetadataStore()) {
+      metadataRequests.assertDiffEquals(2);
+      listRequests.assertDiffEquals(1);
+    }
   }
 
   @Test
@@ -187,6 +196,13 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
         + "In S3, rename deletes any fake directories as a part of "
         + "clean up activity");
     S3AFileSystem fs = getFileSystem();
+
+    // As this test uses the s3 metrics to count the number of fake directory
+    // operations, it depends on side effects happening internally. With
+    // metadata store enabled, it is brittle to change. We disable this test
+    // before the internal behavior w/ or w/o metadata store.
+    assumeFalse(fs.hasMetadataStore());
+
     Path srcBaseDir = path("src");
     mkdirs(srcBaseDir);
     MetricDiff deleteRequests =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
index 1b49d07..27af23a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
@@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
+
 import static org.junit.Assume.*;
 import static org.junit.Assert.*;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java
new file mode 100644
index 0000000..eb4f70b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.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.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
+
+/**
+ * Tests S3A behavior under forced inconsistency via {@link
+ * InconsistentAmazonS3Client}.
+ *
+ * These tests are for validating expected behavior *without* S3Guard, but
+ * may also run with S3Guard enabled.  For tests that validate S3Guard's
+ * consistency features, see {@link ITestS3GuardListConsistency}.
+ */
+public class ITestS3AInconsistency extends AbstractS3ATestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    conf.setClass(S3_CLIENT_FACTORY_IMPL, InconsistentS3ClientFactory.class,
+        S3ClientFactory.class);
+    conf.set(FAIL_INJECT_INCONSISTENCY_KEY, DEFAULT_DELAY_KEY_SUBSTRING);
+    conf.setFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY, 1.0f);
+    conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, DEFAULT_DELAY_KEY_MSEC);
+    return new S3AContract(conf);
+  }
+
+  @Test
+  public void testGetFileStatus() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+
+    // 1. Make sure no ancestor dirs exist
+    Path dir = path("ancestor");
+    fs.delete(dir, true);
+    waitUntilDeleted(dir);
+
+    // 2. Create a descendant file, which implicitly creates ancestors
+    // This file has delayed visibility.
+    touch(getFileSystem(),
+        path("ancestor/file-" + DEFAULT_DELAY_KEY_SUBSTRING));
+
+    // 3. Assert expected behavior.  If S3Guard is enabled, we should be able
+    // to get status for ancestor.  If S3Guard is *not* enabled, S3A will
+    // fail to infer the existence of the ancestor since visibility of the
+    // child file is delayed, and its key prefix search will return nothing.
+    try {
+      FileStatus status = fs.getFileStatus(dir);
+      if (fs.hasMetadataStore()) {
+        assertTrue("Ancestor is dir", status.isDirectory());
+      } else {
+        fail("getFileStatus should fail due to delayed visibility.");
+      }
+    } catch (FileNotFoundException e) {
+      if (fs.hasMetadataStore()) {
+        fail("S3Guard failed to list parent of inconsistent child.");
+      }
+      LOG.info("File not found, as expected.");
+    }
+  }
+
+  private void waitUntilDeleted(final Path p) throws Exception {
+    LambdaTestUtils.eventually(30 * 1000, 1000,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            assertPathDoesNotExist("Dir should be deleted", p);
+            return null;
+          }
+        }
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
index 59fcb05..869d64c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
@@ -22,10 +22,17 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
 import org.junit.Test;
 
+import java.io.ByteArrayInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.concurrent.Callable;
 
 /**
  * Tests of the S3A FileSystem which don't have a specific home and can share
@@ -55,6 +62,26 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
     createNonRecursive(new Path(parent, "fail"));
   }
 
+  @Test
+  public void testPutObjectDirect() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    ObjectMetadata metadata = fs.newObjectMetadata(-1);
+    metadata.setContentLength(-1);
+    Path path = path("putDirect");
+    final PutObjectRequest put = new PutObjectRequest(fs.getBucket(),
+        path.toUri().getPath(),
+        new ByteArrayInputStream("PUT".getBytes()),
+        metadata);
+    LambdaTestUtils.intercept(IllegalStateException.class,
+        new Callable<PutObjectResult>() {
+          @Override
+          public PutObjectResult call() throws Exception {
+            return fs.putObjectDirect(put);
+          }
+        });
+    assertPathDoesNotExist("put object was created", path);
+  }
+
   private FSDataOutputStream createNonRecursive(Path path) throws IOException {
     return getFileSystem().createNonRecursive(path, false, 4096,
         (short) 3, (short) 4096,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java
new file mode 100644
index 0000000..dcc2538
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java
@@ -0,0 +1,61 @@
+/*
+ * 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.s3a;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.junit.Assume;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+/**
+ * Home for testing the creation of new files and directories with S3Guard
+ * enabled.
+ */
+public class ITestS3GuardCreate extends AbstractS3ATestBase {
+
+  /**
+   * Test that ancestor creation during S3AFileSystem#create() is properly
+   * accounted for in the MetadataStore.  This should be handled by the
+   * FileSystem, and be a FS contract test, but S3A does not handle ancestors on
+   * create(), so we need to take care in the S3Guard code to do the right
+   * thing.  This may change: See HADOOP-13221 for more detail.
+   */
+  @Test
+  public void testCreatePopulatesFileAncestors() throws Exception {
+    final S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+    final MetadataStore ms = fs.getMetadataStore();
+    final Path parent = path("testCreatePopulatesFileAncestors");
+
+    try {
+      fs.mkdirs(parent);
+      final Path nestedFile = new Path(parent, "dir1/dir2/file4");
+      touch(fs, nestedFile);
+
+      DirListingMetadata list = ms.listChildren(parent);
+      assertFalse("MetadataStore falsely reports authoritative empty list",
+          list.isEmpty() == Tristate.TRUE);
+    } finally {
+      fs.delete(parent, true);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java
new file mode 100644
index 0000000..fb6e370
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
+import org.junit.Assume;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+/**
+ * Test logic around whether or not a directory is empty, with S3Guard enabled.
+ * The fact that S3AFileStatus has an isEmptyDirectory flag in it makes caching
+ * S3AFileStatus's really tricky, as the flag can change as a side effect of
+ * changes to other paths.
+ * After S3Guard is merged to trunk, we should try to remove the
+ * isEmptyDirectory flag from S3AFileStatus, or maintain it outside
+ * of the MetadataStore.
+ */
+public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase {
+
+  @Test
+  public void testEmptyDirs() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+    MetadataStore configuredMs = fs.getMetadataStore();
+    Path existingDir = path("existing-dir");
+    Path existingFile = path("existing-dir/existing-file");
+    try {
+      // 1. Simulate files already existing in the bucket before we started our
+      // cluster.  Temporarily disable the MetadataStore so it doesn't witness
+      // us creating these files.
+
+      fs.setMetadataStore(new NullMetadataStore());
+      assertTrue(fs.mkdirs(existingDir));
+      touch(fs, existingFile);
+
+
+      // 2. Simulate (from MetadataStore's perspective) starting our cluster and
+      // creating a file in an existing directory.
+      fs.setMetadataStore(configuredMs);  // "start cluster"
+      Path newFile = path("existing-dir/new-file");
+      touch(fs, newFile);
+
+      S3AFileStatus status = fs.innerGetFileStatus(existingDir, true);
+      assertEquals("Should not be empty dir", Tristate.FALSE,
+          status.isEmptyDirectory());
+
+      // 3. Assert that removing the only file the MetadataStore witnessed
+      // being created doesn't cause it to think the directory is now empty.
+      fs.delete(newFile, false);
+      status = fs.innerGetFileStatus(existingDir, true);
+      assertEquals("Should not be empty dir", Tristate.FALSE,
+          status.isEmptyDirectory());
+
+      // 4. Assert that removing the final file, that existed "before"
+      // MetadataStore started, *does* cause the directory to be marked empty.
+      fs.delete(existingFile, false);
+      status = fs.innerGetFileStatus(existingDir, true);
+      assertEquals("Should be empty dir now", Tristate.TRUE,
+          status.isEmptyDirectory());
+    } finally {
+      configuredMs.forgetMetadata(existingFile);
+      configuredMs.forgetMetadata(existingDir);
+    }
+  }
+}


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


[08/48] hadoop git commit: HDFS-11964. Decoding inputs should be correctly prepared in pread. Contributed by Takanobu Asanuma

Posted by in...@apache.org.
HDFS-11964. Decoding inputs should be correctly prepared in pread. Contributed by Takanobu Asanuma


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

Branch: refs/heads/HDFS-10467
Commit: 7a96033b15580a01a2867fa3cab9c1e409dbaafd
Parents: 1b3b993
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Sep 1 17:48:26 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Sep 1 17:48:26 2017 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/PositionStripeReader.java | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a96033b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PositionStripeReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PositionStripeReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PositionStripeReader.java
index 1e9d638..b01b74c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PositionStripeReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PositionStripeReader.java
@@ -57,6 +57,10 @@ class PositionStripeReader extends StripeReader {
     Preconditions.checkState(index >= dataBlkNum &&
         alignedStripe.chunks[index] == null);
 
+    int bufLen = (int) alignedStripe.getSpanInBlock();
+    decodeInputs[index] = new ECChunk(codingBuffer.duplicate(), index * bufLen,
+        bufLen);
+
     alignedStripe.chunks[index] =
         new StripingChunk(decodeInputs[index].getBuffer());
 
@@ -75,7 +79,7 @@ class PositionStripeReader extends StripeReader {
     codingBuffer = dfsStripedInputStream.getBufferPool().
         getBuffer(useDirectBuffer(), bufLen * bufCount);
     ByteBuffer buffer;
-    for (int i = 0; i < decodeInputs.length; i++) {
+    for (int i = 0; i < dataBlkNum; i++) {
       buffer = codingBuffer.duplicate();
       decodeInputs[i] = new ECChunk(buffer, i * bufLen, bufLen);
     }


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


[09/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
new file mode 100644
index 0000000..876cc80
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.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.fs.s3a.scale;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
+
+/**
+ * Test the performance of a MetadataStore.  Useful for load testing.
+ * Could be separated from S3A code, but we're using the S3A scale test
+ * framework for convenience.
+ */
+public abstract class AbstractITestS3AMetadataStoreScale extends
+    S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractITestS3AMetadataStoreScale.class);
+
+  /** Some dummy values for FileStatus contents. */
+  static final long BLOCK_SIZE = 32 * 1024 * 1024;
+  static final long SIZE = BLOCK_SIZE * 2;
+  static final String OWNER = "bob";
+  static final long ACCESS_TIME = System.currentTimeMillis();
+
+  static final Path BUCKET_ROOT = new Path("s3a://fake-bucket/");
+
+  /**
+   * Subclasses should override this to provide the MetadataStore they which
+   * to test.
+   * @return MetadataStore to test against
+   * @throws IOException
+   */
+  public abstract MetadataStore createMetadataStore() throws IOException;
+
+  @Test
+  public void testPut() throws Throwable {
+    describe("Test workload of put() operations");
+
+    // As described in hadoop-aws site docs, count parameter is used for
+    // width and depth of directory tree
+    int width = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
+    int depth = width;
+
+    List<PathMetadata> paths = new ArrayList<>();
+    createDirTree(BUCKET_ROOT, depth, width, paths);
+
+    long count = 1; // Some value in case we throw an exception below
+    try (MetadataStore ms = createMetadataStore()) {
+
+      try {
+        count = populateMetadataStore(paths, ms);
+      } finally {
+        clearMetadataStore(ms, count);
+      }
+    }
+  }
+
+  @Test
+  public void testMoves() throws Throwable {
+    describe("Test workload of batched move() operations");
+
+    // As described in hadoop-aws site docs, count parameter is used for
+    // width and depth of directory tree
+    int width = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
+    int depth = width;
+
+    long operations = getConf().getLong(KEY_OPERATION_COUNT,
+        DEFAULT_OPERATION_COUNT);
+
+    List<PathMetadata> origMetas = new ArrayList<>();
+    createDirTree(BUCKET_ROOT, depth, width, origMetas);
+
+    // Pre-compute source and destination paths for move() loop below
+    List<Path> origPaths = metasToPaths(origMetas);
+    List<PathMetadata> movedMetas = moveMetas(origMetas, BUCKET_ROOT,
+        new Path(BUCKET_ROOT, "moved-here"));
+    List<Path> movedPaths = metasToPaths(movedMetas);
+
+    long count = 1; // Some value in case we throw an exception below
+    try (MetadataStore ms = createMetadataStore()) {
+
+      try {
+        // Setup
+        count = populateMetadataStore(origMetas, ms);
+
+        // Main loop: move things back and forth
+        describe("Running move workload");
+        NanoTimer moveTimer = new NanoTimer();
+        LOG.info("Running {} moves of {} paths each", operations,
+            origMetas.size());
+        for (int i = 0; i < operations; i++) {
+          Collection<Path> toDelete;
+          Collection<PathMetadata> toCreate;
+          if (i % 2 == 0) {
+            toDelete = origPaths;
+            toCreate = movedMetas;
+          } else {
+            toDelete = movedPaths;
+            toCreate = origMetas;
+          }
+          ms.move(toDelete, toCreate);
+        }
+        moveTimer.end();
+        printTiming(LOG, "move", moveTimer, operations);
+      } finally {
+        // Cleanup
+        clearMetadataStore(ms, count);
+      }
+    }
+  }
+
+  /**
+   * Create a copy of given list of PathMetadatas with the paths moved from
+   * src to dest.
+   */
+  private List<PathMetadata> moveMetas(List<PathMetadata> metas, Path src,
+      Path dest) throws IOException {
+    List<PathMetadata> moved = new ArrayList<>(metas.size());
+    for (PathMetadata srcMeta : metas) {
+      S3AFileStatus status = copyStatus((S3AFileStatus)srcMeta.getFileStatus());
+      status.setPath(movePath(status.getPath(), src, dest));
+      moved.add(new PathMetadata(status));
+    }
+    return moved;
+  }
+
+  private Path movePath(Path p, Path src, Path dest) {
+    String srcStr = src.toUri().getPath();
+    String pathStr = p.toUri().getPath();
+    // Strip off src dir
+    pathStr = pathStr.substring(srcStr.length());
+    // Prepend new dest
+    return new Path(dest, pathStr);
+  }
+
+  private S3AFileStatus copyStatus(S3AFileStatus status) {
+    if (status.isDirectory()) {
+      return new S3AFileStatus(status.isEmptyDirectory(), status.getPath(),
+          status.getOwner());
+    } else {
+      return new S3AFileStatus(status.getLen(), status.getModificationTime(),
+          status.getPath(), status.getBlockSize(), status.getOwner());
+    }
+  }
+
+  /** @return number of PathMetadatas put() into MetadataStore */
+  private long populateMetadataStore(Collection<PathMetadata> paths,
+      MetadataStore ms) throws IOException {
+    long count = 0;
+    NanoTimer putTimer = new NanoTimer();
+    describe("Inserting into MetadataStore");
+    for (PathMetadata p : paths) {
+      ms.put(p);
+      count++;
+    }
+    putTimer.end();
+    printTiming(LOG, "put", putTimer, count);
+    return count;
+  }
+
+  private void clearMetadataStore(MetadataStore ms, long count)
+      throws IOException {
+    describe("Recursive deletion");
+    NanoTimer deleteTimer = new NanoTimer();
+    ms.deleteSubtree(BUCKET_ROOT);
+    deleteTimer.end();
+    printTiming(LOG, "delete", deleteTimer, count);
+  }
+
+  private static void printTiming(Logger log, String op, NanoTimer timer,
+      long count) {
+    double msec = (double)timer.duration() / 1000;
+    double msecPerOp = msec / count;
+    log.info(String.format("Elapsed %.2f msec. %.3f msec / %s (%d ops)", msec,
+        msecPerOp, op, count));
+  }
+
+  private static S3AFileStatus makeFileStatus(Path path) throws IOException {
+    return new S3AFileStatus(SIZE, ACCESS_TIME, path, BLOCK_SIZE, OWNER);
+  }
+
+  private static S3AFileStatus makeDirStatus(Path p) throws IOException {
+    return new S3AFileStatus(false, p, OWNER);
+  }
+
+  private List<Path> metasToPaths(List<PathMetadata> metas) {
+    List<Path> paths = new ArrayList<>(metas.size());
+    for (PathMetadata meta : metas) {
+      paths.add(meta.getFileStatus().getPath());
+    }
+    return paths;
+  }
+
+  /**
+   * Recursively create a directory tree.
+   * @param parent Parent dir of the paths to create.
+   * @param depth How many more levels deep past parent to create.
+   * @param width Number of files (and directories, if depth > 0) per directory.
+   * @param paths List to add generated paths to.
+   */
+  private static void createDirTree(Path parent, int depth, int width,
+      Collection<PathMetadata> paths) throws IOException {
+
+    // Create files
+    for (int i = 0; i < width; i++) {
+      Path p = new Path(parent, String.format("file-%d", i));
+      PathMetadata meta = new PathMetadata(makeFileStatus(p));
+      paths.add(meta);
+    }
+
+    if (depth == 0) {
+      return;
+    }
+
+    // Create directories if there is depth remaining
+    for (int i = 0; i < width; i++) {
+      Path dir = new Path(parent, String.format("dir-%d", i));
+      PathMetadata meta = new PathMetadata(makeDirStatus(dir));
+      paths.add(meta);
+      createDirTree(dir, depth-1, width, paths);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index 89fae82..8b163cb 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.amazonaws.event.ProgressEvent;
 import com.amazonaws.event.ProgressEventType;
 import com.amazonaws.event.ProgressListener;
+import org.apache.hadoop.fs.FileStatus;
 import org.junit.FixMethodOrder;
 import org.junit.Test;
 import org.junit.runners.MethodSorters;
@@ -34,11 +35,9 @@ import org.slf4j.LoggerFactory;
 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.Path;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.Statistic;
@@ -222,7 +221,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     assertEquals("active put requests in \n" + fs,
         0, gaugeValue(putRequestsActive));
     ContractTestUtils.assertPathExists(fs, "Huge file", hugefile);
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     ContractTestUtils.assertIsFile(hugefile, status);
     assertEquals("File size in " + status, filesize, status.getLen());
     if (progress != null) {
@@ -324,7 +323,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     String filetype = encrypted ? "encrypted file" : "file";
     describe("Positioned reads of %s %s", filetype, hugefile);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     long filesize = status.getLen();
     int ops = 0;
     final int bufferSize = 8192;
@@ -364,7 +363,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     assumeHugeFileExists();
     describe("Reading %s", hugefile);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     long filesize = status.getLen();
     long blocks = filesize / uploadBlockSize;
     byte[] data = new byte[uploadBlockSize];
@@ -390,7 +389,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     assumeHugeFileExists();
     describe("renaming %s to %s", hugefile, hugefileRenamed);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     long filesize = status.getLen();
     fs.delete(hugefileRenamed, false);
     ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
@@ -401,7 +400,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
         toHuman(timer.nanosPerOperation(mb)));
     bandwidth(timer, filesize);
     logFSState();
-    S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
+    FileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
     assertEquals(filesize, destFileStatus.getLen());
 
     // rename back

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java
new file mode 100644
index 0000000..3de1935
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java
@@ -0,0 +1,48 @@
+/*
+ * 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.s3a.scale;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+
+import java.io.IOException;
+
+import static org.junit.Assume.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Scale test for DynamoDBMetadataStore.
+ */
+public class ITestDynamoDBMetadataStoreScale
+    extends AbstractITestS3AMetadataStoreScale {
+
+  @Override
+  public MetadataStore createMetadataStore() throws IOException {
+    Configuration conf = getFileSystem().getConf();
+    String ddbTable = conf.get(S3GUARD_DDB_TABLE_NAME_KEY);
+    assumeNotNull("DynamoDB table is configured", ddbTable);
+    String ddbEndpoint = conf.get(S3GUARD_DDB_REGION_KEY);
+    assumeNotNull("DynamoDB endpoint is configured", ddbEndpoint);
+
+    DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+    ms.initialize(getFileSystem().getConf());
+    return ms;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java
new file mode 100644
index 0000000..591fb0e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+
+import java.io.IOException;
+
+/**
+ * Scale test for LocalMetadataStore.
+ */
+public class ITestLocalMetadataStoreScale
+    extends AbstractITestS3AMetadataStoreScale {
+  @Override
+  public MetadataStore createMetadataStore() throws IOException {
+    MetadataStore ms = new LocalMetadataStore();
+    ms.initialize(getFileSystem());
+    return ms;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
index b4d3862..e320bb2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
@@ -107,7 +107,7 @@ public class ITestS3AConcurrentOps extends S3AScaleTestBase {
 
   private S3AFileSystem getNormalFileSystem() throws Exception {
     S3AFileSystem s3a = new S3AFileSystem();
-    Configuration conf = new Configuration();
+    Configuration conf = createScaleConfiguration();
     URI rootURI = new URI(conf.get(TEST_FS_S3A_NAME));
     s3a.initialize(rootURI, conf);
     return s3a;
@@ -115,6 +115,7 @@ public class ITestS3AConcurrentOps extends S3AScaleTestBase {
 
   @After
   public void teardown() throws Exception {
+    super.teardown();
     if (auxFs != null) {
       auxFs.delete(testRoot, true);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java
new file mode 100644
index 0000000..fd32ba5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.OutputStream;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
+/**
+ * Tests for create(): performance and/or load testing.
+ */
+public class ITestS3ACreatePerformance extends S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ITestS3ADirectoryPerformance.class);
+
+  private Path basePath;
+  private int basePathDepth;
+  private static final int PATH_DEPTH = 10;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    basePath = getTestPath();
+    basePathDepth = basePath.depth();
+  }
+
+  /**
+   * Test rate at which we can create deeply-nested files from a single thread.
+   * @throws Exception
+   */
+  @Test
+  public void testDeepSequentialCreate() throws Exception {
+    long numOperations = getOperationCount();
+    S3AFileSystem fs = getFileSystem();
+
+    NanoTimer timer = new NanoTimer();
+    for (int i = 0; i < numOperations; i++) {
+      Path p = getPathIteration(i, PATH_DEPTH);
+      OutputStream out = fs.create(p);
+      out.write(40);  // one byte file with some value 40
+      out.close();
+    }
+    timer.end("Time to create %d files of depth %d", getOperationCount(),
+        PATH_DEPTH);
+    LOG.info("Time per create: {} msec",
+        timer.nanosPerOperation(numOperations) / 1000);
+  }
+
+  /* Get a unique path of depth totalDepth for given test iteration. */
+  private Path getPathIteration(long iter, int totalDepth) throws Exception {
+    assertTrue("Test path too long, increase PATH_DEPTH in test.",
+        totalDepth > basePathDepth);
+
+    int neededDirs = totalDepth - basePathDepth - 1;
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < neededDirs; i++) {
+      sb.append("iter-").append(iter);
+      sb.append("-dir-").append(i);
+      sb.append("/");
+    }
+    sb.append("file").append(iter);
+    return new Path(basePath, sb.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
index d71364f..03f1e22 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
@@ -113,14 +113,15 @@ public class ITestS3ADirectoryPerformance extends S3AScaleTestBase {
           listContinueRequests,
           listStatusCalls,
           getFileStatusCalls);
-      assertEquals(listRequests.toString(), 2, listRequests.diff());
+      if (!fs.hasMetadataStore()) {
+        assertEquals(listRequests.toString(), 2, listRequests.diff());
+      }
       reset(metadataRequests,
           listRequests,
           listContinueRequests,
           listStatusCalls,
           getFileStatusCalls);
 
-
     } finally {
       describe("deletion");
       // deletion at the end of the run

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index 236ffcd..83ab210 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.fs.s3a.scale;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
 import org.apache.hadoop.fs.s3a.S3AInputStream;
@@ -56,7 +56,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
 
   private S3AFileSystem s3aFS;
   private Path testData;
-  private S3AFileStatus testDataStatus;
+  private FileStatus testDataStatus;
   private FSDataInputStream in;
   private S3AInstrumentation.InputStreamStatistics streamStatistics;
   public static final int BLOCK_SIZE = 32 * 1024;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index 0f844b1..b2a1aa0 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -126,7 +126,7 @@ public class S3AScaleTestBase extends AbstractS3ATestBase {
    * @return a configuration with which to create FS instances
    */
   protected Configuration createScaleConfiguration() {
-    return new Configuration();
+    return super.createConfiguration();
   }
 
   protected Path getTestPath() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
index d424aa4..e8200da 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
+++ b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
@@ -36,6 +36,25 @@
     <description>The endpoint for s3a://landsat-pds URLs</description>
   </property>
 
+  <!-- Make sure S3Guard is disabled for read-only bucket tests. -->
+  <property>
+    <name>fs.s3a.bucket.landsat-pds.metadatastore.impl</name>
+    <value>${s3guard.null}</value>
+    <description>The read-only landsat-pds repository isn't
+        managed by s3guard</description>
+  </property>
+
+  <!-- Convenience definitions. -->
+  <property>
+    <name>s3guard.null</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+  </property>
+
+  <property>
+    <name>s3guard.dynamo</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</value>
+  </property>
+
   <!--
    This is the default endpoint, which can be used to interact
    with any v2 region.
@@ -110,6 +129,13 @@
     <value>${central.endpoint}</value>
   </property>
 
+  <!-- Scale integration tests may time out on slower connections
+       you can reduce the operation count like so to mitigate this.
+  <property>
+      <name>scale.test.operation.count</name>
+      <value>500</value>
+  </property>
+  -->
 
   <!-- Turn security off for tests by default -->
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
index 1330ed1..9376ebd 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
@@ -19,5 +19,16 @@ log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%
 
 log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
 
-# for debugging low level S3a operations, uncomment this line
-# log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+# for debugging low level S3a operations, uncomment these lines
+# Log all S3A classes
+#log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+
+# Log S3Guard classes
+#log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG
+
+# Enable debug logging of AWS DynamoDB client
+#log4j.logger.com.amazonaws.services.dynamodbv2.AmazonDynamoDB=DEBUG
+
+# Log all HTTP requests made; includes S3 interaction. This may
+# include sensitive information such as account IDs in HTTP headers.
+#log4j.logger.com.amazonaws.request=DEBUG


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


[33/48] 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/9203e566
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9203e566
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9203e566

Branch: refs/heads/HDFS-10467
Commit: 9203e5666fb856f44ac85341791c24064a4a74a7
Parents: 190510f
Author: Inigo Goiri <in...@apache.org>
Authored: Tue May 2 15:49:53 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:08 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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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/9203e566/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


[40/48] 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/02c53b94
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/02c53b94
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/02c53b94

Branch: refs/heads/HDFS-10467
Commit: 02c53b94e90b04e0b453b4b6227118711d99799b
Parents: c0a0692
Author: Inigo Goiri <in...@apache.org>
Authored: Mon Jul 31 10:55:21 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:09 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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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/02c53b94/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


[45/48] 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/e91ecb65
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e91ecb65
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e91ecb65

Branch: refs/heads/HDFS-10467
Commit: e91ecb658a2ccf6abe1b644716bd2c4ff3a34f9e
Parents: 37449af
Author: Inigo Goiri <in...@apache.org>
Authored: Wed Aug 16 17:23:29 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:10 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/e91ecb65/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/e91ecb65/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


[27/48] 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/cd09291f/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/cd09291f/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/cd09291f/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/cd09291f/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/48] hadoop git commit: HDFS-12363. Possible NPE in BlockManager$StorageInfoDefragmenter#scanAndCompactStorages. Contributed by Xiao Chen

Posted by in...@apache.org.
HDFS-12363. Possible NPE in BlockManager$StorageInfoDefragmenter#scanAndCompactStorages. 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/1fbb662c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1fbb662c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1fbb662c

Branch: refs/heads/HDFS-10467
Commit: 1fbb662c7092d08a540acff7e92715693412e486
Parents: 7ecc6db
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Aug 31 22:36:56 2017 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Aug 31 22:36:56 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java     | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fbb662c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 6129db8..e83cbc6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -4487,8 +4487,12 @@ public class BlockManager implements BlockStatsMXBean {
         for (int i = 0; i < datanodesAndStorages.size(); i += 2) {
           namesystem.writeLock();
           try {
-            DatanodeStorageInfo storage = datanodeManager.
-                getDatanode(datanodesAndStorages.get(i)).
+            final DatanodeDescriptor dn = datanodeManager.
+                getDatanode(datanodesAndStorages.get(i));
+            if (dn == null) {
+              continue;
+            }
+            final DatanodeStorageInfo storage = dn.
                 getStorageInfo(datanodesAndStorages.get(i + 1));
             if (storage != null) {
               boolean aborted =


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


[43/48] 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/c7b39ef8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c7b39ef8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c7b39ef8

Branch: refs/heads/HDFS-10467
Commit: c7b39ef87457f5cf15720de926b60252f011291d
Parents: a8d935b
Author: Inigo Goiri <in...@apache.org>
Authored: Fri Aug 4 18:00:12 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:09 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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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/c7b39ef8/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


[10/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
new file mode 100644
index 0000000..c19ae91
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
@@ -0,0 +1,887 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.io.IOUtils;
+
+/**
+ * Main test class for MetadataStore implementations.
+ * Implementations should each create a test by subclassing this and
+ * overriding {@link #createContract()}.
+ * If your implementation may return missing results for recently set paths,
+ * override {@link MetadataStoreTestBase#allowMissing()}.
+ */
+public abstract class MetadataStoreTestBase extends Assert {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MetadataStoreTestBase.class);
+
+  /** Some dummy values for sanity-checking FileStatus contents. */
+  static final long BLOCK_SIZE = 32 * 1024 * 1024;
+  static final int REPLICATION = 1;
+  static final FsPermission PERMISSION = new FsPermission((short)0755);
+  static final String OWNER = "bob";
+  static final String GROUP = "uncles";
+  private final long accessTime = System.currentTimeMillis();
+  private final long modTime = accessTime - 5000;
+
+  /**
+   * Each test should override this.  Will use a new Configuration instance.
+   * @return Contract which specifies the MetadataStore under test plus config.
+   */
+  public abstract AbstractMSContract createContract() throws IOException;
+
+  /**
+   * Each test should override this.
+   * @param conf Base configuration instance to use.
+   * @return Contract which specifies the MetadataStore under test plus config.
+   */
+  public abstract AbstractMSContract createContract(Configuration conf)
+      throws IOException;
+
+  /**
+   * Tests assume that implementations will return recently set results.  If
+   * your implementation does not always hold onto metadata (e.g. LRU or
+   * time-based expiry) you can override this to return false.
+   * @return true if the test should succeed when null results are returned
+   *  from the MetadataStore under test.
+   */
+  public boolean allowMissing() {
+    return false;
+  }
+
+  /**
+   * Pruning is an optional feature for metadata store implementations.
+   * Tests will only check that functionality if it is expected to work.
+   * @return true if the test should expect pruning to work.
+   */
+  public boolean supportsPruning() {
+    return true;
+  }
+
+  /** The MetadataStore contract used to test against. */
+  private AbstractMSContract contract;
+
+  private MetadataStore ms;
+
+  /**
+   * @return reference to the test contract.
+   */
+  protected AbstractMSContract getContract() {
+    return contract;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    LOG.debug("== Setup. ==");
+    contract = createContract();
+    ms = contract.getMetadataStore();
+    assertNotNull("null MetadataStore", ms);
+    assertNotNull("null FileSystem", contract.getFileSystem());
+    ms.initialize(contract.getFileSystem());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    LOG.debug("== Tear down. ==");
+    if (ms != null) {
+      try {
+        ms.destroy();
+      } catch (Exception e) {
+        LOG.warn("Failed to destroy tables in teardown", e);
+      }
+      IOUtils.closeStream(ms);
+      ms = null;
+    }
+  }
+
+  /**
+   * Helper function for verifying DescendantsIterator and
+   * MetadataStoreListFilesIterator behavior.
+   * @param createNodes List of paths to create
+   * @param checkNodes List of paths that the iterator should return
+   */
+  private void doTestDescendantsIterator(
+      Class implementation, String[] createNodes,
+      String[] checkNodes) throws Exception {
+    // we set up the example file system tree in metadata store
+    for (String pathStr : createNodes) {
+      final FileStatus status = pathStr.contains("file")
+          ? basicFileStatus(strToPath(pathStr), 100, false)
+          : basicFileStatus(strToPath(pathStr), 0, true);
+      ms.put(new PathMetadata(status));
+    }
+
+    final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/"));
+    RemoteIterator<FileStatus> iterator;
+    if (implementation == DescendantsIterator.class) {
+      iterator = new DescendantsIterator(ms, rootMeta);
+    } else if (implementation == MetadataStoreListFilesIterator.class) {
+      iterator = new MetadataStoreListFilesIterator(ms, rootMeta, false);
+    } else {
+      throw new UnsupportedOperationException("Unrecognized class");
+    }
+
+    final Set<String> actual = new HashSet<>();
+    while (iterator.hasNext()) {
+      final Path p = iterator.next().getPath();
+      actual.add(Path.getPathWithoutSchemeAndAuthority(p).toString());
+    }
+    LOG.info("We got {} by iterating DescendantsIterator", actual);
+
+    if (!allowMissing()) {
+      assertEquals(Sets.newHashSet(checkNodes), actual);
+    }
+  }
+
+  /**
+   * Test that we can get the whole sub-tree by iterating DescendantsIterator.
+   *
+   * The tree is similar to or same as the example in code comment.
+   */
+  @Test
+  public void testDescendantsIterator() throws Exception {
+    final String[] tree = new String[] {
+        "/dir1",
+        "/dir1/dir2",
+        "/dir1/dir3",
+        "/dir1/dir2/file1",
+        "/dir1/dir2/file2",
+        "/dir1/dir3/dir4",
+        "/dir1/dir3/dir5",
+        "/dir1/dir3/dir4/file3",
+        "/dir1/dir3/dir5/file4",
+        "/dir1/dir3/dir6"
+    };
+    doTestDescendantsIterator(DescendantsIterator.class,
+        tree, tree);
+  }
+
+  /**
+   * Test that we can get the correct subset of the tree with
+   * MetadataStoreListFilesIterator.
+   *
+   * The tree is similar to or same as the example in code comment.
+   */
+  @Test
+  public void testMetadataStoreListFilesIterator() throws Exception {
+    final String[] wholeTree = new String[] {
+        "/dir1",
+        "/dir1/dir2",
+        "/dir1/dir3",
+        "/dir1/dir2/file1",
+        "/dir1/dir2/file2",
+        "/dir1/dir3/dir4",
+        "/dir1/dir3/dir5",
+        "/dir1/dir3/dir4/file3",
+        "/dir1/dir3/dir5/file4",
+        "/dir1/dir3/dir6"
+    };
+    final String[] leafNodes = new String[] {
+        "/dir1/dir2/file1",
+        "/dir1/dir2/file2",
+        "/dir1/dir3/dir4/file3",
+        "/dir1/dir3/dir5/file4"
+    };
+    doTestDescendantsIterator(MetadataStoreListFilesIterator.class, wholeTree,
+        leafNodes);
+  }
+
+  @Test
+  public void testPutNew() throws Exception {
+    /* create three dirs /da1, /da2, /da3 */
+    createNewDirs("/da1", "/da2", "/da3");
+
+    /* It is caller's responsibility to set up ancestor entries beyond the
+     * containing directory.  We only track direct children of the directory.
+     * Thus this will not affect entry for /da1.
+     */
+    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)));
+
+    assertEmptyDirs("/da2", "/da3");
+    assertDirectorySize("/da1/db1", 1);
+
+    /* Check contents of dir status. */
+    PathMetadata dirMeta = ms.get(strToPath("/da1"));
+    if (!allowMissing() || dirMeta != null) {
+      verifyDirStatus(dirMeta.getFileStatus());
+    }
+
+    /* This already exists, and should silently replace it. */
+    ms.put(new PathMetadata(makeDirStatus("/da1/db1")));
+
+    /* If we had putNew(), and used it above, this would be empty again. */
+    assertDirectorySize("/da1", 1);
+
+    assertEmptyDirs("/da2", "/da3");
+
+    /* Ensure new files update correct parent dirs. */
+    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)));
+    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200)));
+    assertDirectorySize("/da1", 1);
+    assertDirectorySize("/da1/db1", 2);
+    assertEmptyDirs("/da2", "/da3");
+    PathMetadata meta = ms.get(strToPath("/da1/db1/fc2"));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get file after put new.", meta);
+      verifyFileStatus(meta.getFileStatus(), 200);
+    }
+  }
+
+  @Test
+  public void testPutOverwrite() throws Exception {
+    final String filePath = "/a1/b1/c1/some_file";
+    final String dirPath = "/a1/b1/c1/d1";
+    ms.put(new PathMetadata(makeFileStatus(filePath, 100)));
+    ms.put(new PathMetadata(makeDirStatus(dirPath)));
+    PathMetadata meta = ms.get(strToPath(filePath));
+    if (!allowMissing() || meta != null) {
+      verifyFileStatus(meta.getFileStatus(), 100);
+    }
+
+    ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false)));
+    meta = ms.get(strToPath(filePath));
+    if (!allowMissing() || meta != null) {
+      verifyFileStatus(meta.getFileStatus(), 9999);
+    }
+  }
+
+  @Test
+  public void testRootDirPutNew() throws Exception {
+    Path rootPath = strToPath("/");
+
+    ms.put(new PathMetadata(makeFileStatus("/file1", 100)));
+    DirListingMetadata dir = ms.listChildren(rootPath);
+    if (!allowMissing() || dir != null) {
+      assertNotNull("Root dir cached", dir);
+      assertFalse("Root not fully cached", dir.isAuthoritative());
+      assertNotNull("have root dir file listing", dir.getListing());
+      assertEquals("One file in root dir", 1, dir.getListing().size());
+      assertEquals("file1 in root dir", strToPath("/file1"),
+          dir.getListing().iterator().next().getFileStatus().getPath());
+    }
+  }
+
+  @Test
+  public void testDelete() throws Exception {
+    setUpDeleteTest();
+
+    ms.delete(strToPath("/ADirectory1/db1/file2"));
+
+    /* Ensure delete happened. */
+    assertDirectorySize("/ADirectory1/db1", 1);
+    PathMetadata meta = ms.get(strToPath("/ADirectory1/db1/file2"));
+    assertTrue("File deleted", meta == null || meta.isDeleted());
+  }
+
+  @Test
+  public void testDeleteSubtree() throws Exception {
+    deleteSubtreeHelper("");
+  }
+
+  @Test
+  public void testDeleteSubtreeHostPath() throws Exception {
+    deleteSubtreeHelper(contract.getFileSystem().getUri().toString());
+  }
+
+  private void deleteSubtreeHelper(String pathPrefix) throws Exception {
+
+    String p = pathPrefix;
+    setUpDeleteTest(p);
+    createNewDirs(p + "/ADirectory1/db1/dc1", p + "/ADirectory1/db1/dc1/dd1");
+    ms.put(new PathMetadata(
+        makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100)));
+    if (!allowMissing()) {
+      assertCached(p + "/ADirectory1/db1");
+    }
+    ms.deleteSubtree(strToPath(p + "/ADirectory1/db1/"));
+
+    assertEmptyDirectory(p + "/ADirectory1");
+    assertDeleted(p + "/ADirectory1/db1");
+    assertDeleted(p + "/ADirectory1/file1");
+    assertDeleted(p + "/ADirectory1/file2");
+    assertDeleted(p + "/ADirectory1/db1/dc1/dd1/deepFile");
+    assertEmptyDirectory(p + "/ADirectory2");
+  }
+
+
+  /*
+   * Some implementations might not support this.  It was useful to test
+   * correctness of the LocalMetadataStore implementation, but feel free to
+   * override this to be a no-op.
+   */
+  @Test
+  public void testDeleteRecursiveRoot() throws Exception {
+    setUpDeleteTest();
+
+    ms.deleteSubtree(strToPath("/"));
+    assertDeleted("/ADirectory1");
+    assertDeleted("/ADirectory2");
+    assertDeleted("/ADirectory2/db1");
+    assertDeleted("/ADirectory2/db1/file1");
+    assertDeleted("/ADirectory2/db1/file2");
+  }
+
+  @Test
+  public void testDeleteNonExisting() throws Exception {
+    // Path doesn't exist, but should silently succeed
+    ms.delete(strToPath("/bobs/your/uncle"));
+
+    // Ditto.
+    ms.deleteSubtree(strToPath("/internets"));
+  }
+
+
+  private void setUpDeleteTest() throws IOException {
+    setUpDeleteTest("");
+  }
+
+  private void setUpDeleteTest(String prefix) throws IOException {
+    createNewDirs(prefix + "/ADirectory1", prefix + "/ADirectory2",
+        prefix + "/ADirectory1/db1");
+    ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file1",
+        100)));
+    ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file2",
+        100)));
+
+    PathMetadata meta = ms.get(strToPath(prefix + "/ADirectory1/db1/file2"));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Found test file", meta);
+      assertDirectorySize(prefix + "/ADirectory1/db1", 2);
+    }
+  }
+
+  @Test
+  public void testGet() throws Exception {
+    final String filePath = "/a1/b1/c1/some_file";
+    final String dirPath = "/a1/b1/c1/d1";
+    ms.put(new PathMetadata(makeFileStatus(filePath, 100)));
+    ms.put(new PathMetadata(makeDirStatus(dirPath)));
+    PathMetadata meta = ms.get(strToPath(filePath));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get found file", meta);
+      verifyFileStatus(meta.getFileStatus(), 100);
+    }
+
+    if (!(ms instanceof NullMetadataStore)) {
+      ms.delete(strToPath(filePath));
+      meta = ms.get(strToPath(filePath));
+      assertTrue("Tombstone not left for deleted file", meta.isDeleted());
+    }
+
+    meta = ms.get(strToPath(dirPath));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get found file (dir)", meta);
+      assertTrue("Found dir", meta.getFileStatus().isDirectory());
+    }
+
+    meta = ms.get(strToPath("/bollocks"));
+    assertNull("Don't get non-existent file", meta);
+  }
+
+  @Test
+  public void testGetEmptyDir() throws Exception {
+    final String dirPath = "/a1/b1/c1/d1";
+    // Creates /a1/b1/c1/d1 as an empty dir
+    setupListStatus();
+
+    // 1. Tell MetadataStore (MS) that there are zero children
+    putListStatusFiles(dirPath, true /* authoritative */
+        /* zero children */);
+
+    // 2. Request a file status for dir, including whether or not the dir
+    // is empty.
+    PathMetadata meta = ms.get(strToPath(dirPath), true);
+
+    // 3. Check that either (a) the MS doesn't track whether or not it is
+    // empty (which is allowed), or (b) the MS knows the dir is empty.
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get should find meta for dir", meta);
+      assertNotEquals("Dir is empty or unknown", Tristate.FALSE,
+          meta.isEmptyDirectory());
+    }
+  }
+
+  @Test
+  public void testGetNonEmptyDir() throws Exception {
+    final String dirPath = "/a1/b1/c1";
+    // Creates /a1/b1/c1 as an non-empty dir
+    setupListStatus();
+
+    // Request a file status for dir, including whether or not the dir
+    // is empty.
+    PathMetadata meta = ms.get(strToPath(dirPath), true);
+
+    // MetadataStore knows /a1/b1/c1 has at least one child.  It is valid
+    // for it to answer either (a) UNKNOWN: the MS doesn't track whether
+    // or not the dir is empty, or (b) the MS knows the dir is non-empty.
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get should find meta for dir", meta);
+      assertNotEquals("Dir is non-empty or unknown", Tristate.TRUE,
+          meta.isEmptyDirectory());
+    }
+  }
+
+  @Test
+  public void testGetDirUnknownIfEmpty() throws Exception {
+    final String dirPath = "/a1/b1/c1/d1";
+    // 1. Create /a1/b1/c1/d1 as an empty dir, but do not tell MetadataStore
+    // (MS) whether or not it has any children.
+    setupListStatus();
+
+    // 2. Request a file status for dir, including whether or not the dir
+    // is empty.
+    PathMetadata meta = ms.get(strToPath(dirPath), true);
+
+    // 3. Assert MS reports isEmptyDir as UNKONWN: We haven't told MS
+    // whether or not the directory has any children.
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get should find meta for dir", meta);
+      assertEquals("Dir empty is unknown", Tristate.UNKNOWN,
+          meta.isEmptyDirectory());
+    }
+  }
+
+  @Test
+  public void testListChildren() throws Exception {
+    setupListStatus();
+
+    DirListingMetadata dirMeta;
+    dirMeta = ms.listChildren(strToPath("/"));
+    if (!allowMissing()) {
+      assertNotNull(dirMeta);
+        /* Cache has no way of knowing it has all entries for root unless we
+         * specifically tell it via put() with
+         * DirListingMetadata.isAuthoritative = true */
+      assertFalse("Root dir is not cached, or partially cached",
+          dirMeta.isAuthoritative());
+      assertListingsEqual(dirMeta.getListing(), "/a1", "/a2");
+    }
+
+    dirMeta = ms.listChildren(strToPath("/a1"));
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertListingsEqual(dirMeta.getListing(), "/a1/b1", "/a1/b2");
+    }
+
+    // TODO HADOOP-14756 instrument MetadataStore for asserting & testing
+    dirMeta = ms.listChildren(strToPath("/a1/b1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertListingsEqual(dirMeta.getListing(), "/a1/b1/file1", "/a1/b1/file2",
+          "/a1/b1/c1");
+    }
+  }
+
+  @Test
+  public void testDirListingRoot() throws Exception {
+    commonTestPutListStatus("/");
+  }
+
+  @Test
+  public void testPutDirListing() throws Exception {
+    commonTestPutListStatus("/a");
+  }
+
+  @Test
+  public void testInvalidListChildren() throws Exception {
+    setupListStatus();
+    assertNull("missing path returns null",
+        ms.listChildren(strToPath("/a1/b1x")));
+  }
+
+  @Test
+  public void testMove() throws Exception {
+    // Create test dir structure
+    createNewDirs("/a1", "/a2", "/a3");
+    createNewDirs("/a1/b1", "/a1/b2");
+    putListStatusFiles("/a1/b1", false, "/a1/b1/file1", "/a1/b1/file2");
+
+    // Assert root listing as expected
+    Collection<PathMetadata> entries;
+    DirListingMetadata dirMeta = ms.listChildren(strToPath("/"));
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertNotNull("Listing root", dirMeta);
+      entries = dirMeta.getListing();
+      assertListingsEqual(entries, "/a1", "/a2", "/a3");
+    }
+
+    // Assert src listing as expected
+    dirMeta = ms.listChildren(strToPath("/a1/b1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertNotNull("Listing /a1/b1", dirMeta);
+      entries = dirMeta.getListing();
+      assertListingsEqual(entries, "/a1/b1/file1", "/a1/b1/file2");
+    }
+
+    // Do the move(): rename(/a1/b1, /b1)
+    Collection<Path> srcPaths = Arrays.asList(strToPath("/a1/b1"),
+        strToPath("/a1/b1/file1"), strToPath("/a1/b1/file2"));
+
+    ArrayList<PathMetadata> destMetas = new ArrayList<>();
+    destMetas.add(new PathMetadata(makeDirStatus("/b1")));
+    destMetas.add(new PathMetadata(makeFileStatus("/b1/file1", 100)));
+    destMetas.add(new PathMetadata(makeFileStatus("/b1/file2", 100)));
+    ms.move(srcPaths, destMetas);
+
+    // Assert src is no longer there
+    dirMeta = ms.listChildren(strToPath("/a1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertNotNull("Listing /a1", dirMeta);
+      entries = dirMeta.withoutTombstones().getListing();
+      assertListingsEqual(entries, "/a1/b2");
+    }
+
+    PathMetadata meta = ms.get(strToPath("/a1/b1/file1"));
+    assertTrue("Src path deleted", meta == null || meta.isDeleted());
+
+    // Assert dest looks right
+    meta = ms.get(strToPath("/b1/file1"));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("dest file not null", meta);
+      verifyFileStatus(meta.getFileStatus(), 100);
+    }
+
+    dirMeta = ms.listChildren(strToPath("/b1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertNotNull("dest listing not null", dirMeta);
+      entries = dirMeta.getListing();
+      assertListingsEqual(entries, "/b1/file1", "/b1/file2");
+    }
+  }
+
+  /**
+   * Test that the MetadataStore differentiates between the same path in two
+   * different buckets.
+   */
+  @Test
+  public void testMultiBucketPaths() throws Exception {
+    String p1 = "s3a://bucket-a/path1";
+    String p2 = "s3a://bucket-b/path2";
+
+    // Make sure we start out empty
+    PathMetadata meta = ms.get(new Path(p1));
+    assertNull("Path should not be present yet.", meta);
+    meta = ms.get(new Path(p2));
+    assertNull("Path2 should not be present yet.", meta);
+
+    // Put p1, assert p2 doesn't match
+    ms.put(new PathMetadata(makeFileStatus(p1, 100)));
+    meta = ms.get(new Path(p2));
+    assertNull("Path 2 should not match path 1.", meta);
+
+    // Make sure delete is correct as well
+    if (!allowMissing()) {
+      ms.delete(new Path(p2));
+      meta = ms.get(new Path(p1));
+      assertNotNull("Path should not have been deleted", meta);
+    }
+    ms.delete(new Path(p1));
+  }
+
+  @Test
+  public void testPruneFiles() throws Exception {
+    Assume.assumeTrue(supportsPruning());
+    createNewDirs("/pruneFiles");
+
+    long oldTime = getTime();
+    ms.put(new PathMetadata(makeFileStatus("/pruneFiles/old", 1, oldTime,
+        oldTime)));
+    DirListingMetadata ls2 = ms.listChildren(strToPath("/pruneFiles"));
+    if (!allowMissing()) {
+      assertListingsEqual(ls2.getListing(), "/pruneFiles/old");
+    }
+
+    // It's possible for the Local implementation to get from /pruneFiles/old's
+    // modification time to here in under 1ms, causing it to not get pruned
+    Thread.sleep(1);
+    long cutoff = System.currentTimeMillis();
+    long newTime = getTime();
+    ms.put(new PathMetadata(makeFileStatus("/pruneFiles/new", 1, newTime,
+        newTime)));
+
+    DirListingMetadata ls;
+    ls = ms.listChildren(strToPath("/pruneFiles"));
+    if (!allowMissing()) {
+      assertListingsEqual(ls.getListing(), "/pruneFiles/new",
+          "/pruneFiles/old");
+    }
+    ms.prune(cutoff);
+    ls = ms.listChildren(strToPath("/pruneFiles"));
+    if (allowMissing()) {
+      assertDeleted("/pruneFiles/old");
+    } else {
+      assertListingsEqual(ls.getListing(), "/pruneFiles/new");
+    }
+  }
+
+  @Test
+  public void testPruneDirs() throws Exception {
+    Assume.assumeTrue(supportsPruning());
+
+    // We only test that files, not dirs, are removed during prune.
+    // We specifically allow directories to remain, as it is more robust
+    // for DynamoDBMetadataStore's prune() implementation: If a
+    // file was created in a directory while it was being pruned, it would
+    // violate the invariant that all ancestors of a file exist in the table.
+
+    createNewDirs("/pruneDirs/dir");
+
+    long oldTime = getTime();
+    ms.put(new PathMetadata(makeFileStatus("/pruneDirs/dir/file",
+        1, oldTime, oldTime)));
+
+    // It's possible for the Local implementation to get from the old
+    // modification time to here in under 1ms, causing it to not get pruned
+    Thread.sleep(1);
+    long cutoff = getTime();
+
+    ms.prune(cutoff);
+
+    assertDeleted("/pruneDirs/dir/file");
+  }
+
+  @Test
+  public void testPruneUnsetsAuthoritative() throws Exception {
+    String rootDir = "/unpruned-root-dir";
+    String grandparentDir = rootDir + "/pruned-grandparent-dir";
+    String parentDir = grandparentDir + "/pruned-parent-dir";
+    String staleFile = parentDir + "/stale-file";
+    String freshFile = rootDir + "/fresh-file";
+    String[] directories = {rootDir, grandparentDir, parentDir};
+
+    createNewDirs(rootDir, grandparentDir, parentDir);
+    long time = System.currentTimeMillis();
+    ms.put(new PathMetadata(
+        new FileStatus(0, false, 0, 0, time - 1, strToPath(staleFile)),
+        Tristate.FALSE, false));
+    ms.put(new PathMetadata(
+        new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)),
+        Tristate.FALSE, false));
+
+    ms.prune(time);
+    DirListingMetadata listing;
+    for (String directory : directories) {
+      Path path = strToPath(directory);
+      if (ms.get(path) != null) {
+        listing = ms.listChildren(path);
+        assertFalse(listing.isAuthoritative());
+      }
+    }
+  }
+
+  /*
+   * Helper functions.
+   */
+
+  /** Modifies paths input array and returns it. */
+  private String[] buildPathStrings(String parent, String... paths)
+      throws IOException {
+    for (int i = 0; i < paths.length; i++) {
+      Path p = new Path(strToPath(parent), paths[i]);
+      paths[i] = p.toString();
+    }
+    return paths;
+  }
+
+  private void commonTestPutListStatus(final String parent) throws IOException {
+    putListStatusFiles(parent, true, buildPathStrings(parent, "file1", "file2",
+        "file3"));
+    DirListingMetadata dirMeta = ms.listChildren(strToPath(parent));
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertNotNull("list after putListStatus", dirMeta);
+      Collection<PathMetadata> entries = dirMeta.getListing();
+      assertNotNull("listStatus has entries", entries);
+      assertListingsEqual(entries,
+          buildPathStrings(parent, "file1", "file2", "file3"));
+    }
+  }
+
+  private void setupListStatus() throws IOException {
+    createNewDirs("/a1", "/a2", "/a1/b1", "/a1/b2", "/a1/b1/c1",
+        "/a1/b1/c1/d1");
+    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100)));
+    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100)));
+  }
+
+  private void assertListingsEqual(Collection<PathMetadata> listing,
+      String ...pathStrs) throws IOException {
+    Set<Path> a = new HashSet<>();
+    for (PathMetadata meta : listing) {
+      a.add(meta.getFileStatus().getPath());
+    }
+
+    Set<Path> b = new HashSet<>();
+    for (String ps : pathStrs) {
+      b.add(strToPath(ps));
+    }
+    assertEquals("Same set of files", b, a);
+  }
+
+  private void putListStatusFiles(String dirPath, boolean authoritative,
+      String... filenames) throws IOException {
+    ArrayList<PathMetadata> metas = new ArrayList<>(filenames .length);
+    for (String filename : filenames) {
+      metas.add(new PathMetadata(makeFileStatus(filename, 100)));
+    }
+    DirListingMetadata dirMeta =
+        new DirListingMetadata(strToPath(dirPath), metas, authoritative);
+    ms.put(dirMeta);
+  }
+
+  private void createNewDirs(String... dirs)
+      throws IOException {
+    for (String pathStr : dirs) {
+      ms.put(new PathMetadata(makeDirStatus(pathStr)));
+    }
+  }
+
+  private void assertDirectorySize(String pathStr, int size)
+      throws IOException {
+    DirListingMetadata dirMeta = ms.listChildren(strToPath(pathStr));
+    if (!allowMissing()) {
+      assertNotNull("Directory " + pathStr + " in cache", dirMeta);
+    }
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertEquals("Number of entries in dir " + pathStr, size,
+          nonDeleted(dirMeta.getListing()).size());
+    }
+  }
+
+  /** @return only file statuses which are *not* marked deleted. */
+  private Collection<PathMetadata> nonDeleted(
+      Collection<PathMetadata> statuses) {
+    Collection<PathMetadata> currentStatuses = new ArrayList<>();
+    for (PathMetadata status : statuses) {
+      if (!status.isDeleted()) {
+        currentStatuses.add(status);
+      }
+    }
+    return currentStatuses;
+  }
+
+  private void assertDeleted(String pathStr) throws IOException {
+    Path path = strToPath(pathStr);
+    PathMetadata meta = ms.get(path);
+    boolean cached = meta != null && !meta.isDeleted();
+    assertFalse(pathStr + " should not be cached.", cached);
+  }
+
+  protected void assertCached(String pathStr) throws IOException {
+    Path path = strToPath(pathStr);
+    PathMetadata meta = ms.get(path);
+    boolean cached = meta != null && !meta.isDeleted();
+    assertTrue(pathStr + " should be cached.", cached);
+  }
+
+  /**
+   * Convenience to create a fully qualified Path from string.
+   */
+  Path strToPath(String p) throws IOException {
+    final Path path = new Path(p);
+    assert path.isAbsolute();
+    return path.makeQualified(contract.getFileSystem().getUri(), null);
+  }
+
+  private void assertEmptyDirectory(String pathStr) throws IOException {
+    assertDirectorySize(pathStr, 0);
+  }
+
+  private void assertEmptyDirs(String ...dirs) throws IOException {
+    for (String pathStr : dirs) {
+      assertEmptyDirectory(pathStr);
+    }
+  }
+
+  FileStatus basicFileStatus(Path path, int size, boolean isDir) throws
+      IOException {
+    return basicFileStatus(path, size, isDir, modTime, accessTime);
+  }
+
+  FileStatus basicFileStatus(Path path, int size, boolean isDir,
+      long newModTime, long newAccessTime) throws IOException {
+    return new FileStatus(size, isDir, REPLICATION, BLOCK_SIZE, newModTime,
+        newAccessTime, PERMISSION, OWNER, GROUP, path);
+  }
+
+  private FileStatus makeFileStatus(String pathStr, int size) throws
+      IOException {
+    return makeFileStatus(pathStr, size, modTime, accessTime);
+  }
+
+  private FileStatus makeFileStatus(String pathStr, int size, long newModTime,
+      long newAccessTime) throws IOException {
+    return basicFileStatus(strToPath(pathStr), size, false,
+        newModTime, newAccessTime);
+  }
+
+  void verifyFileStatus(FileStatus status, long size) {
+    S3ATestUtils.verifyFileStatus(status, size, BLOCK_SIZE, modTime);
+  }
+
+  private FileStatus makeDirStatus(String pathStr) throws IOException {
+    return basicFileStatus(strToPath(pathStr), 0, true, modTime, accessTime);
+  }
+
+  /**
+   * Verify the directory file status. Subclass may verify additional fields.
+   */
+  void verifyDirStatus(FileStatus status) {
+    assertTrue("Is a dir", status.isDirectory());
+    assertEquals("zero length", 0, status.getLen());
+  }
+
+  long getModTime() {
+    return modTime;
+  }
+
+  long getAccessTime() {
+    return accessTime;
+  }
+
+  protected static long getTime() {
+    return System.currentTimeMillis();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java
new file mode 100644
index 0000000..8458252
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java
@@ -0,0 +1,303 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests of {@link DirListingMetadata}.
+ */
+public class TestDirListingMetadata {
+
+  private static final String TEST_OWNER = "hadoop";
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Test
+  public void testNullPath() {
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    new DirListingMetadata(null, null, false);
+  }
+
+  @Test
+  public void testNullListing() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertFalse(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testEmptyListing() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path,
+        new ArrayList<PathMetadata>(0),
+        false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertFalse(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testListing() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testListingUnmodifiable() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = makeTwoDirsOneFile(path);
+    assertNotNull(meta.getListing());
+    exception.expect(UnsupportedOperationException.class);
+    meta.getListing().clear();
+  }
+
+  @Test
+  public void testAuthoritative() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, true);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertTrue(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testSetAuthoritative() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertFalse(meta.isAuthoritative());
+    meta.setAuthoritative(true);
+    assertTrue(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testGet() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+    assertEquals(pathMeta1, meta.get(pathMeta1.getFileStatus().getPath()));
+    assertEquals(pathMeta2, meta.get(pathMeta2.getFileStatus().getPath()));
+    assertEquals(pathMeta3, meta.get(pathMeta3.getFileStatus().getPath()));
+    assertNull(meta.get(new Path(path, "notfound")));
+  }
+
+  @Test
+  public void testGetNull() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.get(null);
+  }
+
+  @Test
+  public void testGetRoot() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.get(new Path("/"));
+  }
+
+  @Test
+  public void testGetNotChild() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.get(new Path("/different/ancestor"));
+  }
+
+  @Test
+  public void testPut() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+    PathMetadata pathMeta4 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir3"), TEST_OWNER));
+    meta.put(pathMeta4.getFileStatus());
+    assertTrue(meta.getListing().contains(pathMeta4));
+    assertEquals(pathMeta4, meta.get(pathMeta4.getFileStatus().getPath()));
+  }
+
+  @Test
+  public void testPutNull() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(null);
+  }
+
+  @Test
+  public void testPutNullPath() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(new S3AFileStatus(true, null, TEST_OWNER));
+  }
+
+  @Test
+  public void testPutRoot() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(new S3AFileStatus(true, new Path("/"), TEST_OWNER));
+  }
+
+  @Test
+  public void testPutNotChild() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(new S3AFileStatus(true, new Path("/different/ancestor"),
+        TEST_OWNER));
+  }
+
+  @Test
+  public void testRemove() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+    meta.remove(pathMeta1.getFileStatus().getPath());
+    assertFalse(meta.getListing().contains(pathMeta1));
+    assertNull(meta.get(pathMeta1.getFileStatus().getPath()));
+  }
+
+  @Test
+  public void testRemoveNull() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.remove(null);
+  }
+
+  @Test
+  public void testRemoveRoot() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.remove(new Path("/"));
+  }
+
+  @Test
+  public void testRemoveNotChild() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.remove(new Path("/different/ancestor"));
+  }
+
+  /*
+   * Create DirListingMetadata with two dirs and one file living in directory
+   * 'parent'
+   */
+  private static DirListingMetadata makeTwoDirsOneFile(Path parent) {
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(parent, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(parent, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(parent, "file1"), 8192,
+            TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    return new DirListingMetadata(parent, listing, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java
new file mode 100644
index 0000000..02eb7b8
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java
@@ -0,0 +1,594 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.MockS3ClientFactory;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ClientFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
+import static org.apache.hadoop.test.LambdaTestUtils.*;
+
+/**
+ * Test that {@link DynamoDBMetadataStore} implements {@link MetadataStore}.
+ *
+ * In this unit test, we use an in-memory DynamoDBLocal server instead of real
+ * AWS DynamoDB. An {@link S3AFileSystem} object is created and shared for
+ * initializing {@link DynamoDBMetadataStore} objects.  There are no real S3
+ * request issued as the underlying AWS S3Client is mocked.  You won't be
+ * charged bills for AWS S3 or DynamoDB when you run this test.
+ *
+ * According to the base class, every test case will have independent contract
+ * to create a new {@link DynamoDBMetadataStore} instance and initializes it.
+ * A table will be created for each test by the test contract, and will be
+ * destroyed after the test case finishes.
+ */
+public class TestDynamoDBMetadataStore extends MetadataStoreTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestDynamoDBMetadataStore.class);
+  private static final String BUCKET = "TestDynamoDBMetadataStore";
+  private static final String S3URI =
+      URI.create(FS_S3A + "://" + BUCKET + "/").toString();
+  public static final PrimaryKey
+      VERSION_MARKER_PRIMARY_KEY = createVersionMarkerPrimaryKey(
+      DynamoDBMetadataStore.VERSION_MARKER);
+
+  /** The DynamoDB instance that can issue requests directly to server. */
+  private static DynamoDB dynamoDB;
+
+  @Rule
+  public final Timeout timeout = new Timeout(60 * 1000);
+
+  /**
+   * Start the in-memory DynamoDBLocal server and initializes s3 file system.
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    DynamoDBLocalClientFactory.startSingletonServer();
+    try {
+      dynamoDB = new DynamoDBMSContract().getMetadataStore().getDynamoDB();
+    } catch (AmazonServiceException e) {
+      final String msg = "Cannot initialize a DynamoDBMetadataStore instance "
+          + "against the local DynamoDB server. Perhaps the DynamoDBLocal "
+          + "server is not configured correctly. ";
+      LOG.error(msg, e);
+      // fail fast if the DynamoDBLocal server can not work
+      throw e;
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (dynamoDB != null) {
+      dynamoDB.shutdown();
+    }
+    DynamoDBLocalClientFactory.stopSingletonServer();
+  }
+
+  /**
+   * Each contract has its own S3AFileSystem and DynamoDBMetadataStore objects.
+   */
+  private static class DynamoDBMSContract extends AbstractMSContract {
+    private final S3AFileSystem s3afs;
+    private final DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+
+    DynamoDBMSContract() throws IOException {
+        this(new Configuration());
+    }
+
+    DynamoDBMSContract(Configuration conf) throws IOException {
+      // using mocked S3 clients
+      conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class,
+          S3ClientFactory.class);
+      conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, S3URI);
+      // setting config for creating a DynamoDBClient against local server
+      conf.set(ACCESS_KEY, "dummy-access-key");
+      conf.set(SECRET_KEY, "dummy-secret-key");
+      conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+      conf.setClass(S3Guard.S3GUARD_DDB_CLIENT_FACTORY_IMPL,
+          DynamoDBLocalClientFactory.class, DynamoDBClientFactory.class);
+
+      // always create new file system object for a test contract
+      s3afs = (S3AFileSystem) FileSystem.newInstance(conf);
+      ms.initialize(s3afs);
+    }
+
+    @Override
+    public S3AFileSystem getFileSystem() {
+      return s3afs;
+    }
+
+    @Override
+    public DynamoDBMetadataStore getMetadataStore() {
+      return ms;
+    }
+  }
+
+  @Override
+  public DynamoDBMSContract createContract() throws IOException {
+    return new DynamoDBMSContract();
+  }
+
+  @Override
+  public DynamoDBMSContract createContract(Configuration conf) throws
+      IOException {
+    return new DynamoDBMSContract(conf);
+  }
+
+  @Override
+  FileStatus basicFileStatus(Path path, int size, boolean isDir)
+      throws IOException {
+    String owner = UserGroupInformation.getCurrentUser().getShortUserName();
+    return isDir
+        ? new S3AFileStatus(true, path, owner)
+        : new S3AFileStatus(size, getModTime(), path, BLOCK_SIZE, owner);
+  }
+
+  private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException {
+    return (DynamoDBMetadataStore) getContract().getMetadataStore();
+  }
+
+  private S3AFileSystem getFileSystem() throws IOException {
+    return (S3AFileSystem) getContract().getFileSystem();
+  }
+
+  /**
+   * This tests that after initialize() using an S3AFileSystem object, the
+   * instance should have been initialized successfully, and tables are ACTIVE.
+   */
+  @Test
+  public void testInitialize() throws IOException {
+    final String tableName = "testInitializeWithFileSystem";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      verifyTableInitialized(tableName);
+      assertNotNull(ddbms.getTable());
+      assertEquals(tableName, ddbms.getTable().getTableName());
+      String expectedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
+          s3afs.getBucketLocation(tableName));
+      assertEquals("DynamoDB table should be in configured region or the same" +
+              " region as S3 bucket",
+          expectedRegion,
+          ddbms.getRegion());
+    }
+  }
+
+  /**
+   * This tests that after initialize() using a Configuration object, the
+   * instance should have been initialized successfully, and tables are ACTIVE.
+   */
+  @Test
+  public void testInitializeWithConfiguration() throws IOException {
+    final String tableName = "testInitializeWithConfiguration";
+    final Configuration conf = getFileSystem().getConf();
+    conf.unset(S3GUARD_DDB_TABLE_NAME_KEY);
+    String savedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
+        getFileSystem().getBucketLocation());
+    conf.unset(S3GUARD_DDB_REGION_KEY);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      fail("Should have failed because the table name is not set!");
+    } catch (IllegalArgumentException ignored) {
+    }
+    // config table name
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      fail("Should have failed because as the region is not set!");
+    } catch (IllegalArgumentException ignored) {
+    }
+    // config region
+    conf.set(S3GUARD_DDB_REGION_KEY, savedRegion);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      verifyTableInitialized(tableName);
+      assertNotNull(ddbms.getTable());
+      assertEquals(tableName, ddbms.getTable().getTableName());
+      assertEquals("Unexpected key schema found!",
+          keySchema(),
+          ddbms.getTable().describe().getKeySchema());
+    }
+  }
+
+  /**
+   * Test that for a large batch write request, the limit is handled correctly.
+   */
+  @Test
+  public void testBatchWrite() throws IOException {
+    final int[] numMetasToDeleteOrPut = {
+        -1, // null
+        0, // empty collection
+        1, // one path
+        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, // exact limit of a batch request
+        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT + 1 // limit + 1
+    };
+    for (int numOldMetas : numMetasToDeleteOrPut) {
+      for (int numNewMetas : numMetasToDeleteOrPut) {
+        doTestBatchWrite(numOldMetas, numNewMetas);
+      }
+    }
+  }
+
+  private void doTestBatchWrite(int numDelete, int numPut) throws IOException {
+    final String root = S3URI + "/testBatchWrite_" + numDelete + '_' + numPut;
+    final Path oldDir = new Path(root, "oldDir");
+    final Path newDir = new Path(root, "newDir");
+    LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir);
+
+    DynamoDBMetadataStore ms = getDynamoMetadataStore();
+    ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)));
+    ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)));
+
+    final List<PathMetadata> oldMetas =
+        numDelete < 0 ? null : new ArrayList<PathMetadata>(numDelete);
+    for (int i = 0; i < numDelete; i++) {
+      oldMetas.add(new PathMetadata(
+          basicFileStatus(new Path(oldDir, "child" + i), i, true)));
+    }
+    final List<PathMetadata> newMetas =
+        numPut < 0 ? null : new ArrayList<PathMetadata>(numPut);
+    for (int i = 0; i < numPut; i++) {
+      newMetas.add(new PathMetadata(
+          basicFileStatus(new Path(newDir, "child" + i), i, false)));
+    }
+
+    Collection<Path> pathsToDelete = null;
+    if (oldMetas != null) {
+      // put all metadata of old paths and verify
+      ms.put(new DirListingMetadata(oldDir, oldMetas, false));
+      assertEquals(0, ms.listChildren(newDir).withoutTombstones().numEntries());
+      assertTrue(CollectionUtils.isEqualCollection(oldMetas,
+          ms.listChildren(oldDir).getListing()));
+
+      pathsToDelete = new ArrayList<>(oldMetas.size());
+      for (PathMetadata meta : oldMetas) {
+        pathsToDelete.add(meta.getFileStatus().getPath());
+      }
+    }
+
+    // move the old paths to new paths and verify
+    ms.move(pathsToDelete, newMetas);
+    assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries());
+    if (newMetas != null) {
+      assertTrue(CollectionUtils.isEqualCollection(newMetas,
+          ms.listChildren(newDir).getListing()));
+    }
+  }
+
+  @Test
+  public void testInitExistingTable() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String tableName = ddbms.getTable().getTableName();
+    verifyTableInitialized(tableName);
+    // create existing table
+    ddbms.initTable();
+    verifyTableInitialized(tableName);
+  }
+
+  /**
+   * Test the low level version check code.
+   */
+  @Test
+  public void testItemVersionCompatibility() throws Throwable {
+    verifyVersionCompatibility("table",
+        createVersionMarker(VERSION_MARKER, VERSION, 0));
+  }
+
+  /**
+   * Test that a version marker entry without the version number field
+   * is rejected as incompatible with a meaningful error message.
+   */
+  @Test
+  public void testItemLacksVersion() throws Throwable {
+    intercept(IOException.class, E_NOT_VERSION_MARKER,
+        new VoidCallable() {
+          @Override
+          public void call() throws Exception {
+            verifyVersionCompatibility("table",
+                new Item().withPrimaryKey(
+                    createVersionMarkerPrimaryKey(VERSION_MARKER)));
+          }
+        });
+  }
+
+  /**
+   * Delete the version marker and verify that table init fails.
+   */
+  @Test
+  public void testTableVersionRequired() throws Exception {
+    Configuration conf = getFileSystem().getConf();
+    int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES,
+        S3GUARD_DDB_MAX_RETRIES_DEFAULT);
+    conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3);
+
+    final DynamoDBMetadataStore ddbms = createContract(conf).getMetadataStore();
+    String tableName = conf.get(S3GUARD_DDB_TABLE_NAME_KEY, BUCKET);
+    Table table = verifyTableInitialized(tableName);
+    table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
+
+    // create existing table
+    intercept(IOException.class, E_NO_VERSION_MARKER,
+        new VoidCallable() {
+          @Override
+          public void call() throws Exception {
+            ddbms.initTable();
+          }
+        });
+
+    conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
+  }
+
+  /**
+   * Set the version value to a different number and verify that
+   * table init fails.
+   */
+  @Test
+  public void testTableVersionMismatch() throws Exception {
+    final DynamoDBMetadataStore ddbms = createContract().getMetadataStore();
+    String tableName = getFileSystem().getConf()
+        .get(S3GUARD_DDB_TABLE_NAME_KEY, BUCKET);
+    Table table = verifyTableInitialized(tableName);
+    table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
+    Item v200 = createVersionMarker(VERSION_MARKER, 200, 0);
+    table.putItem(v200);
+
+    // create existing table
+    intercept(IOException.class, E_INCOMPATIBLE_VERSION,
+        new VoidCallable() {
+          @Override
+          public void call() throws Exception {
+            ddbms.initTable();
+          }
+        });
+  }
+
+  /**
+   * Test that initTable fails with IOException when table does not exist and
+   * table auto-creation is disabled.
+   */
+  @Test
+  public void testFailNonexistentTable() throws IOException {
+    final String tableName = "testFailNonexistentTable";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    conf.unset(S3GUARD_DDB_TABLE_CREATE_KEY);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      fail("Should have failed as table does not exist and table auto-creation"
+          + " is disabled");
+    } catch (IOException ignored) {
+    }
+  }
+
+  /**
+   * Test cases about root directory as it is not in the DynamoDB table.
+   */
+  @Test
+  public void testRootDirectory() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    Path rootPath = new Path(S3URI);
+    verifyRootDirectory(ddbms.get(rootPath), true);
+
+    ddbms.put(new PathMetadata(new S3AFileStatus(true,
+        new Path(rootPath, "foo"),
+        UserGroupInformation.getCurrentUser().getShortUserName())));
+    verifyRootDirectory(ddbms.get(new Path(S3URI)), false);
+  }
+
+  private void verifyRootDirectory(PathMetadata rootMeta, boolean isEmpty) {
+    assertNotNull(rootMeta);
+    final FileStatus status = rootMeta.getFileStatus();
+    assertNotNull(status);
+    assertTrue(status.isDirectory());
+    // UNKNOWN is always a valid option, but true / false should not contradict
+    if (isEmpty) {
+      assertNotSame("Should not be marked non-empty",
+          Tristate.FALSE,
+          rootMeta.isEmptyDirectory());
+    } else {
+      assertNotSame("Should not be marked empty",
+          Tristate.TRUE,
+          rootMeta.isEmptyDirectory());
+    }
+  }
+
+  /**
+   * Test that when moving nested paths, all its ancestors up to destination
+   * root will also be created.
+   * Here is the directory tree before move:
+   * <pre>
+   * testMovePopulateAncestors
+   * ├── a
+   * │   └── b
+   * │       └── src
+   * │           ├── dir1
+   * │           │   └── dir2
+   * │           └── file1.txt
+   * └── c
+   *     └── d
+   *         └── dest
+   *</pre>
+   * As part of rename(a/b/src, d/c/dest), S3A will enumerate the subtree at
+   * a/b/src.  This test verifies that after the move, the new subtree at
+   * 'dest' is reachable from the root (i.e. c/ and c/d exist in the table.
+   * DynamoDBMetadataStore depends on this property to do recursive delete
+   * without a full table scan.
+   */
+  @Test
+  public void testMovePopulatesAncestors() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String testRoot = "/testMovePopulatesAncestors";
+    final String srcRoot = testRoot + "/a/b/src";
+    final String destRoot = testRoot + "/c/d/e/dest";
+
+    final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)));
+    final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)));
+
+    // We don't put the destRoot path here, since put() would create ancestor
+    // entries, and we want to ensure that move() does it, instead.
+
+    // Build enumeration of src / dest paths and do the move()
+    final Collection<Path> fullSourcePaths = Lists.newArrayList(
+        strToPath(srcRoot),
+        strToPath(srcRoot + "/dir1"),
+        strToPath(srcRoot + "/dir1/dir2"),
+        strToPath(srcRoot + "/file1.txt")
+    );
+    final Collection<PathMetadata> pathsToCreate = Lists.newArrayList(
+        new PathMetadata(basicFileStatus(strToPath(destRoot),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1"),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1/dir2"),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/file1.txt"),
+            1024, false))
+    );
+
+    ddbms.move(fullSourcePaths, pathsToCreate);
+
+    // assert that all the ancestors should have been populated automatically
+    assertCached(testRoot + "/c");
+    assertCached(testRoot + "/c/d");
+    assertCached(testRoot + "/c/d/e");
+    assertCached(destRoot /* /c/d/e/dest */);
+
+    // Also check moved files while we're at it
+    assertCached(destRoot + "/dir1");
+    assertCached(destRoot + "/dir1/dir2");
+    assertCached(destRoot + "/file1.txt");
+  }
+
+  @Test
+  public void testProvisionTable() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String tableName = ddbms.getTable().getTableName();
+    final ProvisionedThroughputDescription oldProvision =
+        dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
+    ddbms.provisionTable(oldProvision.getReadCapacityUnits() * 2,
+        oldProvision.getWriteCapacityUnits() * 2);
+    final ProvisionedThroughputDescription newProvision =
+        dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
+    LOG.info("Old provision = {}, new provision = {}",
+        oldProvision, newProvision);
+    assertEquals(oldProvision.getReadCapacityUnits() * 2,
+        newProvision.getReadCapacityUnits().longValue());
+    assertEquals(oldProvision.getWriteCapacityUnits() * 2,
+        newProvision.getWriteCapacityUnits().longValue());
+  }
+
+  @Test
+  public void testDeleteTable() throws IOException {
+    final String tableName = "testDeleteTable";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      // we can list the empty table
+      ddbms.listChildren(new Path(S3URI));
+
+      ddbms.destroy();
+      verifyTableNotExist(tableName);
+
+      // delete table once more; be ResourceNotFoundException swallowed silently
+      ddbms.destroy();
+      verifyTableNotExist(tableName);
+
+      try {
+        // we can no longer list the destroyed table
+        ddbms.listChildren(new Path(S3URI));
+        fail("Should have failed after the table is destroyed!");
+      } catch (IOException ignored) {
+      }
+    }
+  }
+
+  /**
+   * This validates the table is created and ACTIVE in DynamoDB.
+   *
+   * This should not rely on the {@link DynamoDBMetadataStore} implementation.
+   * Return the table
+   */
+  private static Table verifyTableInitialized(String tableName) {
+    final Table table = dynamoDB.getTable(tableName);
+    final TableDescription td = table.describe();
+    assertEquals(tableName, td.getTableName());
+    assertEquals("ACTIVE", td.getTableStatus());
+    return table;
+  }
+
+  /**
+   * This validates the table is not found in DynamoDB.
+   *
+   * This should not rely on the {@link DynamoDBMetadataStore} implementation.
+   */
+  private static void verifyTableNotExist(String tableName) {
+    final Table table = dynamoDB.getTable(tableName);
+    try {
+      table.describe();
+      fail("Expecting ResourceNotFoundException for table '" + tableName + "'");
+    } catch (ResourceNotFoundException ignored) {
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java
new file mode 100644
index 0000000..1b765af
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java
@@ -0,0 +1,140 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
+/**
+ * MetadataStore unit test for {@link LocalMetadataStore}.
+ */
+public class TestLocalMetadataStore extends MetadataStoreTestBase {
+
+  private static final String MAX_ENTRIES_STR = "16";
+
+  private final static class LocalMSContract extends AbstractMSContract {
+
+    private FileSystem fs;
+
+    private LocalMSContract() throws IOException {
+      this(new Configuration());
+    }
+
+    private LocalMSContract(Configuration config) throws IOException {
+      config.set(LocalMetadataStore.CONF_MAX_RECORDS, MAX_ENTRIES_STR);
+      fs = FileSystem.getLocal(config);
+    }
+
+    @Override
+    public FileSystem getFileSystem() {
+      return fs;
+    }
+
+    @Override
+    public MetadataStore getMetadataStore() throws IOException {
+      LocalMetadataStore lms = new LocalMetadataStore();
+      return lms;
+    }
+  }
+
+  @Override
+  public AbstractMSContract createContract() throws IOException {
+    return new LocalMSContract();
+  }
+
+  @Override
+  public AbstractMSContract createContract(Configuration conf) throws
+      IOException {
+    return new LocalMSContract(conf);
+  }
+
+  @Test
+  public void testClearByAncestor() {
+    Map<Path, PathMetadata> map = new HashMap<>();
+
+    // 1. Test paths without scheme/host
+    assertClearResult(map, "", "/", 0);
+    assertClearResult(map, "", "/dirA/dirB", 2);
+    assertClearResult(map, "", "/invalid", 5);
+
+
+    // 2. Test paths w/ scheme/host
+    String p = "s3a://fake-bucket-name";
+    assertClearResult(map, p, "/", 0);
+    assertClearResult(map, p, "/dirA/dirB", 2);
+    assertClearResult(map, p, "/invalid", 5);
+  }
+
+  private static void populateMap(Map<Path, PathMetadata> map,
+      String prefix) {
+    populateEntry(map, new Path(prefix + "/dirA/dirB/"));
+    populateEntry(map, new Path(prefix + "/dirA/dirB/dirC"));
+    populateEntry(map, new Path(prefix + "/dirA/dirB/dirC/file1"));
+    populateEntry(map, new Path(prefix + "/dirA/dirB/dirC/file2"));
+    populateEntry(map, new Path(prefix + "/dirA/file1"));
+  }
+
+  private static void populateEntry(Map<Path, PathMetadata> map,
+      Path path) {
+    map.put(path, new PathMetadata(new FileStatus(0, true, 0, 0, 0, path)));
+  }
+
+  private static int sizeOfMap(Map<Path, PathMetadata> map) {
+    int count = 0;
+    for (PathMetadata meta : map.values()) {
+      if (!meta.isDeleted()) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  private static void assertClearResult(Map <Path, PathMetadata> map,
+      String prefixStr, String pathStr, int leftoverSize) {
+    populateMap(map, prefixStr);
+    LocalMetadataStore.deleteHashByAncestor(new Path(prefixStr + pathStr), map,
+        true);
+    assertEquals(String.format("Map should have %d entries", leftoverSize),
+        leftoverSize, sizeOfMap(map));
+    map.clear();
+  }
+
+  @Override
+  protected void verifyFileStatus(FileStatus status, long size) {
+    S3ATestUtils.verifyFileStatus(status, size, REPLICATION, getModTime(),
+        getAccessTime(),
+        BLOCK_SIZE, OWNER, GROUP, PERMISSION);
+  }
+
+  @Override
+  protected void verifyDirStatus(FileStatus status) {
+    S3ATestUtils.verifyDirStatus(status, REPLICATION, getModTime(),
+        getAccessTime(), OWNER, GROUP, PERMISSION);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java
new file mode 100644
index 0000000..c0541ea
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/**
+ * Run MetadataStore unit tests on the NullMetadataStore implementation.
+ */
+public class TestNullMetadataStore extends MetadataStoreTestBase {
+  private static class NullMSContract extends AbstractMSContract {
+    @Override
+    public FileSystem getFileSystem() throws IOException {
+      Configuration config = new Configuration();
+      return FileSystem.getLocal(config);
+    }
+
+    @Override
+    public MetadataStore getMetadataStore() throws IOException {
+      return new NullMetadataStore();
+    }
+  }
+
+  /** This MetadataStore always says "I don't know, ask the backing store". */
+  @Override
+  public boolean allowMissing() {
+    return true;
+  }
+
+  @Override
+  public AbstractMSContract createContract() {
+    return new NullMSContract();
+  }
+
+  @Override
+  public AbstractMSContract createContract(Configuration conf) {
+    return createContract();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
new file mode 100644
index 0000000..1678746
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
@@ -0,0 +1,238 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.KeyAttribute;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.google.common.base.Preconditions;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static com.amazonaws.services.dynamodbv2.model.KeyType.HASH;
+import static com.amazonaws.services.dynamodbv2.model.KeyType.RANGE;
+import static com.amazonaws.services.dynamodbv2.model.ScalarAttributeType.S;
+import static org.hamcrest.CoreMatchers.anyOf;
+import static org.hamcrest.CoreMatchers.is;
+
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION_MARKER;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION;
+
+/**
+ * Test the PathMetadataDynamoDBTranslation is able to translate between domain
+ * model objects and DynamoDB items.
+ */
+public class TestPathMetadataDynamoDBTranslation extends Assert {
+
+  private static final Path TEST_DIR_PATH = new Path("s3a://test-bucket/myDir");
+  private static final Item TEST_DIR_ITEM = new Item();
+  private static PathMetadata testDirPathMetadata;
+
+  private static final long TEST_FILE_LENGTH = 100;
+  private static final long TEST_MOD_TIME = 9999;
+  private static final long TEST_BLOCK_SIZE = 128;
+  private static final Path TEST_FILE_PATH = new Path(TEST_DIR_PATH, "myFile");
+  private static final Item TEST_FILE_ITEM = new Item();
+  private static PathMetadata testFilePathMetadata;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws IOException {
+    String username = UserGroupInformation.getCurrentUser().getShortUserName();
+
+    testDirPathMetadata =
+        new PathMetadata(new S3AFileStatus(false, TEST_DIR_PATH, username));
+    TEST_DIR_ITEM
+        .withPrimaryKey(PARENT, "/test-bucket", CHILD, TEST_DIR_PATH.getName())
+        .withBoolean(IS_DIR, true);
+
+    testFilePathMetadata = new PathMetadata(
+        new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, TEST_FILE_PATH,
+            TEST_BLOCK_SIZE, username));
+    TEST_FILE_ITEM
+        .withPrimaryKey(PARENT, pathToParentKey(TEST_FILE_PATH.getParent()),
+            CHILD, TEST_FILE_PATH.getName())
+        .withBoolean(IS_DIR, false)
+        .withLong(FILE_LENGTH, TEST_FILE_LENGTH)
+        .withLong(MOD_TIME, TEST_MOD_TIME)
+        .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE);
+  }
+
+  /**
+   * It should not take long time as it doesn't involve remote server operation.
+   */
+  @Rule
+  public final Timeout timeout = new Timeout(30 * 1000);
+
+  @Test
+  public void testKeySchema() {
+    final Collection<KeySchemaElement> keySchema =
+        PathMetadataDynamoDBTranslation.keySchema();
+    assertNotNull(keySchema);
+    assertEquals("There should be HASH and RANGE key in key schema",
+        2, keySchema.size());
+    for (KeySchemaElement element : keySchema) {
+      assertThat(element.getAttributeName(), anyOf(is(PARENT), is(CHILD)));
+      assertThat(element.getKeyType(),
+          anyOf(is(HASH.toString()), is(RANGE.toString())));
+    }
+  }
+
+  @Test
+  public void testAttributeDefinitions() {
+    final Collection<AttributeDefinition> attrs =
+        PathMetadataDynamoDBTranslation.attributeDefinitions();
+    assertNotNull(attrs);
+    assertEquals("There should be HASH and RANGE attributes", 2, attrs.size());
+    for (AttributeDefinition definition : attrs) {
+      assertThat(definition.getAttributeName(), anyOf(is(PARENT), is(CHILD)));
+      assertEquals(S.toString(), definition.getAttributeType());
+    }
+  }
+
+  @Test
+  public void testItemToPathMetadata() throws IOException {
+    final String user =
+        UserGroupInformation.getCurrentUser().getShortUserName();
+    assertNull(itemToPathMetadata(null, user));
+
+    verify(TEST_DIR_ITEM, itemToPathMetadata(TEST_DIR_ITEM, user));
+    verify(TEST_FILE_ITEM, itemToPathMetadata(TEST_FILE_ITEM, user));
+  }
+
+  /**
+   * Verify that the Item and PathMetadata objects hold the same information.
+   */
+  private static void verify(Item item, PathMetadata meta) {
+    assertNotNull(meta);
+    final FileStatus status = meta.getFileStatus();
+    final Path path = status.getPath();
+    assertEquals(item.get(PARENT), pathToParentKey(path.getParent()));
+    assertEquals(item.get(CHILD), path.getName());
+    boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR);
+    assertEquals(isDir, status.isDirectory());
+    long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0;
+    assertEquals(len, status.getLen());
+    long bSize = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0;
+    assertEquals(bSize, status.getBlockSize());
+
+    /*
+     * S3AFileStatue#getModificationTime() reports the current time, so the
+     * following assertion is failing.
+     *
+     * long modTime = item.hasAttribute(MOD_TIME) ? item.getLong(MOD_TIME) : 0;
+     * assertEquals(modTime, status.getModificationTime());
+     */
+  }
+
+  @Test
+  public void testPathMetadataToItem() {
+    verify(pathMetadataToItem(testDirPathMetadata), testDirPathMetadata);
+    verify(pathMetadataToItem(testFilePathMetadata),
+        testFilePathMetadata);
+  }
+
+  @Test
+  public void testPathToParentKeyAttribute() {
+    doTestPathToParentKeyAttribute(TEST_DIR_PATH);
+    doTestPathToParentKeyAttribute(TEST_FILE_PATH);
+  }
+
+  private static void doTestPathToParentKeyAttribute(Path path) {
+    final KeyAttribute attr = pathToParentKeyAttribute(path);
+    assertNotNull(attr);
+    assertEquals(PARENT, attr.getName());
+    // this path is expected as parent filed
+    assertEquals(pathToParentKey(path), attr.getValue());
+  }
+
+  private static String pathToParentKey(Path p) {
+    Preconditions.checkArgument(p.isUriPathAbsolute());
+    URI parentUri = p.toUri();
+    String bucket = parentUri.getHost();
+    Preconditions.checkNotNull(bucket);
+    String s =  "/" + bucket + parentUri.getPath();
+    // strip trailing slash
+    if (s.endsWith("/")) {
+      s = s.substring(0, s.length()-1);
+    }
+    return s;
+  }
+
+  @Test
+  public void testPathToKey() throws Exception {
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        new Callable<PrimaryKey>() {
+          @Override
+          public PrimaryKey call() throws Exception {
+            return pathToKey(new Path("/"));
+          }
+        });
+    doTestPathToKey(TEST_DIR_PATH);
+    doTestPathToKey(TEST_FILE_PATH);
+  }
+
+  private static void doTestPathToKey(Path path) {
+    final PrimaryKey key = pathToKey(path);
+    assertNotNull(key);
+    assertEquals("There should be both HASH and RANGE keys",
+        2, key.getComponents().size());
+
+    for (KeyAttribute keyAttribute : key.getComponents()) {
+      assertThat(keyAttribute.getName(), anyOf(is(PARENT), is(CHILD)));
+      if (PARENT.equals(keyAttribute.getName())) {
+        assertEquals(pathToParentKey(path.getParent()),
+            keyAttribute.getValue());
+      } else {
+        assertEquals(path.getName(), keyAttribute.getValue());
+      }
+    }
+  }
+
+  @Test
+  public void testVersionRoundTrip() throws Throwable {
+    final Item marker = createVersionMarker(VERSION_MARKER, VERSION, 0);
+    assertEquals("Extracted version from " + marker,
+        VERSION, extractVersionFromMarker(marker));
+  }
+
+  @Test
+  public void testVersionMarkerNotStatusIllegalPath() throws Throwable {
+    final Item marker = createVersionMarker(VERSION_MARKER, VERSION, 0);
+    assertNull("Path metadata fromfrom " + marker,
+        itemToPathMetadata(marker, "alice"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java
new file mode 100644
index 0000000..745e7aa
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.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.fs.s3a.s3guard;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Tests for the {@link S3Guard} utility class.
+ */
+public class TestS3Guard extends Assert {
+
+  /**
+   * Basic test to ensure results from S3 and MetadataStore are merged
+   * correctly.
+   */
+  @Test
+  public void testDirListingUnion() throws Exception {
+    MetadataStore ms = new LocalMetadataStore();
+
+    Path dirPath = new Path("s3a://bucket/dir");
+
+    // Two files in metadata store listing
+    PathMetadata m1 = makePathMeta("s3a://bucket/dir/ms-file1", false);
+    PathMetadata m2 = makePathMeta("s3a://bucket/dir/ms-file2", false);
+    DirListingMetadata dirMeta = new DirListingMetadata(dirPath,
+        Arrays.asList(m1, m2), false);
+
+    // Two other files in s3
+    List<FileStatus> s3Listing = Arrays.asList(
+        makeFileStatus("s3a://bucket/dir/s3-file3", false),
+        makeFileStatus("s3a://bucket/dir/s3-file4", false)
+    );
+
+    FileStatus[] result = S3Guard.dirListingUnion(ms, dirPath, s3Listing,
+        dirMeta, false);
+
+    assertEquals("listing length", 4, result.length);
+    assertContainsPath(result, "s3a://bucket/dir/ms-file1");
+    assertContainsPath(result, "s3a://bucket/dir/ms-file2");
+    assertContainsPath(result, "s3a://bucket/dir/s3-file3");
+    assertContainsPath(result, "s3a://bucket/dir/s3-file4");
+  }
+
+  void assertContainsPath(FileStatus[] statuses, String pathStr) {
+    assertTrue("listing doesn't contain " + pathStr,
+        containsPath(statuses, pathStr));
+  }
+
+  boolean containsPath(FileStatus[] statuses, String pathStr) {
+    for (FileStatus s : statuses) {
+      if (s.getPath().toString().equals(pathStr)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private PathMetadata makePathMeta(String pathStr, boolean isDir) {
+    return new PathMetadata(makeFileStatus(pathStr, isDir));
+  }
+
+  private FileStatus makeFileStatus(String pathStr, boolean isDir) {
+    Path p = new Path(pathStr);
+    if (isDir) {
+      return new FileStatus(0, true, 1, 1, System.currentTimeMillis(), p);
+    } else {
+      return new FileStatus(100, false, 1, 1, System.currentTimeMillis(), p);
+    }
+  }
+}


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


[17/48] hadoop git commit: YARN-7071. Add vcores and number of containers in new YARN UI node heat map. Contributed by Abdullah Yousufi.

Posted by in...@apache.org.
YARN-7071. Add vcores and number of containers in new YARN UI node heat map. Contributed by Abdullah Yousufi.


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

Branch: refs/heads/HDFS-10467
Commit: a756704f5aa4797ef605ff33be28fcafe2e334fd
Parents: 621b43e
Author: Sunil G <su...@apache.org>
Authored: Fri Sep 1 20:46:39 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Sep 1 20:46:39 2017 +0530

----------------------------------------------------------------------
 .../main/webapp/app/components/nodes-heatmap.js | 62 ++++++++++++++++++--
 .../src/main/webapp/app/models/yarn-rm-node.js  |  4 +-
 .../app/templates/components/nodes-heatmap.hbs  |  3 +
 .../webapp/app/templates/yarn-nodes/heatmap.hbs |  2 +-
 4 files changed, 61 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a756704f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
index 3acc5de..a1df480 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
@@ -27,6 +27,10 @@ export default BaseChartComponent.extend({
   RACK_MARGIN: 20,
   filter: "",
   selectedCategory: 0,
+  memoryLabel: "Memory",
+  cpuLabel: "VCores",
+  containersLabel: "Containers",
+  totalContainers: 0,
 
   bindTP: function(element, cell) {
     element.on("mouseover", function() {
@@ -75,8 +79,7 @@ export default BaseChartComponent.extend({
       return true;
     }
 
-    var usage = node.get("usedMemoryMB") /
-      (node.get("usedMemoryMB") + node.get("availMemoryMB"));
+    var usage = this.calcUsage(node);
     var lowerLimit = (this.selectedCategory - 1) * 0.2;
     var upperLimit = this.selectedCategory * 0.2;
     if (lowerLimit <= usage && usage <= upperLimit) {
@@ -89,6 +92,7 @@ export default BaseChartComponent.extend({
   //    [{label=label1, value=value1}, ...]
   //    ...
   renderCells: function (model, title) {
+    var selectedOption = d3.select("select").property("value");
     var data = [];
     model.forEach(function (o) {
       data.push(o);
@@ -149,6 +153,7 @@ export default BaseChartComponent.extend({
 
     var chartXOffset = -1;
 
+    this.totalContainers = 0;
     for (i = 0; i < racksArray.length; i++) {
       text = g.append("text")
         .text(racksArray[i])
@@ -166,6 +171,7 @@ export default BaseChartComponent.extend({
         var rack = data[j].get("rack");
 
         if (rack === racksArray[i]) {
+          this.totalContainers += data[j].get("numContainers");
           this.addNode(g, xOffset, yOffset, colorFunc, data[j]);
           xOffset += this.CELL_MARGIN + this.CELL_WIDTH;
           if (xOffset + this.CELL_MARGIN + this.CELL_WIDTH >= layout.x2 -
@@ -192,7 +198,7 @@ export default BaseChartComponent.extend({
 
     layout.y2 = yOffset + layout.margin;
     this.adjustMaxHeight(layout.y2);
-    this.renderTitleAndBG(g, title, layout, false);
+    this.renderTitleAndBG(g, title + selectedOption + ")" , layout, false);
   },
 
   addNode: function (g, xOffset, yOffset, colorFunc, data) {
@@ -200,10 +206,9 @@ export default BaseChartComponent.extend({
       .attr("y", yOffset)
       .attr("x", xOffset)
       .attr("height", this.CELL_HEIGHT)
-      .attr("fill", colorFunc(data.get("usedMemoryMB") /
-        (data.get("usedMemoryMB") + data.get("availMemoryMB"))))
+      .attr("fill", colorFunc(this.calcUsage(data)))
       .attr("width", this.CELL_WIDTH)
-      .attr("tooltiptext", data.get("toolTipText"));
+      .attr("tooltiptext", data.get("toolTipText") + this.getToolTipText(data));
 
     if (this.isNodeSelected(data)) {
       rect.style("opacity", 0.8);
@@ -243,6 +248,18 @@ export default BaseChartComponent.extend({
   },
 
   didInsertElement: function () {
+    var parentId = this.get("parentId");
+    var self = this;
+    var optionsData = [this.memoryLabel, this.cpuLabel, this.containersLabel];
+    d3.select("#heatmap-select")
+      .on('change', function() {
+        self.renderCells(self.get("model"), self.get("title"), self.get("textWidth"));
+      })
+      .selectAll('option')
+      .data(optionsData).enter()
+      .append('option')
+      .text(function (d) { return d; });
+
     this.draw();
   },
 
@@ -252,5 +269,38 @@ export default BaseChartComponent.extend({
       this.selectedCategory = 0;
       this.didInsertElement();
     }
+  },
+
+  calcUsage: function(data) {
+    var selectedOption = d3.select('select').property("value");
+    if (selectedOption === this.memoryLabel) {
+      return data.get("usedMemoryMB") /
+        (data.get("usedMemoryMB") + data.get("availMemoryMB"));
+    }
+    else if (selectedOption === this.cpuLabel) {
+      return data.get("usedVirtualCores") /
+        (data.get("usedVirtualCores") + data.get("availableVirtualCores"));
+    }
+    else if (selectedOption === this.containersLabel) {
+      var totalContainers = this.totalContainers;
+      if (totalContainers === 0) { return 0; }
+      return data.get("numContainers") / totalContainers;
+    }
+  },
+
+  getToolTipText: function(data) {
+    var selectedOption = d3.select('select').property("value");
+    if (selectedOption === this.memoryLabel) {
+      return "<p>Used Memory: " + Math.round(data.get("usedMemoryMB")) + " MB</p>" +
+        "<p>Available Memory: " + Math.round(data.get("availMemoryMB")) + " MB</p>";
+    }
+    else if (selectedOption === this.cpuLabel) {
+      return "<p>Used VCores: " + Math.round(data.get("usedVirtualCores")) + " VCores</p>" +
+        "<p>Available VCores: " + Math.round(data.get("availableVirtualCores")) + " VCores</p>";
+    }
+    else if (selectedOption === this.containersLabel) {
+        return "<p>Containers: " + Math.round(data.get("numContainers")) + " Containers</p>" +
+          "<p>Total Containers: " + this.totalContainers + " Containers</p>";
+    }
   }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a756704f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
index 6baeca2..20b6f5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
@@ -92,9 +92,7 @@ export default DS.Model.extend({
 
   toolTipText: function() {
     return "<p>Rack: " + this.get("rack") + '</p>' +
-           "<p>Host: " + this.get("nodeHostName") + '</p>' +
-           "<p>Used Memory: " + Math.round(this.get("usedMemoryMB")) + ' MB</p>' +
-           "<p>Available Memory: " + Math.round(this.get("availMemoryMB")) + ' MB</p>';
+           "<p>Host: " + this.get("nodeHostName") + '</p>';
   }.property(),
 
   usedMemoryBytes: function() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a756704f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/nodes-heatmap.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/nodes-heatmap.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/nodes-heatmap.hbs
index e9e6261..e7c89d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/nodes-heatmap.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/nodes-heatmap.hbs
@@ -22,6 +22,9 @@
       <input type="text" class="form-control" aria-label="..." placeholder="Enter part of host/rack to filter nodes"
              onchange={{action "applyFilter"}}>
     </div>
+    <div class="col-md-6 container-fluid">
+      <select id="heatmap-select" class="form-control"></select>
+    </div>
   </div>
 </div>
 <p/>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a756704f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-nodes/heatmap.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-nodes/heatmap.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-nodes/heatmap.hbs
index e06249f..0ebe7ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-nodes/heatmap.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-nodes/heatmap.hbs
@@ -21,7 +21,7 @@
   <div class="row">
     <div class="col-lg-12 container-fluid" id="nodes-heatmap-chart">
       {{nodes-heatmap model=model.nodes parentId="nodes-heatmap-chart"
-        title="Node Heatmap Chart (Usage of Memory)"}}
+        title="Node Heatmap Chart (Usage of "}}
     </div>
   </div>
 


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


[25/48] hadoop git commit: Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.

Posted by in...@apache.org.
Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem.


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

Branch: refs/heads/HDFS-10467
Commit: 7996eca7dcfaa1bdf970e32022274f2699bef8a1
Parents: c5281a8
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Sep 1 15:16:40 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Fri Sep 1 15:16:40 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../reservation/AbstractReservationSystem.java  |  90 ++--
 .../AbstractSchedulerPlanFollower.java          | 183 ++++----
 .../reservation/InMemoryPlan.java               | 400 ++++++++++++-----
 .../InMemoryReservationAllocation.java          |  36 +-
 .../reservation/NoOverCommitPolicy.java         |   2 +-
 .../PeriodicRLESparseResourceAllocation.java    | 130 ++++--
 .../resourcemanager/reservation/PlanEdit.java   |  24 +-
 .../resourcemanager/reservation/PlanView.java   |  94 ++--
 .../RLESparseResourceAllocation.java            | 115 ++---
 .../reservation/ReservationAllocation.java      |  60 ++-
 .../reservation/ReservationInputValidator.java  | 134 +++---
 .../reservation/ReservationSystem.java          |   6 +-
 .../reservation/SharingPolicy.java              |  13 +-
 .../reservation/planning/Planner.java           |   2 +-
 .../reservation/planning/PlanningAlgorithm.java |   2 +-
 .../reservation/planning/StageAllocator.java    |   2 +-
 .../planning/StageAllocatorGreedy.java          |   2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   5 +-
 .../planning/StageAllocatorLowCostAligned.java  |   4 +-
 .../reservation/ReservationSystemTestUtil.java  | 135 +++---
 .../reservation/TestInMemoryPlan.java           | 431 ++++++++++++-------
 ...TestPeriodicRLESparseResourceAllocation.java | 109 +++--
 .../TestRLESparseResourceAllocation.java        | 122 +++---
 .../planning/TestSimpleCapacityReplanner.java   |   8 +-
 26 files changed, 1342 insertions(+), 777 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 4944821..27ca957 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -262,6 +262,12 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
       1000L;
 
+  /** The maximum periodicity for the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      RM_PREFIX + "reservation-system.max-periodicity";
+  public static final long DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY =
+      86400000L;
+
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index bd7bf93..1d3111c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
-  @SuppressWarnings("deprecation")
+  @SuppressWarnings({"deprecation", "methodlength"})
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
@@ -69,6 +69,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare.add(YarnConfiguration
         .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
 
     // Federation default configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 5ef4912..5b8772c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -18,6 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -46,17 +57,6 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
  * This is the implementation of {@link ReservationSystem} based on the
  * {@link ResourceScheduler}
@@ -66,8 +66,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 public abstract class AbstractReservationSystem extends AbstractService
     implements ReservationSystem {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractReservationSystem.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractReservationSystem.class);
 
   // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
 
@@ -103,6 +103,8 @@ public abstract class AbstractReservationSystem extends AbstractService
 
   private boolean isRecoveryEnabled = false;
 
+  private long maxPeriodicity;
+
   /**
    * Construct the service.
    * 
@@ -143,36 +145,41 @@ public abstract class AbstractReservationSystem extends AbstractService
     this.conf = conf;
     scheduler = rmContext.getScheduler();
     // Get the plan step size
-    planStepSize =
-        conf.getTimeDuration(
-            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
-            TimeUnit.MILLISECONDS);
+    planStepSize = conf.getTimeDuration(
+        YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+        TimeUnit.MILLISECONDS);
     if (planStepSize < 0) {
       planStepSize =
           YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
     }
+    maxPeriodicity =
+        conf.getLong(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+    if (maxPeriodicity <= 0) {
+      maxPeriodicity =
+          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY;
+    }
     // Create a plan corresponding to every reservable queue
     Set<String> planQueueNames = scheduler.getPlanQueues();
     for (String planQueueName : planQueueNames) {
       Plan plan = initializePlan(planQueueName);
       plans.put(planQueueName, plan);
     }
-    isRecoveryEnabled = conf.getBoolean(
-        YarnConfiguration.RECOVERY_ENABLED,
+    isRecoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
         YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
     if (conf.getBoolean(YarnConfiguration.YARN_RESERVATION_ACL_ENABLE,
-            YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE) &&
-                    conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
-                            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
+        YarnConfiguration.DEFAULT_YARN_RESERVATION_ACL_ENABLE)
+        && conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
+            YarnConfiguration.DEFAULT_YARN_ACL_ENABLE)) {
       reservationsACLsManager = new ReservationsACLsManager(scheduler, conf);
     }
   }
 
   private void loadPlan(String planName,
       Map<ReservationId, ReservationAllocationStateProto> reservations)
-          throws PlanningException {
+      throws PlanningException {
     Plan plan = plans.get(planName);
     Resource minAllocation = getMinAllocation();
     ResourceCalculator rescCalculator = getResourceCalculator();
@@ -248,8 +255,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> planFollowerPolicyClazz =
           conf.getClassByName(planFollowerPolicyClassName);
       if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
-        return (PlanFollower) ReflectionUtils.newInstance(
-            planFollowerPolicyClazz, conf);
+        return (PlanFollower) ReflectionUtils
+            .newInstance(planFollowerPolicyClazz, conf);
       } else {
         throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
             + " not instance of " + PlanFollower.class.getCanonicalName());
@@ -257,7 +264,8 @@ public abstract class AbstractReservationSystem extends AbstractService
     } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException(
           "Could not instantiate PlanFollowerPolicy: "
-              + planFollowerPolicyClassName, e);
+              + planFollowerPolicyClassName,
+          e);
     }
   }
 
@@ -371,9 +379,8 @@ public abstract class AbstractReservationSystem extends AbstractService
   public ReservationId getNewReservationId() {
     writeLock.lock();
     try {
-      ReservationId resId =
-          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
-              resCounter.incrementAndGet());
+      ReservationId resId = ReservationId.newInstance(
+          ResourceManager.getClusterTimeStamp(), resCounter.incrementAndGet());
       LOG.info("Allocated new reservationId: " + resId);
       return resId;
     } finally {
@@ -390,8 +397,11 @@ public abstract class AbstractReservationSystem extends AbstractService
    * Get the default reservation system corresponding to the scheduler
    * 
    * @param scheduler the scheduler for which the reservation system is required
+   *
+   * @return the {@link ReservationSystem} based on the configured scheduler
    */
-  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
+  public static String getDefaultReservationSystem(
+      ResourceScheduler scheduler) {
     if (scheduler instanceof CapacityScheduler) {
       return CapacityReservationSystem.class.getName();
     } else if (scheduler instanceof FairScheduler) {
@@ -409,12 +419,11 @@ public abstract class AbstractReservationSystem extends AbstractService
     Resource maxAllocation = getMaxAllocation();
     ResourceCalculator rescCalc = getResourceCalculator();
     Resource totCap = getPlanQueueCapacity(planQueueName);
-    Plan plan =
-        new InMemoryPlan(getRootQueueMetrics(), adPolicy,
-            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
-            minAllocation, maxAllocation, planQueueName,
-            getReplanner(planQueuePath), getReservationSchedulerConfiguration()
-            .getMoveOnExpiry(planQueuePath), rmContext);
+    Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy,
+        getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation,
+        maxAllocation, planQueueName, getReplanner(planQueuePath),
+        getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath),
+        maxPeriodicity, rmContext);
     LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
@@ -477,8 +486,8 @@ public abstract class AbstractReservationSystem extends AbstractService
       Class<?> admissionPolicyClazz =
           conf.getClassByName(admissionPolicyClassName);
       if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
-        return (SharingPolicy) ReflectionUtils.newInstance(
-            admissionPolicyClazz, conf);
+        return (SharingPolicy) ReflectionUtils.newInstance(admissionPolicyClazz,
+            conf);
       } else {
         throw new YarnRuntimeException("Class: " + admissionPolicyClassName
             + " not instance of " + SharingPolicy.class.getCanonicalName());
@@ -493,8 +502,7 @@ public abstract class AbstractReservationSystem extends AbstractService
     return this.reservationsACLsManager;
   }
 
-  protected abstract ReservationSchedulerConfiguration
-      getReservationSchedulerConfiguration();
+  protected abstract ReservationSchedulerConfiguration getReservationSchedulerConfiguration();
 
   protected abstract String getPlanQueuePath(String planQueueName);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 90357e3..9b6a0b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -33,24 +41,17 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AbstractSchedulerPlanFollower.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractSchedulerPlanFollower.class);
 
   protected Collection<Plan> plans = new ArrayList<Plan>();
   protected YarnScheduler scheduler;
   protected Clock clock;
 
   @Override
-  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+  public void init(Clock clock, ResourceScheduler sched,
+      Collection<Plan> plans) {
     this.clock = clock;
     this.scheduler = sched;
     this.plans.addAll(plans);
@@ -71,7 +72,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
 
   @Override
   public synchronized void synchronizePlan(Plan plan, boolean shouldReplan) {
-     String planQueueName = plan.getQueueName();
+    String planQueueName = plan.getQueueName();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
     }
@@ -82,12 +83,14 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       now += step - (now % step);
     }
     Queue planQueue = getPlanQueue(planQueueName);
-    if (planQueue == null) return;
+    if (planQueue == null) {
+      return;
+    }
 
     // first we publish to the plan the current availability of resources
     Resource clusterResources = scheduler.getClusterResource();
-    Resource planResources = getPlanResources(plan, planQueue,
-        clusterResources);
+    Resource planResources =
+        getPlanResources(plan, planQueue, clusterResources);
     Set<ReservationAllocation> currentReservations =
         plan.getReservationsAtTime(now);
     Set<String> curReservationNames = new HashSet<String>();
@@ -95,12 +98,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     int numRes = getReservedResources(now, currentReservations,
         curReservationNames, reservedResources);
     // create the default reservation queue if it doesnt exist
-    String defReservationId = getReservationIdFromQueueName(planQueueName) +
-        ReservationConstants.DEFAULT_QUEUE_SUFFIX;
-    String defReservationQueue = getReservationQueueName(planQueueName,
-        defReservationId);
-    createDefaultReservationQueue(planQueueName, planQueue,
-        defReservationId);
+    String defReservationId = getReservationIdFromQueueName(planQueueName)
+        + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue =
+        getReservationQueueName(planQueueName, defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue, defReservationId);
     curReservationNames.add(defReservationId);
     // if the resources dedicated to this plan has shrunk invoke replanner
     boolean shouldResize = false;
@@ -149,10 +151,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
       // sort allocations from the one giving up the most resources, to the
       // one asking for the most avoid order-of-operation errors that
       // temporarily violate 100% capacity bound
-      List<ReservationAllocation> sortedAllocations =
-          sortByDelta(
-              new ArrayList<ReservationAllocation>(currentReservations), now,
-              plan);
+      List<ReservationAllocation> sortedAllocations = sortByDelta(
+          new ArrayList<ReservationAllocation>(currentReservations), now, plan);
       for (ReservationAllocation res : sortedAllocations) {
         String currResId = res.getReservationId().toString();
         if (curReservationNames.contains(currResId)) {
@@ -163,10 +163,9 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         if (planResources.getMemorySize() > 0
             && planResources.getVirtualCores() > 0) {
           if (shouldResize) {
-            capToAssign =
-                calculateReservationToPlanProportion(
-                    plan.getResourceCalculator(), planResources,
-                    reservedResources, capToAssign);
+            capToAssign = calculateReservationToPlanProportion(
+                plan.getResourceCalculator(), planResources, reservedResources,
+                capToAssign);
           }
           targetCapacity =
               calculateReservationToPlanRatio(plan.getResourceCalculator(),
@@ -185,7 +184,8 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
           maxCapacity = targetCapacity;
         }
         try {
-          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
+          setQueueEntitlement(planQueueName, currResId, targetCapacity,
+              maxCapacity);
         } catch (YarnException e) {
           LOG.warn("Exception while trying to size reservation for plan: {}",
               currResId, planQueueName, e);
@@ -196,9 +196,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     // compute the default queue capacity
     float defQCap = 1.0f - totalAssignedCapacity;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-          + "currReservation: {} default-queue capacity: {}", planResources,
-          numRes, defQCap);
+      LOG.debug(
+          "PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+              + "currReservation: {} default-queue capacity: {}",
+          planResources, numRes, defQCap);
     }
     // set the default queue to eat-up all remaining capacity
     try {
@@ -225,12 +226,11 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   protected void setQueueEntitlement(String planQueueName, String currResId,
-      float targetCapacity,
-      float maxCapacity) throws YarnException {
-    String reservationQueueName = getReservationQueueName(planQueueName,
-        currResId);
-    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
-        targetCapacity, maxCapacity));
+      float targetCapacity, float maxCapacity) throws YarnException {
+    String reservationQueueName =
+        getReservationQueueName(planQueueName, currResId);
+    scheduler.setEntitlement(reservationQueueName,
+        new QueueEntitlement(targetCapacity, maxCapacity));
   }
 
   // Schedulers have different ways of naming queues. See YARN-2773
@@ -244,14 +244,21 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Then move all apps in the set of queues to the parent plan queue's default
    * reservation queue if move is enabled. Finally cleanups the queue by killing
    * any apps (if move is disabled or move failed) and removing the queue
+   *
+   * @param planQueueName the name of {@code PlanQueue}
+   * @param shouldMove flag to indicate if any running apps should be moved or
+   *          killed
+   * @param toRemove the remnant apps to clean up
+   * @param defReservationQueue the default {@code ReservationQueue} of the
+   *          {@link Plan}
    */
-  protected void cleanupExpiredQueues(String planQueueName,
-      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
+  protected void cleanupExpiredQueues(String planQueueName, boolean shouldMove,
+      Set<String> toRemove, String defReservationQueue) {
     for (String expiredReservationId : toRemove) {
       try {
         // reduce entitlement to 0
-        String expiredReservation = getReservationQueueName(planQueueName,
-            expiredReservationId);
+        String expiredReservation =
+            getReservationQueueName(planQueueName, expiredReservationId);
         setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
         if (shouldMove) {
           moveAppsInQueueSync(expiredReservation, defReservationQueue);
@@ -275,7 +282,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * reservation queue in a synchronous fashion
    */
   private void moveAppsInQueueSync(String expiredReservation,
-                                   String defReservationQueue) {
+      String defReservationQueue) {
     List<ApplicationAttemptId> activeApps =
         scheduler.getAppsInQueue(expiredReservation);
     if (activeApps.isEmpty()) {
@@ -287,16 +294,16 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
         scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
       } catch (YarnException e) {
         LOG.warn(
-            "Encountered unexpected error during migration of application: {}" +
-                " from reservation: {}",
+            "Encountered unexpected error during migration of application: {}"
+                + " from reservation: {}",
             app, expiredReservation, e);
       }
     }
   }
 
-  protected int getReservedResources(long now, Set<ReservationAllocation>
-      currentReservations, Set<String> curReservationNames,
-                                     Resource reservedResources) {
+  protected int getReservedResources(long now,
+      Set<ReservationAllocation> currentReservations,
+      Set<String> curReservationNames, Resource reservedResources) {
     int numRes = 0;
     if (currentReservations != null) {
       numRes = currentReservations.size();
@@ -312,23 +319,30 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
    * Sort in the order from the least new amount of resources asked (likely
    * negative) to the highest. This prevents "order-of-operation" errors related
    * to exceeding 100% capacity temporarily.
+   *
+   * @param currentReservations the currently active reservations
+   * @param now the current time
+   * @param plan the {@link Plan} that is being considered
+   *
+   * @return the sorted list of {@link ReservationAllocation}s
    */
   protected List<ReservationAllocation> sortByDelta(
       List<ReservationAllocation> currentReservations, long now, Plan plan) {
-    Collections.sort(currentReservations, new ReservationAllocationComparator(
-        now, this, plan));
+    Collections.sort(currentReservations,
+        new ReservationAllocationComparator(now, this, plan));
     return currentReservations;
   }
 
   /**
-   * Get queue associated with reservable queue named
-   * @param planQueueName Name of the reservable queue
+   * Get queue associated with reservable queue named.
+   *
+   * @param planQueueName name of the reservable queue
    * @return queue associated with the reservable queue
    */
   protected abstract Queue getPlanQueue(String planQueueName);
 
   /**
-   * Resizes reservations based on currently available resources
+   * Resizes reservations based on currently available resources.
    */
   private Resource calculateReservationToPlanProportion(
       ResourceCalculator rescCalculator, Resource availablePlanResources,
@@ -338,7 +352,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Calculates ratio of reservationResources to planResources
+   * Calculates ratio of reservationResources to planResources.
    */
   private float calculateReservationToPlanRatio(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -348,7 +362,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Check if plan resources are less than expected reservation resources
+   * Check if plan resources are less than expected reservation resources.
    */
   private boolean arePlanResourcesLessThanReservations(
       ResourceCalculator rescCalculator, Resource clusterResources,
@@ -358,38 +372,56 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
   }
 
   /**
-   * Get a list of reservation queues for this planQueue
+   * Get a list of reservation queues for this planQueue.
+   *
+   * @param planQueue the queue for the current {@link Plan}
+   *
+   * @return the queues corresponding to the reservations
    */
   protected abstract List<? extends Queue> getChildReservationQueues(
       Queue planQueue);
 
   /**
-   * Add a new reservation queue for reservation currResId for this planQueue
+   * Add a new reservation queue for reservation currResId for this planQueue.
    */
-  protected abstract void addReservationQueue(
-      String planQueueName, Queue queue, String currResId);
+  protected abstract void addReservationQueue(String planQueueName, Queue queue,
+      String currResId);
 
   /**
-   * Creates the default reservation queue for use when no reservation is
-   * used for applications submitted to this planQueue
+   * Creates the default reservation queue for use when no reservation is used
+   * for applications submitted to this planQueue.
+   *
+   * @param planQueueName name of the reservable queue
+   * @param queue the queue for the current {@link Plan}
+   * @param defReservationQueue name of the default {@code ReservationQueue}
    */
-  protected abstract void createDefaultReservationQueue(
-      String planQueueName, Queue queue, String defReservationQueue);
+  protected abstract void createDefaultReservationQueue(String planQueueName,
+      Queue queue, String defReservationQueue);
 
   /**
-   * Get plan resources for this planQueue
+   * Get plan resources for this planQueue.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param clusterResources the resources available in the cluster
+   *
+   * @return the resources allocated to the specified {@link Plan}
    */
-  protected abstract Resource getPlanResources(
-      Plan plan, Queue queue, Resource clusterResources);
+  protected abstract Resource getPlanResources(Plan plan, Queue queue,
+      Resource clusterResources);
 
   /**
    * Get reservation queue resources if it exists otherwise return null.
+   *
+   * @param plan the current {@link Plan} being considered
+   * @param reservationId the identifier of the reservation
+   *
+   * @return the resources allocated to the specified reservation
    */
   protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
       ReservationId reservationId);
 
-  private static class ReservationAllocationComparator implements
-      Comparator<ReservationAllocation> {
+  private static class ReservationAllocationComparator
+      implements Comparator<ReservationAllocation> {
     AbstractSchedulerPlanFollower planFollower;
     long now;
     Plan plan;
@@ -404,14 +436,12 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     private Resource getUnallocatedReservedResources(
         ReservationAllocation reservation) {
       Resource resResource;
-      Resource reservationResource = planFollower
-          .getReservationQueueResourceIfExists
-              (plan, reservation.getReservationId());
+      Resource reservationResource =
+          planFollower.getReservationQueueResourceIfExists(plan,
+              reservation.getReservationId());
       if (reservationResource != null) {
-        resResource =
-            Resources.subtract(
-                reservation.getResourcesAtTime(now),
-                reservationResource);
+        resResource = Resources.subtract(reservation.getResourcesAtTime(now),
+            reservationResource);
       } else {
         resResource = reservation.getResourcesAtTime(now);
       }
@@ -428,4 +458,3 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index 783fd09..9eb1820 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -33,9 +33,10 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.Planner;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
@@ -64,9 +65,14 @@ public class InMemoryPlan implements Plan {
 
   private RLESparseResourceAllocation rleSparseVector;
 
+  private PeriodicRLESparseResourceAllocation periodicRle;
+
   private Map<String, RLESparseResourceAllocation> userResourceAlloc =
       new HashMap<String, RLESparseResourceAllocation>();
 
+  private Map<String, RLESparseResourceAllocation> userPeriodicResourceAlloc =
+      new HashMap<String, RLESparseResourceAllocation>();
+
   private Map<String, RLESparseResourceAllocation> userActiveReservationCount =
       new HashMap<String, RLESparseResourceAllocation>();
 
@@ -96,15 +102,27 @@ public class InMemoryPlan implements Plan {
       String queueName, Planner replanner, boolean getMoveOnExpiry,
       RMContext rmContext) {
     this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
-        maxAlloc, queueName, replanner, getMoveOnExpiry, rmContext,
-        new UTCClock());
+        maxAlloc, queueName, replanner, getMoveOnExpiry,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY,
+        rmContext);
   }
 
   public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
       ReservationAgent agent, Resource totalCapacity, long step,
       ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
       String queueName, Planner replanner, boolean getMoveOnExpiry,
-      RMContext rmContext, Clock clock) {
+      long maxPeriodicty, RMContext rmContext) {
+    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
+        maxAlloc, queueName, replanner, getMoveOnExpiry, maxPeriodicty,
+        rmContext, new UTCClock());
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
+      ReservationAgent agent, Resource totalCapacity, long step,
+      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
+      String queueName, Planner replanner, boolean getMoveOnExpiry,
+      long maxPeriodicty, RMContext rmContext, Clock clock) {
     this.queueMetrics = queueMetrics;
     this.policy = policy;
     this.agent = agent;
@@ -114,6 +132,8 @@ public class InMemoryPlan implements Plan {
     this.minAlloc = minAlloc;
     this.maxAlloc = maxAlloc;
     this.rleSparseVector = new RLESparseResourceAllocation(resCalc);
+    this.periodicRle =
+        new PeriodicRLESparseResourceAllocation(resCalc, maxPeriodicty);
     this.queueName = queueName;
     this.replanner = replanner;
     this.getMoveOnExpiry = getMoveOnExpiry;
@@ -126,6 +146,39 @@ public class InMemoryPlan implements Plan {
     return queueMetrics;
   }
 
+  private RLESparseResourceAllocation getUserRLEResourceAllocation(String user,
+      long period) {
+    RLESparseResourceAllocation resAlloc = null;
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.containsKey(user)) {
+        resAlloc = userPeriodicResourceAlloc.get(user);
+      } else {
+        resAlloc = new PeriodicRLESparseResourceAllocation(resCalc,
+            periodicRle.getTimePeriod());
+        userPeriodicResourceAlloc.put(user, resAlloc);
+      }
+    } else {
+      if (userResourceAlloc.containsKey(user)) {
+        resAlloc = userResourceAlloc.get(user);
+      } else {
+        resAlloc = new RLESparseResourceAllocation(resCalc);
+        userResourceAlloc.put(user, resAlloc);
+      }
+    }
+    return resAlloc;
+  }
+
+  private void gcUserRLEResourceAllocation(String user, long period) {
+    if (period > 0) {
+      if (userPeriodicResourceAlloc.get(user).isEmpty()) {
+        userPeriodicResourceAlloc.remove(user);
+      }
+    } else {
+      if (userResourceAlloc.get(user).isEmpty()) {
+        userResourceAlloc.remove(user);
+      }
+    }
+  }
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
@@ -133,11 +186,10 @@ public class InMemoryPlan implements Plan {
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
-    if (resAlloc == null) {
-      resAlloc = new RLESparseResourceAllocation(resCalc);
-      userResourceAlloc.put(user, resAlloc);
-    }
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
+
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
     if (resCount == null) {
       resCount = new RLESparseResourceAllocation(resCalc);
@@ -149,14 +201,43 @@ public class InMemoryPlan implements Plan {
 
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.addInterval(r.getKey(), r.getValue());
-      rleSparseVector.addInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.addInterval(newInterval, r.getValue());
+            resAlloc.addInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.addInterval(newInterval, r.getValue());
+          resAlloc.addInterval(newInterval, r.getValue());
+        }
+
+      } else {
+        rleSparseVector.addInterval(r.getKey(), r.getValue());
+        resAlloc.addInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
     resCount.addInterval(new ReservationInterval(earliestActive, latestActive),
         Resource.newInstance(1, 1));
   }
@@ -166,27 +247,55 @@ public class InMemoryPlan implements Plan {
     Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
-    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    long period = reservation.getPeriodicity();
+    RLESparseResourceAllocation resAlloc =
+        getUserRLEResourceAllocation(user, period);
 
     long earliestActive = Long.MAX_VALUE;
     long latestActive = Long.MIN_VALUE;
     for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
-      resAlloc.removeInterval(r.getKey(), r.getValue());
-      rleSparseVector.removeInterval(r.getKey(), r.getValue());
-      if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
-          ZERO_RESOURCE)) {
-        earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
-        latestActive = Math.max(latestActive, r.getKey().getEndTime());
+      if (period > 0L) {
+        for (int i = 0; i < periodicRle.getTimePeriod() / period; i++) {
+
+          long rStart = r.getKey().getStartTime() + i * period;
+          long rEnd = r.getKey().getEndTime() + i * period;
+
+          // handle wrap-around
+          if (rEnd > periodicRle.getTimePeriod()) {
+            long diff = rEnd - periodicRle.getTimePeriod();
+            rEnd = periodicRle.getTimePeriod();
+            ReservationInterval newInterval = new ReservationInterval(0, diff);
+            periodicRle.removeInterval(newInterval, r.getValue());
+            resAlloc.removeInterval(newInterval, r.getValue());
+          }
+
+          ReservationInterval newInterval =
+              new ReservationInterval(rStart, rEnd);
+          periodicRle.removeInterval(newInterval, r.getValue());
+          resAlloc.removeInterval(newInterval, r.getValue());
+        }
+      } else {
+        rleSparseVector.removeInterval(r.getKey(), r.getValue());
+        resAlloc.removeInterval(r.getKey(), r.getValue());
+        if (Resources.greaterThan(resCalc, totalCapacity, r.getValue(),
+            ZERO_RESOURCE)) {
+          earliestActive = Math.min(earliestActive, r.getKey().getStartTime());
+          latestActive = Math.max(latestActive, r.getKey().getEndTime());
+        }
       }
     }
-    if (resAlloc.isEmpty()) {
-      userResourceAlloc.remove(user);
-    }
+    gcUserRLEResourceAllocation(user, period);
 
     RLESparseResourceAllocation resCount = userActiveReservationCount.get(user);
-    resCount.removeInterval(new ReservationInterval(earliestActive,
-        latestActive), Resource.newInstance(1, 1));
+    // periodic reservations are active from start time and good till cancelled
+    if (period > 0L) {
+      earliestActive = reservation.getStartTime();
+      latestActive = Long.MAX_VALUE;
+    }
+    resCount.removeInterval(
+        new ReservationInterval(earliestActive, latestActive),
+        Resource.newInstance(1, 1));
     if (resCount.isEmpty()) {
       userActiveReservationCount.remove(user);
     }
@@ -198,9 +307,9 @@ public class InMemoryPlan implements Plan {
       if (currentReservations != null) {
         Set<ReservationAllocation> flattenedReservations =
             new TreeSet<ReservationAllocation>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-            currentReservations.values()) {
-          flattenedReservations.addAll(reservationEntries);
+        for (Set<InMemoryReservationAllocation> res : currentReservations
+            .values()) {
+          flattenedReservations.addAll(res);
         }
         return flattenedReservations;
       } else {
@@ -218,19 +327,16 @@ public class InMemoryPlan implements Plan {
     InMemoryReservationAllocation inMemReservation =
         (InMemoryReservationAllocation) reservation;
     if (inMemReservation.getUser() == null) {
-      String errMsg =
-          "The specified Reservation with ID "
-              + inMemReservation.getReservationId()
-              + " is not mapped to any user";
+      String errMsg = "The specified Reservation with ID "
+          + inMemReservation.getReservationId() + " is not mapped to any user";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     writeLock.lock();
     try {
       if (reservationTable.containsKey(inMemReservation.getReservationId())) {
-        String errMsg =
-            "The specified Reservation with ID "
-                + inMemReservation.getReservationId() + " already exists";
+        String errMsg = "The specified Reservation with ID "
+            + inMemReservation.getReservationId() + " already exists";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -246,9 +352,8 @@ public class InMemoryPlan implements Plan {
               getQueueName(), inMemReservation.getReservationId().toString());
         }
       }
-      ReservationInterval searchInterval =
-          new ReservationInterval(inMemReservation.getStartTime(),
-              inMemReservation.getEndTime());
+      ReservationInterval searchInterval = new ReservationInterval(
+          inMemReservation.getStartTime(), inMemReservation.getEndTime());
       Set<InMemoryReservationAllocation> reservations =
           currentReservations.get(searchInterval);
       if (reservations == null) {
@@ -280,9 +385,8 @@ public class InMemoryPlan implements Plan {
       ReservationId resId = reservation.getReservationId();
       ReservationAllocation currReservation = getReservationById(resId);
       if (currReservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + resId
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + resId
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -318,9 +422,8 @@ public class InMemoryPlan implements Plan {
 
   private boolean removeReservation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    ReservationInterval searchInterval =
-        new ReservationInterval(reservation.getStartTime(),
-            reservation.getEndTime());
+    ReservationInterval searchInterval = new ReservationInterval(
+        reservation.getStartTime(), reservation.getEndTime());
     Set<InMemoryReservationAllocation> reservations =
         currentReservations.get(searchInterval);
     if (reservations != null) {
@@ -337,16 +440,15 @@ public class InMemoryPlan implements Plan {
         currentReservations.remove(searchInterval);
       }
     } else {
-      String errMsg =
-          "The specified Reservation with ID " + reservation.getReservationId()
-              + " does not exist in the plan";
+      String errMsg = "The specified Reservation with ID "
+          + reservation.getReservationId() + " does not exist in the plan";
       LOG.error(errMsg);
       throw new IllegalArgumentException(errMsg);
     }
     reservationTable.remove(reservation.getReservationId());
     decrementAllocation(reservation);
     LOG.info("Sucessfully deleted reservation: {} in plan.",
-            reservation.getReservationId());
+        reservation.getReservationId());
     return true;
   }
 
@@ -356,9 +458,8 @@ public class InMemoryPlan implements Plan {
     try {
       ReservationAllocation reservation = getReservationById(reservationID);
       if (reservation == null) {
-        String errMsg =
-            "The specified Reservation with ID " + reservationID
-                + " does not exist in the plan";
+        String errMsg = "The specified Reservation with ID " + reservationID
+            + " does not exist in the plan";
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
@@ -453,66 +554,90 @@ public class InMemoryPlan implements Plan {
       long start, long end) {
     readLock.lock();
     try {
+      // merge periodic and non-periodic allocations
       RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
+      RLESparseResourceAllocation userPeriodicResAlloc =
+          userPeriodicResourceAlloc.get(user);
 
+      if (userResAlloc != null && userPeriodicResAlloc != null) {
+        return RLESparseResourceAllocation.merge(resCalc, totalCapacity,
+            userResAlloc, userPeriodicResAlloc, RLEOperator.add, start, end);
+      }
       if (userResAlloc != null) {
         return userResAlloc.getRangeOverlapping(start, end);
-      } else {
-        return new RLESparseResourceAllocation(resCalc);
       }
+      if (userPeriodicResAlloc != null) {
+        return userPeriodicResAlloc.getRangeOverlapping(start, end);
+      }
+    } catch (PlanningException e) {
+      LOG.warn("Exception while trying to merge periodic"
+          + " and non-periodic user allocations: {}", e.getMessage(), e);
     } finally {
       readLock.unlock();
     }
+    return new RLESparseResourceAllocation(resCalc);
   }
 
   @Override
   public Resource getTotalCommittedResources(long t) {
     readLock.lock();
     try {
-      return rleSparseVector.getCapacityAtTime(t);
+      return Resources.add(rleSparseVector.getCapacityAtTime(t),
+          periodicRle.getCapacityAtTime(t));
     } finally {
       readLock.unlock();
     }
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval) {
     return getReservations(reservationID, interval, null);
   }
 
   @Override
-  public Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user) {
+  public Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user) {
     if (reservationID != null) {
       ReservationAllocation allocation = getReservationById(reservationID);
-      if (allocation == null){
+      if (allocation == null) {
         return Collections.emptySet();
       }
       return Collections.singleton(allocation);
     }
 
-    long startTime = interval == null? 0 : interval.getStartTime();
-    long endTime = interval == null? Long.MAX_VALUE : interval.getEndTime();
+    long startTime = interval == null ? 0 : interval.getStartTime();
+    long endTime = interval == null ? Long.MAX_VALUE : interval.getEndTime();
 
     ReservationInterval searchInterval =
-            new ReservationInterval(endTime, Long.MAX_VALUE);
+        new ReservationInterval(endTime, Long.MAX_VALUE);
     readLock.lock();
     try {
-      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>>
-            reservations = currentReservations.headMap(searchInterval, true);
-      if (!reservations.isEmpty()) {
-        Set<ReservationAllocation> flattenedReservations =
-                new HashSet<>();
-        for (Set<InMemoryReservationAllocation> reservationEntries :
-                reservations.values()) {
-          for (InMemoryReservationAllocation res : reservationEntries) {
-            if (res.getEndTime() > startTime) {
-              if (user != null && !user.isEmpty()
-                      && !res.getUser().equals(user)) {
-                continue;
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> res =
+          currentReservations.headMap(searchInterval, true);
+      if (!res.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations = new HashSet<>();
+        for (Set<InMemoryReservationAllocation> resEntries : res.values()) {
+          for (InMemoryReservationAllocation reservation : resEntries) {
+            // validate user
+            if (user != null && !user.isEmpty()
+                && !reservation.getUser().equals(user)) {
+              continue;
+            }
+            // handle periodic reservations
+            long period = reservation.getPeriodicity();
+            if (period > 0) {
+              long t = endTime % period;
+              // check for both contained and wrap-around reservations
+              if ((t - startTime) * (t - endTime)
+                  * (startTime - endTime) >= 0) {
+                flattenedReservations.add(reservation);
+              }
+            } else {
+              // check for non-periodic reservations
+              if (reservation.getEndTime() > startTime) {
+                flattenedReservations.add(reservation);
               }
-              flattenedReservations.add(res);
             }
           }
         }
@@ -550,36 +675,82 @@ public class InMemoryPlan implements Plan {
 
   @Override
   public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException {
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException {
     readLock.lock();
     try {
-      // create RLE of totCapacity
-      TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
-      totAvailable.put(start, Resources.clone(totalCapacity));
-      RLESparseResourceAllocation totRLEAvail =
-          new RLESparseResourceAllocation(totAvailable, resCalc);
-
-      // subtract used from available
-      RLESparseResourceAllocation netAvailable;
-
-      netAvailable =
-          RLESparseResourceAllocation.merge(resCalc,
-              Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
-              RLEOperator.subtractTestNonNegative, start, end);
-
-      // add back in old reservation used resources if any
-      ReservationAllocation old = reservationTable.get(oldId);
-      if (old != null) {
-        netAvailable =
-            RLESparseResourceAllocation.merge(resCalc,
-                Resources.clone(totalCapacity), netAvailable,
-                old.getResourcesOverTime(), RLEOperator.add, start, end);
+
+      // for non-periodic return simple available resources
+      if (period == 0) {
+
+        // create RLE of totCapacity
+        TreeMap<Long, Resource> totAvailable = new TreeMap<Long, Resource>();
+        totAvailable.put(start, Resources.clone(totalCapacity));
+        RLESparseResourceAllocation totRLEAvail =
+            new RLESparseResourceAllocation(totAvailable, resCalc);
+
+        // subtract used from available
+        RLESparseResourceAllocation netAvailable;
+
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), totRLEAvail, rleSparseVector,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // remove periodic component
+        netAvailable = RLESparseResourceAllocation.merge(resCalc,
+            Resources.clone(totalCapacity), netAvailable, periodicRle,
+            RLEOperator.subtractTestNonNegative, start, end);
+
+        // add back in old reservation used resources if any
+        ReservationAllocation old = reservationTable.get(oldId);
+        if (old != null) {
+
+          RLESparseResourceAllocation addBackPrevious =
+              old.getResourcesOverTime(start, end);
+          netAvailable = RLESparseResourceAllocation.merge(resCalc,
+              Resources.clone(totalCapacity), netAvailable, addBackPrevious,
+              RLEOperator.add, start, end);
+        }
+        // lower it if this is needed by the sharing policy
+        netAvailable = getSharingPolicy().availableResources(netAvailable, this,
+            user, oldId, start, end);
+        return netAvailable;
+      } else {
+
+        if (periodicRle.getTimePeriod() % period != 0) {
+          throw new PlanningException("The reservation periodicity (" + period
+              + ") must be" + "an exact divider of the system maxPeriod ("
+              + periodicRle.getTimePeriod() + ")");
+        }
+
+        // find the minimum resources available among all the instances that fit
+        // in the LCM
+        long numInstInLCM = periodicRle.getTimePeriod() / period;
+
+        RLESparseResourceAllocation minOverLCM =
+            getAvailableResourceOverTime(user, oldId, start, end, 0);
+        for (int i = 1; i < numInstInLCM; i++) {
+
+          long rStart = start + i * period;
+          long rEnd = end + i * period;
+
+          // recursive invocation of non-periodic range (to pick raw-info)
+          RLESparseResourceAllocation snapShot =
+              getAvailableResourceOverTime(user, oldId, rStart, rEnd, 0);
+
+          // time-align on start
+          snapShot.shift(-(i * period));
+
+          // pick the minimum amount of resources in each time interval
+          minOverLCM =
+              RLESparseResourceAllocation.merge(resCalc, getTotalCapacity(),
+                  minOverLCM, snapShot, RLEOperator.min, start, end);
+
+        }
+
+        return minOverLCM;
+
       }
-      // lower it if this is needed by the sharing policy
-      netAvailable =
-          getSharingPolicy().availableResources(netAvailable, this, user,
-              oldId, start, end);
-      return netAvailable;
     } finally {
       readLock.unlock();
     }
@@ -637,7 +808,7 @@ public class InMemoryPlan implements Plan {
   public String toCumulativeString() {
     readLock.lock();
     try {
-      return rleSparseVector.toString();
+      return rleSparseVector.toString() + "\n" + periodicRle.toString();
     } finally {
       readLock.unlock();
     }
@@ -689,11 +860,18 @@ public class InMemoryPlan implements Plan {
   }
 
   @Override
-  public RLESparseResourceAllocation getCumulativeLoadOverTime(
-      long start, long end) {
+  public RLESparseResourceAllocation getCumulativeLoadOverTime(long start,
+      long end) throws PlanningException {
     readLock.lock();
     try {
-      return rleSparseVector.getRangeOverlapping(start, end);
+
+      RLESparseResourceAllocation ret =
+          rleSparseVector.getRangeOverlapping(start, end);
+      ret = RLESparseResourceAllocation.merge(resCalc, totalCapacity, ret,
+          periodicRle.getRangeOverlapping(start, end), RLEOperator.add, start,
+          end);
+
+      return ret;
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index 69fd43f..00c8e44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -42,6 +42,7 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
+  private long periodicity = 0;
 
   private RLESparseResourceAllocation resourcesOverTime;
 
@@ -67,9 +68,16 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
     this.allocationRequests = allocations;
     this.planName = planName;
     this.hasGang = hasGang;
-    resourcesOverTime = new RLESparseResourceAllocation(calculator);
-    for (Map.Entry<ReservationInterval, Resource> r : allocations
-        .entrySet()) {
+    if (contract != null && contract.getRecurrenceExpression() != null) {
+      this.periodicity = Long.parseLong(contract.getRecurrenceExpression());
+    }
+    if (periodicity > 0) {
+      resourcesOverTime =
+          new PeriodicRLESparseResourceAllocation(calculator, periodicity);
+    } else {
+      resourcesOverTime = new RLESparseResourceAllocation(calculator);
+    }
+    for (Map.Entry<ReservationInterval, Resource> r : allocations.entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
     }
   }
@@ -133,17 +141,33 @@ public class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public RLESparseResourceAllocation getResourcesOverTime(){
+  public RLESparseResourceAllocation getResourcesOverTime() {
     return resourcesOverTime;
   }
 
   @Override
+  public RLESparseResourceAllocation getResourcesOverTime(long start,
+      long end) {
+    return resourcesOverTime.getRangeOverlapping(start, end);
+  }
+
+  @Override
+  public long getPeriodicity() {
+    return periodicity;
+  }
+
+  @Override
+  public void setPeriodicity(long period) {
+    periodicity = period;
+  }
+
+  @Override
   public String toString() {
     StringBuilder sBuf = new StringBuilder();
     sBuf.append(getReservationId()).append(" user:").append(getUser())
         .append(" startTime: ").append(getStartTime()).append(" endTime: ")
-        .append(getEndTime()).append(" alloc:\n[")
-        .append(resourcesOverTime.toString()).append("] ");
+        .append(getEndTime()).append(" Periodiciy: ").append(periodicity)
+        .append(" alloc:\n[").append(resourcesOverTime.toString()).append("] ");
     return sBuf.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 55f1d00..49d4702 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -40,7 +40,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
 
     RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
         reservation.getUser(), reservation.getReservationId(),
-        reservation.getStartTime(), reservation.getEndTime());
+        reservation.getStartTime(), reservation.getEndTime(), 0);
 
     // test the reservation does not exceed what is available
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 8e3be8b3..7bc44f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -18,47 +18,94 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
- * This data structure stores a periodic RLESparseResourceAllocation.
+ * This data structure stores a periodic {@link RLESparseResourceAllocation}.
  * Default period is 1 day (86400000ms).
  */
-public class PeriodicRLESparseResourceAllocation extends
-    RLESparseResourceAllocation {
+public class PeriodicRLESparseResourceAllocation
+    extends RLESparseResourceAllocation {
 
   // Log
-  private static final Logger LOG = LoggerFactory
-      .getLogger(PeriodicRLESparseResourceAllocation.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicRLESparseResourceAllocation.class);
 
   private long timePeriod;
 
   /**
    * Constructor.
    *
-   * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use.
    * @param timePeriod Time period in milliseconds.
    */
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector, Long timePeriod) {
-    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+      ResourceCalculator resourceCalculator, Long timePeriod) {
+    super(resourceCalculator);
     this.timePeriod = timePeriod;
   }
 
   /**
    * Constructor. Default time period set to 1 day.
    *
+   * @param resourceCalculator {@link ResourceCalculator} the resource
+   *          calculator to use..
+   */
+  public PeriodicRLESparseResourceAllocation(
+      ResourceCalculator resourceCalculator) {
+    this(resourceCalculator,
+        YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_MAX_PERIODICITY);
+  }
+
+  /**
+   * Constructor.
+   *
    * @param rleVector {@link RLESparseResourceAllocation} with the run-length
-              encoded data.
+   *          encoded data.
+   * @param timePeriod Time period in milliseconds.
    */
+  @VisibleForTesting
   public PeriodicRLESparseResourceAllocation(
-      RLESparseResourceAllocation rleVector) {
-    this(rleVector, 86400000L);
+      RLESparseResourceAllocation rleVector, Long timePeriod) {
+    super(rleVector.getCumulative(), rleVector.getResourceCalculator());
+    this.timePeriod = timePeriod;
+
+    // make sure the PeriodicRLE is zero-based, and handles wrap-around
+    long delta = (getEarliestStartTime() % timePeriod - getEarliestStartTime());
+    shift(delta);
+
+    List<Long> toRemove = new ArrayList<>();
+    Map<Long, Resource> toAdd = new TreeMap<>();
+
+    for (Map.Entry<Long, Resource> entry : cumulativeCapacity.entrySet()) {
+      if (entry.getKey() > timePeriod) {
+        toRemove.add(entry.getKey());
+        if (entry.getValue() != null) {
+          toAdd.put(timePeriod, entry.getValue());
+          long prev = entry.getKey() % timePeriod;
+          toAdd.put(prev, this.getCapacityAtTime(prev));
+          toAdd.put(0L, entry.getValue());
+        }
+      }
+    }
+    for (Long l : toRemove) {
+      cumulativeCapacity.remove(l);
+    }
+    cumulativeCapacity.putAll(toAdd);
   }
 
   /**
@@ -78,24 +125,25 @@ public class PeriodicRLESparseResourceAllocation extends
    * The interval may include 0, but the end time must be strictly less than
    * timePeriod.
    *
-   * @param interval {@link ReservationInterval} to which the specified
-   *          resource is to be added.
+   * @param interval {@link ReservationInterval} to which the specified resource
+   *          is to be added.
    * @param resource {@link Resource} to be added to the interval specified.
    * @return true if addition is successful, false otherwise
    */
-  public boolean addInterval(ReservationInterval interval,
-      Resource resource) {
+  public boolean addInterval(ReservationInterval interval, Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
+
     if (startTime >= 0 && endTime > startTime && endTime <= timePeriod) {
       return super.addInterval(interval, resource);
     } else {
-      LOG.info("Cannot set capacity beyond end time: " + timePeriod);
+      LOG.info("Cannot set capacity beyond end time: " + timePeriod + " was ("
+          + interval.toString() + ")");
       return false;
     }
   }
 
-   /**
+  /**
    * Removes a resource for the specified interval.
    *
    * @param interval the {@link ReservationInterval} for which the resource is
@@ -103,14 +151,15 @@ public class PeriodicRLESparseResourceAllocation extends
    * @param resource the {@link Resource} to be removed.
    * @return true if removal is successful, false otherwise
    */
-  public boolean removeInterval(
-      ReservationInterval interval, Resource resource) {
+  public boolean removeInterval(ReservationInterval interval,
+      Resource resource) {
     long startTime = interval.getStartTime();
     long endTime = interval.getEndTime();
     // If the resource to be subtracted is less than the minimum resource in
     // the range, abort removal to avoid negative capacity.
-    if (!Resources.fitsIn(
-        resource, super.getMinimumCapacityInInterval(interval))) {
+    // TODO revesit decrementing endTime
+    if (!Resources.fitsIn(resource, getMinimumCapacityInInterval(
+        new ReservationInterval(startTime, endTime - 1)))) {
       LOG.info("Request to remove more resources than what is available");
       return false;
     }
@@ -125,17 +174,16 @@ public class PeriodicRLESparseResourceAllocation extends
   /**
    * Get maximum capacity at periodic offsets from the specified time.
    *
-   * @param tick UTC time base from which offsets are specified for finding
-   *          the maximum capacity.
-   * @param period periodic offset at which capacities are evaluted.
+   * @param tick UTC time base from which offsets are specified for finding the
+   *          maximum capacity.
+   * @param period periodic offset at which capacities are evaluated.
    * @return the maximum {@link Resource} across the specified time instants.
    * @return true if removal is successful, false otherwise
    */
   public Resource getMaximumPeriodicCapacity(long tick, long period) {
     Resource maxResource;
     if (period < timePeriod) {
-      maxResource =
-          super.getMaximumPeriodicCapacity(tick % timePeriod, period);
+      maxResource = super.getMaximumPeriodicCapacity(tick % timePeriod, period);
     } else {
       // if period is greater than the length of PeriodicRLESparseAllocation,
       // only a single value exists in this interval.
@@ -164,4 +212,30 @@ public class PeriodicRLESparseResourceAllocation extends
     return ret.toString();
   }
 
+  @Override
+  public RLESparseResourceAllocation getRangeOverlapping(long start, long end) {
+    NavigableMap<Long, Resource> unrolledMap = new TreeMap<>();
+    readLock.lock();
+    try {
+      long relativeStart = (start >= 0) ? start % timePeriod : 0;
+      NavigableMap<Long, Resource> cumulativeMap = this.getCumulative();
+      Long previous = cumulativeMap.floorKey(relativeStart);
+      previous = (previous != null) ? previous : 0;
+      for (long i = 0; i <= (end - start) / timePeriod; i++) {
+        for (Map.Entry<Long, Resource> e : cumulativeMap.entrySet()) {
+          long curKey = e.getKey() + (i * timePeriod);
+          if (curKey >= previous && (start + curKey - relativeStart) <= end) {
+            unrolledMap.put(curKey, e.getValue());
+          }
+        }
+      }
+      RLESparseResourceAllocation rle =
+          new RLESparseResourceAllocation(unrolledMap, getResourceCalculator());
+      rle.shift(start - relativeStart);
+      return rle;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 504a250..9afa324 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -28,54 +28,58 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.Plan
 public interface PlanEdit extends PlanContext, PlanView {
 
   /**
-   * Add a new {@link ReservationAllocation} to the plan
+   * Add a new {@link ReservationAllocation} to the plan.
    * 
    * @param reservation the {@link ReservationAllocation} to be added to the
    *          plan
    * @param isRecovering flag to indicate if reservation is being added as part
    *          of failover or not
    * @return true if addition is successful, false otherwise
+   * @throws PlanningException if addition is unsuccessful
    */
-  public boolean addReservation(ReservationAllocation reservation,
+  boolean addReservation(ReservationAllocation reservation,
       boolean isRecovering) throws PlanningException;
 
   /**
    * Updates an existing {@link ReservationAllocation} in the plan. This is
-   * required for re-negotiation
+   * required for re-negotiation.
    * 
    * @param reservation the {@link ReservationAllocation} to be updated the plan
    * @return true if update is successful, false otherwise
+   * @throws PlanningException if update is unsuccessful
    */
-  public boolean updateReservation(ReservationAllocation reservation)
+  boolean updateReservation(ReservationAllocation reservation)
       throws PlanningException;
 
   /**
    * Delete an existing {@link ReservationAllocation} from the plan identified
    * uniquely by its {@link ReservationId}. This will generally be used for
-   * garbage collection
+   * garbage collection.
    * 
    * @param reservationID the {@link ReservationAllocation} to be deleted from
    *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
+   * @throws PlanningException if deletion is unsuccessful
    */
-  public boolean deleteReservation(ReservationId reservationID)
+  boolean deleteReservation(ReservationId reservationID)
       throws PlanningException;
 
   /**
    * Method invoked to garbage collect old reservations. It cleans up expired
-   * reservations that have fallen out of the sliding archival window
+   * reservations that have fallen out of the sliding archival window.
    * 
    * @param tick the current time from which the archival window is computed
+   * @throws PlanningException if archival is unsuccessful
    */
-  public void archiveCompletedReservations(long tick) throws PlanningException;
+  void archiveCompletedReservations(long tick) throws PlanningException;
 
   /**
    * Sets the overall capacity in terms of {@link Resource} assigned to this
-   * plan
+   * plan.
    * 
    * @param capacity the overall capacity in terms of {@link Resource} assigned
    *          to this plan
    */
-  public void setTotalCapacity(Resource capacity);
+  void setTotalCapacity(Resource capacity);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7996eca7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 2767993..4035f68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -17,50 +17,50 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import java.util.Set;
+
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 
-import java.util.Set;
-
 /**
  * This interface provides a read-only view on the allocations made in this
  * plan. This methods are used for example by {@code ReservationAgent}s to
  * determine the free resources in a certain point in time, and by
  * PlanFollowerPolicy to publish this plan to the scheduler.
  */
-public interface PlanView extends PlanContext {
+interface PlanView extends PlanContext {
 
   /**
    * Return a set of {@link ReservationAllocation} identified by the user who
    * made the reservation.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @param user the user to retrieve the reservation allocation from.
    * @return a set of {@link ReservationAllocation} identified by the user who
-   * made the reservation
+   *         made the reservation
    */
-  Set<ReservationAllocation> getReservations(ReservationId
-                    reservationID, ReservationInterval interval, String user);
+  Set<ReservationAllocation> getReservations(ReservationId reservationID,
+      ReservationInterval interval, String user);
 
   /**
    * Return a set of {@link ReservationAllocation} identified by any user.
    *
    * @param reservationID the unqiue id to identify the
-   * {@link ReservationAllocation}
+   *          {@link ReservationAllocation}
    * @param interval the time interval used to retrieve the reservation
-   *                 allocations from. Only reservations with start time no
-   *                 greater than the interval end time, and end time no less
-   *                 than the interval start time will be selected.
+   *          allocations from. Only reservations with start time no greater
+   *          than the interval end time, and end time no less than the interval
+   *          start time will be selected.
    * @return a set of {@link ReservationAllocation} identified by any user
    */
   Set<ReservationAllocation> getReservations(ReservationId reservationID,
-                    ReservationInterval interval);
+      ReservationInterval interval);
 
   /**
    * Return a {@link ReservationAllocation} identified by its
@@ -70,7 +70,7 @@ public interface PlanView extends PlanContext {
    *          {@link ReservationAllocation}
    * @return {@link ReservationAllocation} identified by the specified id
    */
-  public ReservationAllocation getReservationById(ReservationId reservationID);
+  ReservationAllocation getReservationById(ReservationId reservationID);
 
   /**
    * Return a set of {@link ReservationAllocation} that belongs to a certain
@@ -78,11 +78,10 @@ public interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return set of active {@link ReservationAllocation}s for this
-   *         user at this time
+   * @return set of active {@link ReservationAllocation}s for this user at this
+   *         time
    */
-  public Set<ReservationAllocation> getReservationByUserAtTime(String user,
-      long t);
+  Set<ReservationAllocation> getReservationByUserAtTime(String user, long t);
 
   /**
    * Gets all the active reservations at the specified point of time
@@ -91,14 +90,14 @@ public interface PlanView extends PlanContext {
    *          requested
    * @return set of active reservations at the specified time
    */
-  public Set<ReservationAllocation> getReservationsAtTime(long tick);
+  Set<ReservationAllocation> getReservationsAtTime(long tick);
 
   /**
    * Gets all the reservations in the plan
    * 
    * @return set of all reservations handled by this Plan
    */
-  public Set<ReservationAllocation> getAllReservations();
+  Set<ReservationAllocation> getAllReservations();
 
   /**
    * Returns the total {@link Resource} reserved for all users at the specified
@@ -126,61 +125,68 @@ public interface PlanView extends PlanContext {
    * 
    * @return the time (UTC in ms) at which the first reservation starts
    */
-  public long getEarliestStartTime();
+  long getEarliestStartTime();
 
   /**
    * Returns the time (UTC in ms) at which the last reservation terminates
    *
    * @return the time (UTC in ms) at which the last reservation terminates
    */
-  public long getLastEndTime();
+  long getLastEndTime();
 
   /**
    * This method returns the amount of resources available to a given user
    * (optionally if removing a certain reservation) over the start-end time
-   * range.
+   * range. If the request is periodic (period is non-zero) we return the
+   * minimum amount of resources available to periodic reservations (in all
+   * "period" windows within the system maxPeriod / LCM).
    *
-   * @param user
-   * @param oldId
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param oldId the identifier of the existing reservation
+   * @param start start of the time interval.
+   * @param end end of the time interval.
+   * @param period the ms periodicty for this request (loop and pick min till
+   *          maxPeriodicity)
    * @return a view of the plan as it is available to this user
-   * @throws PlanningException
+   * @throws PlanningException if operation is unsuccessful
    */
-  public RLESparseResourceAllocation getAvailableResourceOverTime(String user,
-      ReservationId oldId, long start, long end) throws PlanningException;
+  RLESparseResourceAllocation getAvailableResourceOverTime(String user,
+      ReservationId oldId, long start, long end, long period)
+      throws PlanningException;
 
   /**
    * This method returns a RLE encoded view of the user reservation count
    * utilization between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of reservation used over time
    */
-  public RLESparseResourceAllocation getReservationCountForUserOverTime(
-      String user, long start, long end);
+  RLESparseResourceAllocation getReservationCountForUserOverTime(String user,
+      long start, long end);
 
   /**
    * This method returns a RLE encoded view of the user reservation utilization
    * between start and end time.
    *
-   * @param user
-   * @param start
-   * @param end
+   * @param user the user being considered
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE encoded view of resources used over time
    */
-  public RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
+  RLESparseResourceAllocation getConsumptionForUserOverTime(String user,
       long start, long end);
 
   /**
    * Get the cumulative load over a time interval.
    *
-   * @param start Start of the time interval.
-   * @param end End of the time interval.
+   * @param start start of the time interval.
+   * @param end end of the time interval.
    * @return RLE sparse allocation.
+   * @throws PlanningException if operation is unsuccessful
    */
-  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end);
+  RLESparseResourceAllocation getCumulativeLoadOverTime(long start, long end)
+      throws PlanningException;
 
 }


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


[16/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
HADOOP-13345 HS3Guard: Improved Consistency for S3A.
Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu,
Sean Mackrory, Steve Loughran and others.


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

Branch: refs/heads/HDFS-10467
Commit: 621b43e254afaff708cd6fc4698b29628f6abc33
Parents: 7a96033
Author: Steve Loughran <st...@apache.org>
Authored: Fri Sep 1 14:13:41 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Sep 1 14:13:41 2017 +0100

----------------------------------------------------------------------
 .../main/resources/assemblies/hadoop-tools.xml  |   13 +
 hadoop-common-project/hadoop-common/pom.xml     |    5 +
 .../apache/hadoop/fs/AbstractFileSystem.java    |    8 +
 .../java/org/apache/hadoop/fs/FileContext.java  |    9 +
 .../src/main/resources/core-default.xml         |  108 ++
 .../hadoop/fs/FileSystemContractBaseTest.java   |   16 +-
 .../fs/contract/AbstractContractRenameTest.java |   63 ++
 .../org/apache/hadoop/test/LambdaTestUtils.java |  112 ++
 hadoop-project/pom.xml                          |   19 +
 .../hadoop-aws/dev-support/findbugs-exclude.xml |    6 +
 hadoop-tools/hadoop-aws/pom.xml                 |  129 ++-
 .../org/apache/hadoop/fs/s3a/Constants.java     |  133 ++-
 .../hadoop/fs/s3a/DefaultS3ClientFactory.java   |  233 ++++
 .../fs/s3a/InconsistentAmazonS3Client.java      |  434 ++++++++
 .../fs/s3a/InconsistentS3ClientFactory.java     |   40 +
 .../java/org/apache/hadoop/fs/s3a/Listing.java  |  263 ++++-
 .../hadoop/fs/s3a/S3ABlockOutputStream.java     |   17 +-
 .../org/apache/hadoop/fs/s3a/S3AFileStatus.java |   45 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  572 ++++++++--
 .../hadoop/fs/s3a/S3AInstrumentation.java       |   81 +-
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |   14 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |   36 +-
 .../apache/hadoop/fs/s3a/S3ClientFactory.java   |  190 +---
 .../org/apache/hadoop/fs/s3a/Statistic.java     |   13 +-
 .../java/org/apache/hadoop/fs/s3a/Tristate.java |   32 +
 .../org/apache/hadoop/fs/s3a/UploadInfo.java    |   43 +
 .../fs/s3a/s3guard/DescendantsIterator.java     |  142 +++
 .../fs/s3a/s3guard/DirListingMetadata.java      |  322 ++++++
 .../fs/s3a/s3guard/DynamoDBClientFactory.java   |  132 +++
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   | 1010 ++++++++++++++++++
 .../fs/s3a/s3guard/LocalMetadataStore.java      |  435 ++++++++
 .../hadoop/fs/s3a/s3guard/LruHashMap.java       |   50 +
 .../hadoop/fs/s3a/s3guard/MetadataStore.java    |  221 ++++
 .../s3guard/MetadataStoreListFilesIterator.java |  169 +++
 .../fs/s3a/s3guard/NullMetadataStore.java       |  104 ++
 .../hadoop/fs/s3a/s3guard/PathMetadata.java     |  143 +++
 .../PathMetadataDynamoDBTranslation.java        |  304 ++++++
 .../apache/hadoop/fs/s3a/s3guard/S3Guard.java   |  463 ++++++++
 .../hadoop/fs/s3a/s3guard/S3GuardTool.java      |  924 ++++++++++++++++
 .../hadoop/fs/s3a/s3guard/package-info.java     |   30 +
 .../hadoop/fs/s3native/S3xLoginHelper.java      |    4 +
 .../src/main/shellprofile.d/hadoop-s3guard.sh   |   37 +
 .../src/site/markdown/tools/hadoop-aws/index.md |    3 +-
 .../site/markdown/tools/hadoop-aws/s3guard.md   |  610 +++++++++++
 .../site/markdown/tools/hadoop-aws/testing.md   |  288 ++++-
 .../fs/contract/s3a/ITestS3AContractCreate.java |   14 +
 .../fs/contract/s3a/ITestS3AContractDelete.java |   14 +
 .../fs/contract/s3a/ITestS3AContractDistCp.java |    7 +
 .../s3a/ITestS3AContractGetFileStatus.java      |    4 +
 .../fs/contract/s3a/ITestS3AContractMkdir.java  |   14 +
 .../fs/contract/s3a/ITestS3AContractOpen.java   |   14 +
 .../fs/contract/s3a/ITestS3AContractRename.java |   13 +
 .../contract/s3a/ITestS3AContractRootDir.java   |   14 +
 .../fs/contract/s3a/ITestS3AContractSeek.java   |   14 +
 .../hadoop/fs/s3a/AbstractS3AMockTest.java      |    9 +-
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      |   26 +-
 .../fs/s3a/ITestS3AAWSCredentialsProvider.java  |    4 +
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |    3 +-
 .../fs/s3a/ITestS3ACopyFromLocalFile.java       |    3 +-
 .../hadoop/fs/s3a/ITestS3ACredentialsInURL.java |   13 +-
 .../hadoop/fs/s3a/ITestS3ADelayedFNF.java       |   62 ++
 .../hadoop/fs/s3a/ITestS3AEmptyDirectory.java   |   83 ++
 .../hadoop/fs/s3a/ITestS3AEncryptionSSEC.java   |  319 +++---
 .../fs/s3a/ITestS3AFileOperationCost.java       |   40 +-
 .../fs/s3a/ITestS3AFileSystemContract.java      |    1 +
 .../hadoop/fs/s3a/ITestS3AInconsistency.java    |  100 ++
 .../hadoop/fs/s3a/ITestS3AMiscOperations.java   |   27 +
 .../hadoop/fs/s3a/ITestS3GuardCreate.java       |   61 ++
 .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java    |   85 ++
 .../fs/s3a/ITestS3GuardListConsistency.java     |  544 ++++++++++
 .../hadoop/fs/s3a/ITestS3GuardWriteBack.java    |  141 +++
 .../hadoop/fs/s3a/MockS3ClientFactory.java      |    3 +
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |   12 +
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  197 +++-
 .../org/apache/hadoop/fs/s3a/TestListing.java   |  118 ++
 .../ITestS3AFileContextStatistics.java          |    4 +-
 .../s3a/fileContext/ITestS3AFileContextURI.java |   19 +-
 .../fs/s3a/s3guard/AbstractMSContract.java      |   33 +
 .../s3guard/AbstractS3GuardToolTestBase.java    |  161 +++
 .../s3a/s3guard/DynamoDBLocalClientFactory.java |  157 +++
 .../s3a/s3guard/ITestS3GuardConcurrentOps.java  |  160 +++
 .../s3a/s3guard/ITestS3GuardToolDynamoDB.java   |  134 +++
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   |  149 +++
 .../fs/s3a/s3guard/MetadataStoreTestBase.java   |  887 +++++++++++++++
 .../fs/s3a/s3guard/TestDirListingMetadata.java  |  303 ++++++
 .../s3a/s3guard/TestDynamoDBMetadataStore.java  |  594 ++++++++++
 .../fs/s3a/s3guard/TestLocalMetadataStore.java  |  140 +++
 .../fs/s3a/s3guard/TestNullMetadataStore.java   |   58 +
 .../TestPathMetadataDynamoDBTranslation.java    |  238 +++++
 .../hadoop/fs/s3a/s3guard/TestS3Guard.java      |   93 ++
 .../AbstractITestS3AMetadataStoreScale.java     |  250 +++++
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java |   13 +-
 .../scale/ITestDynamoDBMetadataStoreScale.java  |   48 +
 .../s3a/scale/ITestLocalMetadataStoreScale.java |   37 +
 .../fs/s3a/scale/ITestS3AConcurrentOps.java     |    3 +-
 .../fs/s3a/scale/ITestS3ACreatePerformance.java |   86 ++
 .../s3a/scale/ITestS3ADirectoryPerformance.java |    5 +-
 .../scale/ITestS3AInputStreamPerformance.java   |    4 +-
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   |    2 +-
 .../hadoop-aws/src/test/resources/core-site.xml |   26 +
 .../src/test/resources/log4j.properties         |   15 +-
 101 files changed, 13065 insertions(+), 538 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
index bc9548b..0a4367d 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
@@ -174,6 +174,19 @@
       <directory>../hadoop-sls/target/hadoop-sls-${project.version}/sls</directory>
       <outputDirectory>/share/hadoop/${hadoop.component}/sls</outputDirectory>
     </fileSet>
+    <fileSet>
+      <directory>../hadoop-aws/src/main/bin</directory>
+      <outputDirectory>/bin</outputDirectory>
+      <fileMode>0755</fileMode>
+    </fileSet>
+    <fileSet>
+      <directory>../hadoop-aws/src/main/shellprofile.d</directory>
+      <includes>
+        <include>*</include>
+      </includes>
+      <outputDirectory>/libexec/shellprofile.d</outputDirectory>
+      <fileMode>0755</fileMode>
+    </fileSet>
   </fileSets>
   <dependencySets>
     <dependencySet>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index e9222bb..1d188ba 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -172,6 +172,11 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
index 9bea8f9..df14ee8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -132,6 +133,13 @@ public abstract class AbstractFileSystem {
         CONSTRUCTOR_CACHE.put(theClass, meth);
       }
       result = meth.newInstance(uri, conf);
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof RuntimeException) {
+        throw (RuntimeException) cause;
+      } else {
+        throw new RuntimeException(cause);
+      }
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index fef968b..21733b3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -331,6 +331,15 @@ public class FileContext {
           return AbstractFileSystem.get(uri, conf);
         }
       });
+    } catch (RuntimeException ex) {
+      // RTEs can wrap other exceptions; if there is an IOException inner,
+      // throw it direct.
+      Throwable cause = ex.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else {
+        throw ex;
+      }
     } catch (InterruptedException ex) {
       LOG.error(ex.toString());
       throw new IOException("Failed to get the AbstractFileSystem for path: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index cb061aa..9e2c553 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1309,12 +1309,120 @@
 </property>
 
 <property>
+    <name>fs.s3a.metadatastore.authoritative</name>
+    <value>false</value>
+    <description>
+        When true, allow MetadataStore implementations to act as source of
+        truth for getting file status and directory listings.  Even if this
+        is set to true, MetadataStore implementations may choose not to
+        return authoritative results.  If the configured MetadataStore does
+        not support being authoritative, this setting will have no effect.
+    </description>
+</property>
+
+<property>
+    <name>fs.s3a.metadatastore.impl</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+    <description>
+        Fully-qualified name of the class that implements the MetadataStore
+        to be used by s3a.  The default class, NullMetadataStore, has no
+        effect: s3a will continue to treat the backing S3 service as the one
+        and only source of truth for file and directory metadata.
+    </description>
+</property>
+
+<property>
+    <name>fs.s3a.s3guard.cli.prune.age</name>
+    <value>86400000</value>
+    <description>
+        Default age (in milliseconds) after which to prune metadata from the
+        metadatastore when the prune command is run.  Can be overridden on the
+        command-line.
+    </description>
+</property>
+
+
+<property>
   <name>fs.s3a.impl</name>
   <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
   <description>The implementation class of the S3A Filesystem</description>
 </property>
 
 <property>
+  <name>fs.s3a.s3guard.ddb.region</name>
+  <value></value>
+  <description>
+    AWS DynamoDB region to connect to. An up-to-date list is
+    provided in the AWS Documentation: regions and endpoints. Without this
+    property, the S3Guard will operate table in the associated S3 bucket region.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table</name>
+  <value></value>
+  <description>
+    The DynamoDB table name to operate. Without this property, the respective
+    S3 bucket name will be used.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.create</name>
+  <value>false</value>
+  <description>
+    If true, the S3A client will create the table if it does not already exist.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.read</name>
+  <value>500</value>
+  <description>
+    Provisioned throughput requirements for read operations in terms of capacity
+    units for the DynamoDB table.  This config value will only be used when
+    creating a new DynamoDB table, though later you can manually provision by
+    increasing or decreasing read capacity as needed for existing tables.
+    See DynamoDB documents for more information.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.write</name>
+  <value>100</value>
+  <description>
+    Provisioned throughput requirements for write operations in terms of
+    capacity units for the DynamoDB table.  Refer to related config
+    fs.s3a.s3guard.ddb.table.capacity.read before usage.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.max.retries</name>
+  <value>9</value>
+    <description>
+      Max retries on batched DynamoDB operations before giving up and
+      throwing an IOException.  Each retry is delayed with an exponential
+      backoff timer which starts at 100 milliseconds and approximately
+      doubles each time.  The minimum wait before throwing an exception is
+      sum(100, 200, 400, 800, .. 100*2^N-1 ) == 100 * ((2^N)-1)
+      So N = 9 yields at least 51.1 seconds (51,100) milliseconds of blocking
+      before throwing an IOException.
+    </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.background.sleep</name>
+  <value>25</value>
+  <description>
+    Length (in milliseconds) of pause between each batch of deletes when
+    pruning metadata.  Prevents prune operations (which can typically be low
+    priority background operations) from overly interfering with other I/O
+    operations.
+  </description>
+</property>
+
+<property>
   <name>fs.AbstractFileSystem.s3a.impl</name>
   <value>org.apache.hadoop.fs.s3a.S3A</value>
   <description>The implementation class of the S3A AbstractFileSystem.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
index 92e2135..9d8cd64 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
@@ -748,13 +748,27 @@ public abstract class FileSystemContractBaseTest {
 
   /**
    * This a sanity check to make sure that any filesystem's handling of
-   * renames doesn't cause any regressions
+   * renames empty dirs doesn't cause any regressions.
+   */
+  public void testRenameEmptyToDirWithSamePrefixAllowed() throws Throwable {
+    assumeTrue(renameSupported());
+    Path parentdir = path("testRenameEmptyToDirWithSamePrefixAllowed");
+    fs.mkdirs(parentdir);
+    Path dest = path("testRenameEmptyToDirWithSamePrefixAllowedDest");
+    rename(parentdir, dest, true, false, true);
+  }
+
+  /**
+   * This a sanity check to make sure that any filesystem's handling of
+   * renames non-empty dirs doesn't cause any regressions.
    */
   @Test
   public void testRenameToDirWithSamePrefixAllowed() throws Throwable {
     assumeTrue(renameSupported());
     final Path parentdir = path("testRenameToDirWithSamePrefixAllowed");
     fs.mkdirs(parentdir);
+    // Before renaming, we create one file under the source parent directory
+    createFile(new Path(parentdir, "mychild"));
     final Path dest = path("testRenameToDirWithSamePrefixAllowedDest");
     rename(parentdir, dest, true, false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
index b0dcb93..b6d0a49 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
@@ -222,4 +222,67 @@ public abstract class AbstractContractRenameTest extends
     assertPathDoesNotExist("not deleted",
         new Path(srcDir, "source.txt"));
   }
+
+  /**
+   * Test that after renaming, the nested subdirectory is moved along with all
+   * its ancestors.
+   */
+  @Test
+  public void testRenamePopulatesDirectoryAncestors() throws IOException {
+    final FileSystem fs = getFileSystem();
+    final Path src = path("testRenamePopulatesDirectoryAncestors/source");
+    fs.mkdirs(src);
+    final String nestedDir = "/dir1/dir2/dir3/dir4";
+    fs.mkdirs(path(src + nestedDir));
+
+    Path dst = path("testRenamePopulatesDirectoryAncestorsNew");
+
+    fs.rename(src, dst);
+    validateAncestorsMoved(src, dst, nestedDir);
+  }
+
+  /**
+   * Test that after renaming, the nested file is moved along with all its
+   * ancestors. It is similar to {@link #testRenamePopulatesDirectoryAncestors}.
+   */
+  @Test
+  public void testRenamePopulatesFileAncestors() throws IOException {
+    final FileSystem fs = getFileSystem();
+    final Path src = path("testRenamePopulatesFileAncestors/source");
+    fs.mkdirs(src);
+    final String nestedFile = "/dir1/dir2/dir3/file4";
+    byte[] srcDataset = dataset(256, 'a', 'z');
+    writeDataset(fs, path(src + nestedFile), srcDataset, srcDataset.length,
+        1024, false);
+
+    Path dst = path("testRenamePopulatesFileAncestorsNew");
+
+    fs.rename(src, dst);
+    validateAncestorsMoved(src, dst, nestedFile);
+  }
+
+  /**
+   * Validate that the nested path and its ancestors should have been moved.
+   *
+   * @param src the source root to move
+   * @param dst the destination root to move
+   * @param nestedPath the nested path to move
+   */
+  private void validateAncestorsMoved(Path src, Path dst, String nestedPath)
+      throws IOException {
+    assertIsDirectory(dst);
+    assertPathDoesNotExist("src path should not exist", path(src + nestedPath));
+    assertPathExists("dst path should exist", path(dst + nestedPath));
+
+    Path path = new Path(nestedPath).getParent();
+    while (path != null && !path.isRoot()) {
+      final Path parentSrc = path(src + path.toString());
+      assertPathDoesNotExist(parentSrc + " is not deleted", parentSrc);
+      final Path parentDst = path(dst + path.toString());
+      assertPathExists(parentDst + " should exist after rename", parentDst);
+      assertIsDirectory(parentDst);
+      path = path.getParent();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
index 1fa5c3f..00cfa44 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
@@ -249,6 +249,23 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Variant of {@link #eventually(int, Callable, Callable)} method for
+   * void lambda expressions.
+   * @param timeoutMillis timeout in milliseconds.
+   * Can be zero, in which case only one attempt is made before failing.
+   * @param eval expression to evaluate
+   * @param retry retry interval generator
+   * @throws Exception the last exception thrown before timeout was triggered
+   * @throws FailFastException if raised -without any retry attempt.
+   * @throws InterruptedException if interrupted during the sleep operation.
+   */
+  public static void eventually(int timeoutMillis,
+      VoidCallable eval,
+      Callable<Integer> retry) throws Exception {
+    eventually(timeoutMillis, new VoidCaller(eval), retry);
+  }
+
+  /**
    * Simplified {@link #eventually(int, Callable, Callable)} method
    * with a fixed interval.
    * <p>
@@ -277,6 +294,25 @@ public final class LambdaTestUtils {
   }
 
   /**
+   /**
+   * Variant of {@link #eventually(int, int, Callable)} method for
+   * void lambda expressions.
+   * @param timeoutMillis timeout in milliseconds.
+   * Can be zero, in which case only one attempt is made before failing.
+   * @param intervalMillis interval in milliseconds
+   * @param eval expression to evaluate
+   * @throws Exception the last exception thrown before timeout was triggered
+   * @throws FailFastException if raised -without any retry attempt.
+   * @throws InterruptedException if interrupted during the sleep operation.
+   */
+  public static void eventually(int timeoutMillis,
+      int intervalMillis,
+      VoidCallable eval) throws Exception {
+    eventually(timeoutMillis, eval,
+        new FixedRetryInterval(intervalMillis));
+  }
+
+  /**
    * Intercept an exception; throw an {@code AssertionError} if one not raised.
    * The caught exception is rethrown if it is of the wrong class or
    * does not contain the text defined in {@code contained}.
@@ -319,6 +355,32 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Variant of {@link #intercept(Class, Callable)} to simplify void
+   * invocations.
+   * @param clazz class of exception; the raised exception must be this class
+   * <i>or a subclass</i>.
+   * @param eval expression to eval
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type
+   * @throws Exception any other exception raised
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   */
+  public static <E extends Throwable> E intercept(
+      Class<E> clazz,
+      VoidCallable eval)
+      throws Exception {
+    try {
+      eval.call();
+      throw new AssertionError("Expected an exception");
+    } catch (Throwable e) {
+      if (clazz.isAssignableFrom(e.getClass())) {
+        return (E)e;
+      }
+      throw e;
+    }
+  }
+
+  /**
    * Intercept an exception; throw an {@code AssertionError} if one not raised.
    * The caught exception is rethrown if it is of the wrong class or
    * does not contain the text defined in {@code contained}.
@@ -359,6 +421,29 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Variant of {@link #intercept(Class, Callable)} to simplify void
+   * invocations.
+   * @param clazz class of exception; the raised exception must be this class
+   * <i>or a subclass</i>.
+   * @param contained string which must be in the {@code toString()} value
+   * of the exception
+   * @param eval expression to eval
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type
+   * @throws Exception any other exception raised
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   */
+  public static <E extends Throwable> E intercept(
+      Class<E> clazz,
+      String contained,
+      VoidCallable eval)
+      throws Exception {
+    E ex = intercept(clazz, eval);
+    GenericTestUtils.assertExceptionContains(contained, ex);
+    return ex;
+  }
+
+  /**
    * Robust string converter for exception messages; if the {@code toString()}
    * method throws an exception then that exception is caught and logged,
    * then a simple string of the classname logged.
@@ -518,4 +603,31 @@ public final class LambdaTestUtils {
       return new FailFastException(String.format(format, args));
     }
   }
+
+  /**
+   * A simple interface for lambdas, which returns nothing; this exists
+   * to simplify lambda tests on operations with no return value.
+   */
+  public interface VoidCallable {
+    void call() throws Exception;
+  }
+
+  /**
+   * Bridge class to make {@link VoidCallable} something to use in anything
+   * which takes an {@link Callable}.
+   */
+  public static class VoidCaller implements Callable<Void> {
+    private final VoidCallable callback;
+
+    public VoidCaller(VoidCallable callback) {
+      this.callback = callback;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      callback.call();
+      return null;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 938ef05..9a52d76 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -870,6 +870,17 @@
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-configuration2</artifactId>
         <version>2.1</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-lang3</artifactId>
+        <version>3.4</version>
       </dependency>
       <dependency>
         <groupId>org.slf4j</groupId>
@@ -1734,4 +1745,12 @@
       </build>
     </profile>
   </profiles>
+
+  <repositories>
+    <repository>
+      <id>dynamodb-local-oregon</id>
+      <name>DynamoDB Local Release Repository</name>
+      <url>https://s3-us-west-2.amazonaws.com/dynamodb-local/release</url>
+    </repository>
+  </repositories>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index ffb0a79..82ec16e 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -26,4 +26,10 @@
   <Match>
     <Class name="org.apache.hadoop.fs.s3.INode" />
   </Match>
+  <!-- Redundant null check makes code clearer, future-proof here. -->
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3a.S3AFileSystem" />
+    <Method name="s3Exists" />
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE" />
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index 91e94a6..bcb0e07 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -36,6 +36,7 @@
     <downloadSources>true</downloadSources>
     <hadoop.tmp.dir>${project.build.directory}/test</hadoop.tmp.dir>
 
+    <dynamodb.local.version>1.11.86</dynamodb.local.version>
     <!-- are scale tests enabled ? -->
     <fs.s3a.scale.test.enabled>unset</fs.s3a.scale.test.enabled>
     <!-- Size in MB of huge files. -->
@@ -44,6 +45,11 @@
     <fs.s3a.scale.test.huge.partitionsize>unset</fs.s3a.scale.test.huge.partitionsize>
     <!-- Timeout in seconds for scale tests.-->
     <fs.s3a.scale.test.timeout>3600</fs.s3a.scale.test.timeout>
+    <!-- are scale tests enabled ? -->
+    <fs.s3a.s3guard.test.enabled>false</fs.s3a.s3guard.test.enabled>
+    <fs.s3a.s3guard.test.authoritative>false</fs.s3a.s3guard.test.authoritative>
+    <fs.s3a.s3guard.test.implementation>local</fs.s3a.s3guard.test.implementation>
+
   </properties>
 
   <profiles>
@@ -164,6 +170,11 @@
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
                     <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
                     <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                    <!-- S3Guard -->
+                    <fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
+                    <fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
+                    <fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
+
                   </systemPropertyVariables>
                   <!-- Some tests cannot run in parallel.  Tests that cover -->
                   <!-- access to the root directory must run in isolation -->
@@ -205,6 +216,10 @@
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
                     <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
                     <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                    <!-- S3Guard -->
+                    <fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
+                    <fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
+                    <fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
                   </systemPropertyVariables>
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- parallel execution. -->
@@ -247,6 +262,10 @@
                     <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
                     <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                    <!-- S3Guard -->
+                    <fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
+                    <fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
+                    <fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
                   </systemPropertyVariables>
                   <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                 </configuration>
@@ -269,6 +288,60 @@
         <fs.s3a.scale.test.enabled>true</fs.s3a.scale.test.enabled>
       </properties>
     </profile>
+
+    <!-- Turn on S3Guard tests-->
+    <profile>
+      <id>s3guard</id>
+      <activation>
+        <property>
+          <name>s3guard</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.s3guard.test.enabled>true</fs.s3a.s3guard.test.enabled>
+      </properties>
+    </profile>
+
+    <!-- Switch to DynamoDB for S3Guard. Has no effect unless S3Guard is enabled -->
+    <profile>
+      <id>dynamo</id>
+      <activation>
+        <property>
+          <name>dynamo</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.s3guard.test.implementation>dynamo</fs.s3a.s3guard.test.implementation>
+      </properties>
+    </profile>
+
+    <!-- Switch to DynamoDBLocal for S3Guard. Has no effect unless S3Guard is enabled -->
+    <profile>
+      <id>dynamodblocal</id>
+      <activation>
+        <property>
+          <name>dynamodblocal</name>
+        </property>
+      </activation>
+      <properties>
+        <fs.s3a.s3guard.test.implementation>dynamodblocal</fs.s3a.s3guard.test.implementation>
+      </properties>
+    </profile>
+
+    <!-- Switch S3Guard from Authoritative=false to true
+     Has no effect unless S3Guard is enabled -->
+    <profile>
+      <id>non-auth</id>
+      <activation>
+        <property>
+          <name>auth</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.s3guard.test.authoritative>true</fs.s3a.s3guard.test.authoritative>
+      </properties>
+    </profile>
+
   </profiles>
 
   <build>
@@ -296,16 +369,48 @@
         <artifactId>maven-dependency-plugin</artifactId>
         <executions>
           <execution>
-            <id>deplist</id>
+            <id>deplist1</id>
             <phase>compile</phase>
             <goals>
               <goal>list</goal>
             </goals>
             <configuration>
-              <!-- build a shellprofile -->
+              <!-- build a shellprofile for hadoop-aws optional tools -->
               <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt</outputFile>
             </configuration>
           </execution>
+          <execution>
+            <id>copy</id>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeScope>test</includeScope>
+              <includeTypes>so,dll,dylib</includeTypes>
+              <outputDirectory>${project.build.directory}/native-libs</outputDirectory>
+            </configuration>
+          </execution>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+            </configuration>
+          </execution>
+          <execution>
+            <id>deplist2</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- referenced by the s3guard command -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt</outputFile>
+            </configuration>
+          </execution>
         </executions>
       </plugin>
     </plugins>
@@ -334,6 +439,26 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>DynamoDBLocal</artifactId>
+      <version>${dynamodb.local.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.hamcrest</groupId>
+          <artifactId>hamcrest-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-http</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.commons</groupId>
+          <artifactId>commons-lang3</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 9e15b3f..1a464d0 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -267,6 +267,11 @@ public final class Constants {
 
   public static final String USER_AGENT_PREFIX = "fs.s3a.user.agent.prefix";
 
+  /** Whether or not to allow MetadataStore to be source of truth. */
+  public static final String METADATASTORE_AUTHORITATIVE =
+      "fs.s3a.metadatastore.authoritative";
+  public static final boolean DEFAULT_METADATASTORE_AUTHORITATIVE = false;
+
   /** read ahead buffer size to prevent connection re-establishments. */
   public static final String READAHEAD_RANGE = "fs.s3a.readahead.range";
   public static final long DEFAULT_READAHEAD_RANGE = 64 * 1024;
@@ -312,7 +317,7 @@ public final class Constants {
   @InterfaceStability.Unstable
   public static final Class<? extends S3ClientFactory>
       DEFAULT_S3_CLIENT_FACTORY_IMPL =
-          S3ClientFactory.DefaultS3ClientFactory.class;
+          DefaultS3ClientFactory.class;
 
   /**
    * Maximum number of partitions in a multipart upload: {@value}.
@@ -320,4 +325,130 @@ public final class Constants {
   @InterfaceAudience.Private
   public static final int MAX_MULTIPART_COUNT = 10000;
 
+  /**
+   * Classname of the S3A-specific output committer factory. This
+   * is what must be declared when attempting to use
+   */
+  @InterfaceStability.Unstable
+  public static final String S3A_OUTPUT_COMMITTER_FACTORY =
+      "org.apache.hadoop.fs.s3a.commit.S3AOutputCommitterFactory";
+
+  /* Constants. */
+  public static final String S3_METADATA_STORE_IMPL =
+      "fs.s3a.metadatastore.impl";
+
+  /** Minimum period of time (in milliseconds) to keep metadata (may only be
+   * applied when a prune command is manually run).
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_CLI_PRUNE_AGE =
+      "fs.s3a.s3guard.cli.prune.age";
+
+  /**
+   * The region of the DynamoDB service.
+   *
+   * This config has no default value. If the user does not set this, the
+   * S3Guard will operate table in the associated S3 bucket region.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_REGION_KEY =
+      "fs.s3a.s3guard.ddb.region";
+
+  /**
+   * The DynamoDB table name to use.
+   *
+   * This config has no default value. If the user does not set this, the
+   * S3Guard implementation will use the respective S3 bucket name.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_NAME_KEY =
+      "fs.s3a.s3guard.ddb.table";
+
+  /**
+   * Whether to create the DynamoDB table if the table does not exist.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_CREATE_KEY =
+      "fs.s3a.s3guard.ddb.table.create";
+
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_CAPACITY_READ_KEY =
+      "fs.s3a.s3guard.ddb.table.capacity.read";
+  public static final long S3GUARD_DDB_TABLE_CAPACITY_READ_DEFAULT = 500;
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY =
+      "fs.s3a.s3guard.ddb.table.capacity.write";
+  public static final long S3GUARD_DDB_TABLE_CAPACITY_WRITE_DEFAULT = 100;
+
+  /**
+   * The maximum put or delete requests per BatchWriteItem request.
+   *
+   * Refer to Amazon API reference for this limit.
+   */
+  public static final int S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT = 25;
+
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_MAX_RETRIES =
+      "fs.s3a.s3guard.ddb.max.retries";
+  /**
+   * Max retries on batched DynamoDB operations before giving up and
+   * throwing an IOException.  Default is {@value}. See core-default.xml for
+   * more detail.
+   */
+  public static final int S3GUARD_DDB_MAX_RETRIES_DEFAULT = 9;
+
+  /**
+   * Period of time (in milliseconds) to sleep between batches of writes.
+   * Currently only applies to prune operations, as they are naturally a
+   * lower priority than other operations.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY =
+      "fs.s3a.s3guard.ddb.background.sleep";
+  public static final int S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT = 25;
+
+  /**
+   * V1 committer.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3A_OUTPUT_COMMITTER_MRV1 =
+      "org.apache.hadoop.fs.s3a.commit.S3OutputCommitterMRv1";
+
+  /**
+   * The default "Null" metadata store: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_METASTORE_NULL
+      = "org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore";
+
+  /**
+   * Use Local memory for the metadata: {@value}.
+   * This is not coherent across processes and must be used for testing only.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_METASTORE_LOCAL
+      = "org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore";
+
+  /**
+   * Use DynamoDB for the metadata: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_METASTORE_DYNAMO
+      = "org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore";
+
+  /**
+   * Inconsistency (visibility delay) injection settings.
+   */
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_INCONSISTENCY_KEY =
+      "fs.s3a.failinject.inconsistency.key.substring";
+
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_INCONSISTENCY_MSEC =
+      "fs.s3a.failinject.inconsistency.msec";
+
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_INCONSISTENCY_PROBABILITY =
+      "fs.s3a.failinject.inconsistency.probability";
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
new file mode 100644
index 0000000..f33b25e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.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.fs.s3a;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.Protocol;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.S3ClientOptions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.VersionInfo;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
+import static org.apache.hadoop.fs.s3a.S3AUtils.intOption;
+
+/**
+ * The default factory implementation, which calls the AWS SDK to configure
+ * and create an {@link AmazonS3Client} that communicates with the S3 service.
+ */
+public class DefaultS3ClientFactory extends Configured implements
+    S3ClientFactory {
+
+  protected static final Logger LOG = S3AFileSystem.LOG;
+
+  @Override
+  public AmazonS3 createS3Client(URI name) throws IOException {
+    Configuration conf = getConf();
+    AWSCredentialsProvider credentials =
+        createAWSCredentialProviderSet(name, conf);
+    final ClientConfiguration awsConf = createAwsConf(getConf());
+    AmazonS3 s3 = newAmazonS3Client(credentials, awsConf);
+    return createAmazonS3Client(s3, conf, credentials, awsConf);
+  }
+
+  /**
+   * Create a new {@link ClientConfiguration}.
+   * @param conf The Hadoop configuration
+   * @return new AWS client configuration
+   */
+  public static ClientConfiguration createAwsConf(Configuration conf) {
+    final ClientConfiguration awsConf = new ClientConfiguration();
+    initConnectionSettings(conf, awsConf);
+    initProxySupport(conf, awsConf);
+    initUserAgent(conf, awsConf);
+    return awsConf;
+  }
+
+  /**
+   * Wrapper around constructor for {@link AmazonS3} client.  Override this to
+   * provide an extended version of the client
+   * @param credentials credentials to use
+   * @param awsConf  AWS configuration
+   * @return  new AmazonS3 client
+   */
+  protected AmazonS3 newAmazonS3Client(
+      AWSCredentialsProvider credentials, ClientConfiguration awsConf) {
+    return new AmazonS3Client(credentials, awsConf);
+  }
+
+  /**
+   * Initializes all AWS SDK settings related to connection management.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   */
+  private static void initConnectionSettings(Configuration conf,
+      ClientConfiguration awsConf) {
+    awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
+        DEFAULT_MAXIMUM_CONNECTIONS, 1));
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
+        DEFAULT_SECURE_CONNECTIONS);
+    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
+    awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
+        DEFAULT_MAX_ERROR_RETRIES, 0));
+    awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
+        DEFAULT_ESTABLISH_TIMEOUT, 0));
+    awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
+        DEFAULT_SOCKET_TIMEOUT, 0));
+    int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
+        DEFAULT_SOCKET_SEND_BUFFER, 2048);
+    int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
+        DEFAULT_SOCKET_RECV_BUFFER, 2048);
+    awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
+    String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
+    if (!signerOverride.isEmpty()) {
+      LOG.debug("Signer override = {}", signerOverride);
+      awsConf.setSignerOverride(signerOverride);
+    }
+  }
+
+  /**
+   * Initializes AWS SDK proxy support if configured.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   * @throws IllegalArgumentException if misconfigured
+   */
+  private static void initProxySupport(Configuration conf,
+      ClientConfiguration awsConf) throws IllegalArgumentException {
+    String proxyHost = conf.getTrimmed(PROXY_HOST, "");
+    int proxyPort = conf.getInt(PROXY_PORT, -1);
+    if (!proxyHost.isEmpty()) {
+      awsConf.setProxyHost(proxyHost);
+      if (proxyPort >= 0) {
+        awsConf.setProxyPort(proxyPort);
+      } else {
+        if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
+          LOG.warn("Proxy host set without port. Using HTTPS default 443");
+          awsConf.setProxyPort(443);
+        } else {
+          LOG.warn("Proxy host set without port. Using HTTP default 80");
+          awsConf.setProxyPort(80);
+        }
+      }
+      String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
+      String proxyPassword = conf.getTrimmed(PROXY_PASSWORD);
+      if ((proxyUsername == null) != (proxyPassword == null)) {
+        String msg = "Proxy error: " + PROXY_USERNAME + " or " +
+            PROXY_PASSWORD + " set without the other.";
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg);
+      }
+      awsConf.setProxyUsername(proxyUsername);
+      awsConf.setProxyPassword(proxyPassword);
+      awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
+      awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
+                "domain {} as workstation {}", awsConf.getProxyHost(),
+            awsConf.getProxyPort(),
+            String.valueOf(awsConf.getProxyUsername()),
+            awsConf.getProxyPassword(), awsConf.getProxyDomain(),
+            awsConf.getProxyWorkstation());
+      }
+    } else if (proxyPort >= 0) {
+      String msg =
+          "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
+      LOG.error(msg);
+      throw new IllegalArgumentException(msg);
+    }
+  }
+
+  /**
+   * Initializes the User-Agent header to send in HTTP requests to the S3
+   * back-end.  We always include the Hadoop version number.  The user also
+   * may set an optional custom prefix to put in front of the Hadoop version
+   * number.  The AWS SDK interally appends its own information, which seems
+   * to include the AWS SDK version, OS and JVM version.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   */
+  private static void initUserAgent(Configuration conf,
+      ClientConfiguration awsConf) {
+    String userAgent = "Hadoop " + VersionInfo.getVersion();
+    String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
+    if (!userAgentPrefix.isEmpty()) {
+      userAgent = userAgentPrefix + ", " + userAgent;
+    }
+    LOG.debug("Using User-Agent: {}", userAgent);
+    awsConf.setUserAgentPrefix(userAgent);
+  }
+
+  /**
+   * Creates an {@link AmazonS3Client} from the established configuration.
+   *
+   * @param conf Hadoop configuration
+   * @param credentials AWS credentials
+   * @param awsConf AWS SDK configuration
+   * @return S3 client
+   * @throws IllegalArgumentException if misconfigured
+   */
+  private static AmazonS3 createAmazonS3Client(AmazonS3 s3, Configuration conf,
+      AWSCredentialsProvider credentials, ClientConfiguration awsConf)
+      throws IllegalArgumentException {
+    String endPoint = conf.getTrimmed(ENDPOINT, "");
+    if (!endPoint.isEmpty()) {
+      try {
+        s3.setEndpoint(endPoint);
+      } catch (IllegalArgumentException e) {
+        String msg = "Incorrect endpoint: "  + e.getMessage();
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg, e);
+      }
+    }
+    enablePathStyleAccessIfRequired(s3, conf);
+    return s3;
+  }
+
+  /**
+   * Enables path-style access to S3 buckets if configured.  By default, the
+   * behavior is to use virtual hosted-style access with URIs of the form
+   * http://bucketname.s3.amazonaws.com.  Enabling path-style access and a
+   * region-specific endpoint switches the behavior to use URIs of the form
+   * http://s3-eu-west-1.amazonaws.com/bucketname.
+   *
+   * @param s3 S3 client
+   * @param conf Hadoop configuration
+   */
+  private static void enablePathStyleAccessIfRequired(AmazonS3 s3,
+      Configuration conf) {
+    final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
+    if (pathStyleAccess) {
+      LOG.debug("Enabling path style access!");
+      s3.setS3ClientOptions(S3ClientOptions.builder()
+          .setPathStyleAccess(true)
+          .build());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
new file mode 100644
index 0000000..5e9cb3f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
@@ -0,0 +1,434 @@
+/*
+ * 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.s3a;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.model.DeleteObjectRequest;
+import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.DeleteObjectsResult;
+import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A wrapper around {@link com.amazonaws.services.s3.AmazonS3} that injects
+ * inconsistency and/or errors.  Used for testing S3Guard.
+ * Currently only delays listing visibility, not affecting GET.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InconsistentAmazonS3Client extends AmazonS3Client {
+
+  /**
+   * Keys containing this substring will be subject to delayed visibility.
+   */
+  public static final String DEFAULT_DELAY_KEY_SUBSTRING = "DELAY_LISTING_ME";
+
+  /**
+   * How many seconds affected keys will be delayed from appearing in listing.
+   * This should probably be a config value.
+   */
+  public static final long DEFAULT_DELAY_KEY_MSEC = 5 * 1000;
+
+  public static final float DEFAULT_DELAY_KEY_PROBABILITY = 1.0f;
+
+  /** Special config value since we can't store empty strings in XML. */
+  public static final String MATCH_ALL_KEYS = "*";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InconsistentAmazonS3Client.class);
+
+  /** Empty string matches all keys. */
+  private String delayKeySubstring;
+
+  /** Probability to delay visibility of a matching key. */
+  private float delayKeyProbability;
+
+  /** Time in milliseconds to delay visibility of newly modified object. */
+  private long delayKeyMsec;
+
+  /**
+   * Composite of data we need to track about recently deleted objects:
+   * when it was deleted (same was with recently put objects) and the object
+   * summary (since we should keep returning it for sometime after its
+   * deletion).
+   */
+  private static class Delete {
+    private Long time;
+    private S3ObjectSummary summary;
+
+    Delete(Long time, S3ObjectSummary summary) {
+      this.time = time;
+      this.summary = summary;
+    }
+
+    public Long time() {
+      return time;
+    }
+
+    public S3ObjectSummary summary() {
+      return summary;
+    }
+  }
+
+  /** Map of key to delay -> time it was deleted + object summary (object
+   * summary is null for prefixes. */
+  private Map<String, Delete> delayedDeletes = new HashMap<>();
+
+  /** Map of key to delay -> time it was created. */
+  private Map<String, Long> delayedPutKeys = new HashMap<>();
+
+  public InconsistentAmazonS3Client(AWSCredentialsProvider credentials,
+      ClientConfiguration clientConfiguration, Configuration conf) {
+    super(credentials, clientConfiguration);
+    setupConfig(conf);
+  }
+
+  protected void setupConfig(Configuration conf) {
+
+    delayKeySubstring = conf.get(FAIL_INJECT_INCONSISTENCY_KEY,
+        DEFAULT_DELAY_KEY_SUBSTRING);
+    // "" is a substring of all strings, use it to match all keys.
+    if (delayKeySubstring.equals(MATCH_ALL_KEYS)) {
+      delayKeySubstring = "";
+    }
+    delayKeyProbability = conf.getFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY,
+        DEFAULT_DELAY_KEY_PROBABILITY);
+    delayKeyMsec = conf.getLong(FAIL_INJECT_INCONSISTENCY_MSEC,
+        DEFAULT_DELAY_KEY_MSEC);
+    LOG.info("Enabled with {} msec delay, substring {}, probability {}",
+        delayKeyMsec, delayKeySubstring, delayKeyProbability);
+  }
+
+  /**
+   * Clear all oustanding inconsistent keys.  After calling this function,
+   * listings should behave normally (no failure injection), until additional
+   * keys are matched for delay, e.g. via putObject(), deleteObject().
+   */
+  public void clearInconsistency() {
+    LOG.info("clearing all delayed puts / deletes");
+    delayedDeletes.clear();
+    delayedPutKeys.clear();
+  }
+
+  /**
+   * Convenience function for test code to cast from supertype.
+   * @param c supertype to cast from
+   * @return subtype, not null
+   * @throws Exception on error
+   */
+  public static InconsistentAmazonS3Client castFrom(AmazonS3 c) throws
+      Exception {
+    InconsistentAmazonS3Client ic = null;
+    if (c instanceof InconsistentAmazonS3Client) {
+      ic = (InconsistentAmazonS3Client) c;
+    }
+    Preconditions.checkNotNull(ic, "Not an instance of " +
+        "InconsistentAmazonS3Client");
+    return ic;
+  }
+
+  @Override
+  public DeleteObjectsResult deleteObjects(DeleteObjectsRequest
+      deleteObjectsRequest)
+      throws AmazonClientException, AmazonServiceException {
+    for (DeleteObjectsRequest.KeyVersion keyVersion :
+        deleteObjectsRequest.getKeys()) {
+      registerDeleteObject(keyVersion.getKey(), deleteObjectsRequest
+          .getBucketName());
+    }
+    return super.deleteObjects(deleteObjectsRequest);
+  }
+
+  @Override
+  public void deleteObject(DeleteObjectRequest deleteObjectRequest)
+      throws AmazonClientException, AmazonServiceException {
+    String key = deleteObjectRequest.getKey();
+    LOG.debug("key {}", key);
+    registerDeleteObject(key, deleteObjectRequest.getBucketName());
+    super.deleteObject(deleteObjectRequest);
+  }
+
+  /* We should only need to override this version of putObject() */
+  @Override
+  public PutObjectResult putObject(PutObjectRequest putObjectRequest)
+      throws AmazonClientException, AmazonServiceException {
+    LOG.debug("key {}", putObjectRequest.getKey());
+    registerPutObject(putObjectRequest);
+    return super.putObject(putObjectRequest);
+  }
+
+  /* We should only need to override this version of listObjects() */
+  @Override
+  public ObjectListing listObjects(ListObjectsRequest listObjectsRequest)
+      throws AmazonClientException, AmazonServiceException {
+    LOG.debug("prefix {}", listObjectsRequest.getPrefix());
+    ObjectListing listing = super.listObjects(listObjectsRequest);
+    listing = filterListObjects(listObjectsRequest, listing);
+    listing = restoreListObjects(listObjectsRequest, listing);
+    return listing;
+  }
+
+  private void addSummaryIfNotPresent(List<S3ObjectSummary> list,
+      S3ObjectSummary item) {
+    // Behavior of S3ObjectSummary
+    String key = item.getKey();
+    for (S3ObjectSummary member : list) {
+      if (member.getKey().equals(key)) {
+        return;
+      }
+    }
+    list.add(item);
+  }
+
+  /**
+   * Add prefix of child to given list.  The added prefix will be equal to
+   * ancestor plus one directory past ancestor.  e.g.:
+   * if ancestor is "/a/b/c" and child is "/a/b/c/d/e/file" then "a/b/c/d" is
+   * added to list.
+   * @param prefixes list to add to
+   * @param ancestor path we are listing in
+   * @param child full path to get prefix from
+   */
+  private void addPrefixIfNotPresent(List<String> prefixes, String ancestor,
+      String child) {
+    Path prefixCandidate = new Path(child).getParent();
+    Path ancestorPath = new Path(ancestor);
+    Preconditions.checkArgument(child.startsWith(ancestor), "%s does not " +
+        "start with %s", child, ancestor);
+    while (!prefixCandidate.isRoot()) {
+      Path nextParent = prefixCandidate.getParent();
+      if (nextParent.equals(ancestorPath)) {
+        String prefix = prefixCandidate.toString();
+        if (!prefixes.contains(prefix)) {
+          prefixes.add(prefix);
+        }
+        return;
+      }
+      prefixCandidate = nextParent;
+    }
+  }
+
+  /**
+   * Checks that the parent key is an ancestor of the child key.
+   * @param parent key that may be the parent.
+   * @param child key that may be the child.
+   * @param recursive if false, only return true for direct children.  If
+   *                  true, any descendant will count.
+   * @return true if parent is an ancestor of child
+   */
+  private boolean isDescendant(String parent, String child, boolean recursive) {
+    if (recursive) {
+      if (!parent.endsWith("/")) {
+        parent = parent + "/";
+      }
+      return child.startsWith(parent);
+    } else {
+      Path actualParentPath = new Path(child).getParent();
+      Path expectedParentPath = new Path(parent);
+      return actualParentPath.equals(expectedParentPath);
+    }
+  }
+
+  /**
+   * Simulate eventual consistency of delete for this list operation:  Any
+   * recently-deleted keys will be added.
+   * @param request List request
+   * @param rawListing listing returned from underlying S3
+   * @return listing with recently-deleted items restored
+   */
+  private ObjectListing restoreListObjects(ListObjectsRequest request,
+      ObjectListing rawListing) {
+    List<S3ObjectSummary> outputList = rawListing.getObjectSummaries();
+    List<String> outputPrefixes = rawListing.getCommonPrefixes();
+    // recursive list has no delimiter, returns everything that matches a
+    // prefix.
+    boolean recursiveObjectList = !("/".equals(request.getDelimiter()));
+
+    // Go through all deleted keys
+    for (String key : new HashSet<>(delayedDeletes.keySet())) {
+      Delete delete = delayedDeletes.get(key);
+      if (isKeyDelayed(delete.time(), key)) {
+        if (isDescendant(request.getPrefix(), key, recursiveObjectList)) {
+          if (delete.summary() != null) {
+            addSummaryIfNotPresent(outputList, delete.summary());
+          }
+        }
+        // Non-recursive list has delimiter: will return rolled-up prefixes for
+        // all keys that are not direct children
+        if (!recursiveObjectList) {
+          if (isDescendant(request.getPrefix(), key, true)) {
+            addPrefixIfNotPresent(outputPrefixes, request.getPrefix(), key);
+          }
+        }
+      } else {
+        // Clean up any expired entries
+        delayedDeletes.remove(key);
+      }
+    }
+
+    return new CustomObjectListing(rawListing, outputList, outputPrefixes);
+  }
+
+  private ObjectListing filterListObjects(ListObjectsRequest request,
+      ObjectListing rawListing) {
+
+    // Filter object listing
+    List<S3ObjectSummary> outputList = new ArrayList<>();
+    for (S3ObjectSummary s : rawListing.getObjectSummaries()) {
+      String key = s.getKey();
+      if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
+        outputList.add(s);
+      }
+    }
+
+    // Filter prefixes (directories)
+    List<String> outputPrefixes = new ArrayList<>();
+    for (String key : rawListing.getCommonPrefixes()) {
+      if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
+        outputPrefixes.add(key);
+      }
+    }
+
+    return new CustomObjectListing(rawListing, outputList, outputPrefixes);
+  }
+
+  private boolean isKeyDelayed(Long enqueueTime, String key) {
+    if (enqueueTime == null) {
+      LOG.debug("no delay for key {}", key);
+      return false;
+    }
+    long currentTime = System.currentTimeMillis();
+    long deadline = enqueueTime + delayKeyMsec;
+    if (currentTime >= deadline) {
+      delayedDeletes.remove(key);
+      LOG.debug("no longer delaying {}", key);
+      return false;
+    } else  {
+      LOG.info("delaying {}", key);
+      return true;
+    }
+  }
+
+  private void registerDeleteObject(String key, String bucket) {
+    if (shouldDelay(key)) {
+      // Record summary so we can add it back for some time post-deletion
+      S3ObjectSummary summary = null;
+      ObjectListing list = listObjects(bucket, key);
+      for (S3ObjectSummary result : list.getObjectSummaries()) {
+        if (result.getKey().equals(key)) {
+          summary = result;
+          break;
+        }
+      }
+      delayedDeletes.put(key, new Delete(System.currentTimeMillis(), summary));
+    }
+  }
+
+  private void registerPutObject(PutObjectRequest req) {
+    String key = req.getKey();
+    if (shouldDelay(key)) {
+      enqueueDelayedPut(key);
+    }
+  }
+
+  /**
+   * Should we delay listing visibility for this key?
+   * @param key key which is being put
+   * @return true if we should delay
+   */
+  private boolean shouldDelay(String key) {
+    boolean delay = key.contains(delayKeySubstring);
+    delay = delay && trueWithProbability(delayKeyProbability);
+    LOG.debug("{} -> {}", key, delay);
+    return delay;
+  }
+
+
+  private boolean trueWithProbability(float p) {
+    return Math.random() < p;
+  }
+
+  /**
+   * Record this key as something that should not become visible in
+   * listObject replies for a while, to simulate eventual list consistency.
+   * @param key key to delay visibility of
+   */
+  private void enqueueDelayedPut(String key) {
+    LOG.debug("delaying put of {}", key);
+    delayedPutKeys.put(key, System.currentTimeMillis());
+  }
+
+  /** Since ObjectListing is immutable, we just override it with wrapper. */
+  private static class CustomObjectListing extends ObjectListing {
+
+    private final List<S3ObjectSummary> customListing;
+    private final List<String> customPrefixes;
+
+    CustomObjectListing(ObjectListing rawListing,
+        List<S3ObjectSummary> customListing,
+        List<String> customPrefixes) {
+      super();
+      this.customListing = customListing;
+      this.customPrefixes = customPrefixes;
+
+      this.setBucketName(rawListing.getBucketName());
+      this.setCommonPrefixes(rawListing.getCommonPrefixes());
+      this.setDelimiter(rawListing.getDelimiter());
+      this.setEncodingType(rawListing.getEncodingType());
+      this.setMarker(rawListing.getMarker());
+      this.setMaxKeys(rawListing.getMaxKeys());
+      this.setNextMarker(rawListing.getNextMarker());
+      this.setPrefix(rawListing.getPrefix());
+      this.setTruncated(rawListing.isTruncated());
+    }
+
+    @Override
+    public List<S3ObjectSummary> getObjectSummaries() {
+      return customListing;
+    }
+
+    @Override
+    public List<String> getCommonPrefixes() {
+      return customPrefixes;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
new file mode 100644
index 0000000..17d268b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.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.fs.s3a;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * S3 Client factory used for testing with eventual consistency fault injection.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
+
+  @Override
+  protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
+      ClientConfiguration awsConf) {
+    LOG.warn("** FAILURE INJECTION ENABLED.  Do not run in production! **");
+    return new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
index 30d8e6f..8efa218 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -22,18 +22,25 @@ import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+
+import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.NoSuchElementException;
+import java.util.Set;
 
 import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
 import static org.apache.hadoop.fs.s3a.S3AUtils.createFileStatus;
@@ -54,6 +61,43 @@ public class Listing {
   }
 
   /**
+   * Create a FileStatus iterator against a provided list of file status, with
+   * a given status filter.
+   *
+   * @param fileStatuses the provided list of file status. NO remote calls.
+   * @param filter file path filter on which paths to accept
+   * @param acceptor the file status acceptor
+   * @return the file status iterator
+   */
+  ProvidedFileStatusIterator createProvidedFileStatusIterator(
+      FileStatus[] fileStatuses,
+      PathFilter filter,
+      FileStatusAcceptor acceptor) {
+    return new ProvidedFileStatusIterator(fileStatuses, filter, acceptor);
+  }
+
+  /**
+   * Create a FileStatus iterator against a path, with a given list object
+   * request.
+   *
+   * @param listPath path of the listing
+   * @param request initial request to make
+   * @param filter the filter on which paths to accept
+   * @param acceptor the class/predicate to decide which entries to accept
+   * in the listing based on the full file status.
+   * @return the iterator
+   * @throws IOException IO Problems
+   */
+  FileStatusListingIterator createFileStatusListingIterator(
+      Path listPath,
+      ListObjectsRequest request,
+      PathFilter filter,
+      Listing.FileStatusAcceptor acceptor) throws IOException {
+    return createFileStatusListingIterator(listPath, request, filter, acceptor,
+        null);
+  }
+
+  /**
    * Create a FileStatus iterator against a path, with a given
    * list object request.
    * @param listPath path of the listing
@@ -61,6 +105,8 @@ public class Listing {
    * @param filter the filter on which paths to accept
    * @param acceptor the class/predicate to decide which entries to accept
    * in the listing based on the full file status.
+   * @param providedStatus the provided list of file status, which may contain
+   *                       items that are not listed from source.
    * @return the iterator
    * @throws IOException IO Problems
    */
@@ -68,11 +114,13 @@ public class Listing {
       Path listPath,
       ListObjectsRequest request,
       PathFilter filter,
-      Listing.FileStatusAcceptor acceptor) throws IOException {
+      Listing.FileStatusAcceptor acceptor,
+      RemoteIterator<FileStatus> providedStatus) throws IOException {
     return new FileStatusListingIterator(
         new ObjectListingIterator(listPath, request),
         filter,
-        acceptor);
+        acceptor,
+        providedStatus);
   }
 
   /**
@@ -80,12 +128,27 @@ public class Listing {
    * @param statusIterator an iterator over the remote status entries
    * @return a new remote iterator
    */
+  @VisibleForTesting
   LocatedFileStatusIterator createLocatedFileStatusIterator(
       RemoteIterator<FileStatus> statusIterator) {
     return new LocatedFileStatusIterator(statusIterator);
   }
 
   /**
+   * Create an located status iterator that wraps another to filter out a set
+   * of recently deleted items.
+   * @param iterator an iterator over the remote located status entries.
+   * @param tombstones set of paths that are recently deleted and should be
+   *                   filtered.
+   * @return a new remote iterator.
+   */
+  @VisibleForTesting
+  TombstoneReconcilingIterator createTombstoneReconcilingIterator(
+      RemoteIterator<LocatedFileStatus> iterator, Set<Path> tombstones) {
+    return new TombstoneReconcilingIterator(iterator, tombstones);
+  }
+
+  /**
    * Interface to implement by the logic deciding whether to accept a summary
    * entry or path as a valid file or directory.
    */
@@ -108,6 +171,13 @@ public class Listing {
      * should be generated.)
      */
     boolean accept(Path keyPath, String commonPrefix);
+
+    /**
+     * Predicate to decide whether or not to accept a file status.
+     * @param status file status containing file path information
+     * @return true if the status is accepted else false
+     */
+    boolean accept(FileStatus status);
   }
 
   /**
@@ -115,9 +185,9 @@ public class Listing {
    * value.
    *
    * If the status value is null, the iterator declares that it has no data.
-   * This iterator is used to handle {@link listStatus()} calls where the path
-   * handed in refers to a file, not a directory: this is the iterator
-   * returned.
+   * This iterator is used to handle {@link S3AFileSystem#listStatus} calls
+   * where the path handed in refers to a file, not a directory: this is the
+   * iterator returned.
    */
   static final class SingleStatusRemoteIterator
       implements RemoteIterator<LocatedFileStatus> {
@@ -169,6 +239,47 @@ public class Listing {
   }
 
   /**
+   * This wraps up a provided non-null list of file status as a remote iterator.
+   *
+   * It firstly filters the provided list and later {@link #next} call will get
+   * from the filtered list. This suffers from scalability issues if the
+   * provided list is too large.
+   *
+   * There is no remote data to fetch.
+   */
+  static class ProvidedFileStatusIterator
+      implements RemoteIterator<FileStatus> {
+    private final ArrayList<FileStatus> filteredStatusList;
+    private int index = 0;
+
+    ProvidedFileStatusIterator(FileStatus[] fileStatuses, PathFilter filter,
+        FileStatusAcceptor acceptor) {
+      Preconditions.checkArgument(fileStatuses != null, "Null status list!");
+
+      filteredStatusList = new ArrayList<>(fileStatuses.length);
+      for (FileStatus status : fileStatuses) {
+        if (filter.accept(status.getPath()) && acceptor.accept(status)) {
+          filteredStatusList.add(status);
+        }
+      }
+      filteredStatusList.trimToSize();
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      return index < filteredStatusList.size();
+    }
+
+    @Override
+    public FileStatus next() throws IOException {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      return filteredStatusList.get(index++);
+    }
+  }
+
+  /**
    * Wraps up object listing into a remote iterator which will ask for more
    * listing data if needed.
    *
@@ -179,7 +290,7 @@ public class Listing {
    * iterator can declare that there is more data available.
    *
    * The need to filter the results precludes the iterator from simply
-   * declaring that if the {@link S3AFileSystem.ObjectListingIterator#hasNext()}
+   * declaring that if the {@link ObjectListingIterator#hasNext()}
    * is true then there are more results. Instead the next batch of results must
    * be retrieved and filtered.
    *
@@ -208,20 +319,33 @@ public class Listing {
     /** Iterator over the current set of results. */
     private ListIterator<FileStatus> statusBatchIterator;
 
+    private final Set<FileStatus> providedStatus;
+    private Iterator<FileStatus> providedStatusIterator;
+
     /**
      * Create an iterator over file status entries.
      * @param source the listing iterator from a listObjects call.
      * @param filter the filter on which paths to accept
      * @param acceptor the class/predicate to decide which entries to accept
      * in the listing based on the full file status.
+     * @param providedStatus the provided list of file status, which may contain
+     *                       items that are not listed from source.
      * @throws IOException IO Problems
      */
     FileStatusListingIterator(ObjectListingIterator source,
         PathFilter filter,
-        FileStatusAcceptor acceptor) throws IOException {
+        FileStatusAcceptor acceptor,
+        RemoteIterator<FileStatus> providedStatus) throws IOException {
       this.source = source;
       this.filter = filter;
       this.acceptor = acceptor;
+      this.providedStatus = new HashSet<>();
+      for (; providedStatus != null && providedStatus.hasNext();) {
+        final FileStatus status = providedStatus.next();
+        if (filter.accept(status.getPath()) && acceptor.accept(status)) {
+          this.providedStatus.add(status);
+        }
+      }
       // build the first set of results. This will not trigger any
       // remote IO, assuming the source iterator is in its initial
       // iteration
@@ -233,26 +357,53 @@ public class Listing {
      * If there is data in the local filtered list, return true.
      * Else: request more data util that condition is met, or there
      * is no more remote listing data.
+     * Lastly, return true if the {@code providedStatusIterator}
+     * has left items.
      * @return true if a call to {@link #next()} will succeed.
      * @throws IOException
      */
     @Override
     public boolean hasNext() throws IOException {
-      return statusBatchIterator.hasNext() || requestNextBatch();
+      return sourceHasNext() || providedStatusIterator.hasNext();
+    }
+
+    private boolean sourceHasNext() throws IOException {
+      if (statusBatchIterator.hasNext() || requestNextBatch()) {
+        return true;
+      } else {
+        // turn to file status that are only in provided list
+        if (providedStatusIterator == null) {
+          LOG.debug("Start iterating the provided status.");
+          providedStatusIterator = providedStatus.iterator();
+        }
+        return false;
+      }
     }
 
     @Override
     public FileStatus next() throws IOException {
-      if (!hasNext()) {
-        throw new NoSuchElementException();
+      final FileStatus status;
+      if (sourceHasNext()) {
+        status = statusBatchIterator.next();
+        // We remove from provided list the file status listed by S3 so that
+        // this does not return duplicate items.
+        LOG.debug("Removing the status from provided file status {}", status);
+        providedStatus.remove(status);
+      } else {
+        if (providedStatusIterator.hasNext()) {
+          status = providedStatusIterator.next();
+          LOG.debug("Returning provided file status {}", status);
+        } else {
+          throw new NoSuchElementException();
+        }
       }
-      return statusBatchIterator.next();
+      return status;
     }
 
     /**
      * Try to retrieve another batch.
      * Note that for the initial batch,
-     * {@link S3AFileSystem.ObjectListingIterator} does not generate a request;
+     * {@link ObjectListingIterator} does not generate a request;
      * it simply returns the initial set.
      *
      * @return true if a new batch was created.
@@ -312,7 +463,7 @@ public class Listing {
       for (String prefix : objects.getCommonPrefixes()) {
         Path keyPath = owner.keyToQualifiedPath(prefix);
         if (acceptor.accept(keyPath, prefix) && filter.accept(keyPath)) {
-          FileStatus status = new S3AFileStatus(false, keyPath,
+          FileStatus status = new S3AFileStatus(Tristate.FALSE, keyPath,
               owner.getUsername());
           LOG.debug("Adding directory: {}", status);
           added++;
@@ -352,7 +503,7 @@ public class Listing {
    * instance.
    *
    * 2. Second and later invocations will continue the ongoing listing,
-   * calling {@link #continueListObjects(ObjectListing)} to request the next
+   * calling {@link S3AFileSystem#continueListObjects} to request the next
    * batch of results.
    *
    * 3. The {@link #hasNext()} predicate returns true for the initial call,
@@ -504,6 +655,11 @@ public class Listing {
     public boolean accept(Path keyPath, String prefix) {
       return false;
     }
+
+    @Override
+    public boolean accept(FileStatus status) {
+      return (status != null) && status.isFile();
+    }
   }
 
   /**
@@ -534,6 +690,80 @@ public class Listing {
   }
 
   /**
+   * Wraps another iterator and filters out files that appear in the provided
+   * set of tombstones.  Will read ahead in the iterator when necessary to
+   * ensure that emptiness is detected early enough if only deleted objects
+   * remain in the source iterator.
+   */
+  static class TombstoneReconcilingIterator implements
+      RemoteIterator<LocatedFileStatus> {
+    private LocatedFileStatus next = null;
+    private final RemoteIterator<LocatedFileStatus> iterator;
+    private final Set<Path> tombstones;
+
+    /**
+     * @param iterator Source iterator to filter
+     * @param tombstones set of tombstone markers to filter out of results
+     */
+    TombstoneReconcilingIterator(RemoteIterator<LocatedFileStatus>
+        iterator, Set<Path> tombstones) {
+      this.iterator = iterator;
+      if (tombstones != null) {
+        this.tombstones = tombstones;
+      } else {
+        this.tombstones = Collections.EMPTY_SET;
+      }
+    }
+
+    private boolean fetch() throws IOException {
+      while (next == null && iterator.hasNext()) {
+        LocatedFileStatus candidate = iterator.next();
+        if (!tombstones.contains(candidate.getPath())) {
+          next = candidate;
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public boolean hasNext() throws IOException {
+      if (next != null) {
+        return true;
+      }
+      return fetch();
+    }
+
+    public LocatedFileStatus next() throws IOException {
+      if (hasNext()) {
+        LocatedFileStatus result = next;
+        next = null;
+        fetch();
+        return result;
+      }
+      throw new NoSuchElementException();
+    }
+  }
+
+  /**
+   * Accept all entries except those which map to S3N pseudo directory markers.
+   */
+  static class AcceptAllButS3nDirs implements FileStatusAcceptor {
+
+    public boolean accept(Path keyPath, S3ObjectSummary summary) {
+      return !summary.getKey().endsWith(S3N_FOLDER_SUFFIX);
+    }
+
+    public boolean accept(Path keyPath, String prefix) {
+      return !keyPath.toString().endsWith(S3N_FOLDER_SUFFIX);
+    }
+
+    public boolean accept(FileStatus status) {
+      return !status.getPath().toString().endsWith(S3N_FOLDER_SUFFIX);
+    }
+
+  }
+
+  /**
    * Accept all entries except the base path and those which map to S3N
    * pseudo directory markers.
    */
@@ -575,6 +805,11 @@ public class Listing {
     public boolean accept(Path keyPath, String prefix) {
       return !keyPath.equals(qualifiedPath);
     }
+
+    @Override
+    public boolean accept(FileStatus status) {
+      return (status != null) && !status.getPath().equals(qualifiedPath);
+    }
   }
 
   /**


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


[13/48] hadoop git commit: HADOOP-13345 HS3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
new file mode 100644
index 0000000..8515bfb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
@@ -0,0 +1,304 @@
+/*
+ * 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.s3a.s3guard;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.KeyAttribute;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.KeyType;
+import com.amazonaws.services.dynamodbv2.model.ScalarAttributeType;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * Defines methods for translating between domain model objects and their
+ * representations in the DynamoDB schema.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+final class PathMetadataDynamoDBTranslation {
+
+  /** The HASH key name of each item. */
+  @VisibleForTesting
+  static final String PARENT = "parent";
+  /** The RANGE key name of each item. */
+  @VisibleForTesting
+  static final String CHILD = "child";
+  @VisibleForTesting
+  static final String IS_DIR = "is_dir";
+  @VisibleForTesting
+  static final String MOD_TIME = "mod_time";
+  @VisibleForTesting
+  static final String FILE_LENGTH = "file_length";
+  @VisibleForTesting
+  static final String BLOCK_SIZE = "block_size";
+  static final String IS_DELETED = "is_deleted";
+
+  /** Table version field {@value} in version marker item. */
+  @VisibleForTesting
+  static final String TABLE_VERSION = "table_version";
+
+  /** Table creation timestampfield {@value} in version marker item. */
+  @VisibleForTesting
+  static final String TABLE_CREATED = "table_created";
+
+  /** The version marker field is invalid. */
+  static final String E_NOT_VERSION_MARKER = "Not a version marker: ";
+
+  /**
+   * Returns the key schema for the DynamoDB table.
+   *
+   * @return DynamoDB key schema
+   */
+  static Collection<KeySchemaElement> keySchema() {
+    return Arrays.asList(
+        new KeySchemaElement(PARENT, KeyType.HASH),
+        new KeySchemaElement(CHILD, KeyType.RANGE));
+  }
+
+  /**
+   * Returns the attribute definitions for the DynamoDB table.
+   *
+   * @return DynamoDB attribute definitions
+   */
+  static Collection<AttributeDefinition> attributeDefinitions() {
+    return Arrays.asList(
+        new AttributeDefinition(PARENT, ScalarAttributeType.S),
+        new AttributeDefinition(CHILD, ScalarAttributeType.S));
+  }
+
+  /**
+   * Converts a DynamoDB item to a {@link PathMetadata}.
+   *
+   * @param item DynamoDB item to convert
+   * @return {@code item} converted to a {@link PathMetadata}
+   */
+  static PathMetadata itemToPathMetadata(Item item, String username)
+      throws IOException {
+    if (item == null) {
+      return null;
+    }
+
+    String parentStr = item.getString(PARENT);
+    Preconditions.checkNotNull(parentStr, "No parent entry in item %s", item);
+    String childStr = item.getString(CHILD);
+    Preconditions.checkNotNull(childStr, "No child entry in item %s", item);
+
+    // Skip table version markers, which are only non-absolute paths stored.
+    Path rawPath = new Path(parentStr, childStr);
+    if (!rawPath.isAbsoluteAndSchemeAuthorityNull()) {
+      return null;
+    }
+
+    Path parent = new Path(Constants.FS_S3A + ":/" + parentStr + "/");
+    Path path = new Path(parent, childStr);
+
+    boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR);
+    final FileStatus fileStatus;
+    if (isDir) {
+      fileStatus = DynamoDBMetadataStore.makeDirStatus(path, username);
+    } else {
+      long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0;
+      long modTime = item.hasAttribute(MOD_TIME) ? item.getLong(MOD_TIME) : 0;
+      long block = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0;
+      fileStatus = new FileStatus(len, false, 1, block, modTime, 0, null,
+          username, username, path);
+    }
+    boolean isDeleted =
+        item.hasAttribute(IS_DELETED) && item.getBoolean(IS_DELETED);
+
+    return new PathMetadata(fileStatus, Tristate.UNKNOWN, isDeleted);
+  }
+
+  /**
+   * Converts a {@link PathMetadata} to a DynamoDB item.
+   *
+   * @param meta {@link PathMetadata} to convert
+   * @return {@code meta} converted to DynamoDB item
+   */
+  static Item pathMetadataToItem(PathMetadata meta) {
+    Preconditions.checkNotNull(meta);
+    final FileStatus status = meta.getFileStatus();
+    final Item item = new Item().withPrimaryKey(pathToKey(status.getPath()));
+    if (status.isDirectory()) {
+      item.withBoolean(IS_DIR, true);
+    } else {
+      item.withLong(FILE_LENGTH, status.getLen())
+          .withLong(MOD_TIME, status.getModificationTime())
+          .withLong(BLOCK_SIZE, status.getBlockSize());
+    }
+    item.withBoolean(IS_DELETED, meta.isDeleted());
+    return item;
+  }
+
+  /**
+   * The version marker has a primary key whose PARENT is {@code name};
+   * this MUST NOT be a value which represents an absolute path.
+   * @param name name of the version marker
+   * @param version version number
+   * @param timestamp creation timestamp
+   * @return an item representing a version marker.
+   */
+  static Item createVersionMarker(String name, int version, long timestamp) {
+    return new Item().withPrimaryKey(createVersionMarkerPrimaryKey(name))
+        .withInt(TABLE_VERSION, version)
+        .withLong(TABLE_CREATED, timestamp);
+  }
+
+  /**
+   * Create the primary key of the version marker.
+   * @param name key name
+   * @return the key to use when registering or resolving version markers
+   */
+  static PrimaryKey createVersionMarkerPrimaryKey(String name) {
+    return new PrimaryKey(PARENT, name, CHILD, name);
+  }
+
+  /**
+   * Extract the version from a version marker item.
+   * @param marker version marker item
+   * @return the extracted version field
+   * @throws IOException if the item is not a version marker
+   */
+  static int extractVersionFromMarker(Item marker) throws IOException {
+    if (marker.hasAttribute(TABLE_VERSION)) {
+      return marker.getInt(TABLE_VERSION);
+    } else {
+      throw new IOException(E_NOT_VERSION_MARKER + marker);
+    }
+  }
+
+  /**
+   * Extract the creation time, if present.
+   * @param marker version marker item
+   * @return the creation time, or null
+   * @throws IOException if the item is not a version marker
+   */
+  static Long extractCreationTimeFromMarker(Item marker) throws IOException {
+    if (marker.hasAttribute(TABLE_CREATED)) {
+      return marker.getLong(TABLE_CREATED);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Converts a collection {@link PathMetadata} to a collection DynamoDB items.
+   *
+   * @see #pathMetadataToItem(PathMetadata)
+   */
+  static Item[] pathMetadataToItem(Collection<PathMetadata> metas) {
+    if (metas == null) {
+      return null;
+    }
+
+    final Item[] items = new Item[metas.size()];
+    int i = 0;
+    for (PathMetadata meta : metas) {
+      items[i++] = pathMetadataToItem(meta);
+    }
+    return items;
+  }
+
+  /**
+   * Converts a {@link Path} to a DynamoDB equality condition on that path as
+   * parent, suitable for querying all direct children of the path.
+   *
+   * @param path the path; can not be null
+   * @return DynamoDB equality condition on {@code path} as parent
+   */
+  static KeyAttribute pathToParentKeyAttribute(Path path) {
+    return new KeyAttribute(PARENT, pathToParentKey(path));
+  }
+
+  /**
+   * e.g. {@code pathToParentKey(s3a://bucket/path/a) -> /bucket/path/a}
+   * @param path path to convert
+   * @return string for parent key
+   */
+  static String pathToParentKey(Path path) {
+    Preconditions.checkNotNull(path);
+    Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute");
+    URI uri = path.toUri();
+    String bucket = uri.getHost();
+    Preconditions.checkArgument(!StringUtils.isEmpty(bucket),
+        "Path missing bucket");
+    String pKey = "/" + bucket + uri.getPath();
+
+    // Strip trailing slash
+    if (pKey.endsWith("/")) {
+      pKey = pKey.substring(0, pKey.length() - 1);
+    }
+    return pKey;
+  }
+
+  /**
+   * Converts a {@link Path} to a DynamoDB key, suitable for getting the item
+   * matching the path.
+   *
+   * @param path the path; can not be null
+   * @return DynamoDB key for item matching {@code path}
+   */
+  static PrimaryKey pathToKey(Path path) {
+    Preconditions.checkArgument(!path.isRoot(),
+        "Root path is not mapped to any PrimaryKey");
+    return new PrimaryKey(PARENT, pathToParentKey(path.getParent()), CHILD,
+        path.getName());
+  }
+
+  /**
+   * Converts a collection of {@link Path} to a collection of DynamoDB keys.
+   *
+   * @see #pathToKey(Path)
+   */
+  static PrimaryKey[] pathToKey(Collection<Path> paths) {
+    if (paths == null) {
+      return null;
+    }
+
+    final PrimaryKey[] keys = new PrimaryKey[paths.size()];
+    int i = 0;
+    for (Path p : paths) {
+      keys[i++] = pathToKey(p);
+    }
+    return keys;
+  }
+
+  /**
+   * There is no need to instantiate this class.
+   */
+  private PathMetadataDynamoDBTranslation() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
new file mode 100644
index 0000000..7e4aec1
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.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.fs.s3a.s3guard;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
+import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_PUT_PATH_LATENCY;
+import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_PUT_PATH_REQUEST;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createUploadFileStatus;
+
+/**
+ * Logic for integrating MetadataStore with S3A.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class S3Guard {
+  private static final Logger LOG = LoggerFactory.getLogger(S3Guard.class);
+
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  @VisibleForTesting
+  public static final String S3GUARD_DDB_CLIENT_FACTORY_IMPL =
+      "fs.s3a.s3guard.ddb.client.factory.impl";
+
+  static final Class<? extends DynamoDBClientFactory>
+      S3GUARD_DDB_CLIENT_FACTORY_IMPL_DEFAULT =
+      DynamoDBClientFactory.DefaultDynamoDBClientFactory.class;
+  private static final FileStatus[] EMPTY_LISTING = new FileStatus[0];
+
+  // Utility class.  All static functions.
+  private S3Guard() { }
+
+  /* Utility functions. */
+
+  /**
+   * Create a new instance of the configured MetadataStore.
+   * The returned MetadataStore will have been initialized via
+   * {@link MetadataStore#initialize(FileSystem)} by this function before
+   * returning it.  Callers must clean up by calling
+   * {@link MetadataStore#close()} when done using the MetadataStore.
+   *
+   * @param fs  FileSystem whose Configuration specifies which
+   *            implementation to use.
+   * @return Reference to new MetadataStore.
+   * @throws IOException if the metadata store cannot be instantiated
+   */
+  public static MetadataStore getMetadataStore(FileSystem fs)
+      throws IOException {
+    Preconditions.checkNotNull(fs);
+    Configuration conf = fs.getConf();
+    Preconditions.checkNotNull(conf);
+    MetadataStore msInstance;
+    try {
+      Class<? extends MetadataStore> msClass = getMetadataStoreClass(conf);
+      msInstance = ReflectionUtils.newInstance(msClass, conf);
+      LOG.debug("Using {} metadata store for {} filesystem",
+          msClass.getSimpleName(), fs.getScheme());
+      msInstance.initialize(fs);
+      return msInstance;
+    } catch (RuntimeException | IOException e) {
+      String message = "Failed to instantiate metadata store " +
+          conf.get(S3_METADATA_STORE_IMPL)
+          + " defined in " + S3_METADATA_STORE_IMPL
+          + ": " + e;
+      LOG.error(message, e);
+      if (e instanceof IOException) {
+        throw e;
+      } else {
+        throw new IOException(message, e);
+      }
+    }
+  }
+
+  private static Class<? extends MetadataStore> getMetadataStoreClass(
+      Configuration conf) {
+    if (conf == null) {
+      return NullMetadataStore.class;
+    }
+
+    return conf.getClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+            MetadataStore.class);
+  }
+
+
+  /**
+   * Helper function which puts a given S3AFileStatus into the MetadataStore and
+   * returns the same S3AFileStatus. Instrumentation monitors the put operation.
+   * @param ms MetadataStore to {@code put()} into.
+   * @param status status to store
+   * @param instrumentation instrumentation of the s3a file system
+   * @return The same status as passed in
+   * @throws IOException if metadata store update failed
+   */
+  public static S3AFileStatus putAndReturn(MetadataStore ms,
+      S3AFileStatus status,
+      S3AInstrumentation instrumentation) throws IOException {
+    long startTimeNano = System.nanoTime();
+    ms.put(new PathMetadata(status));
+    instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
+        (System.nanoTime() - startTimeNano));
+    instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
+    return status;
+  }
+
+  /**
+   * Convert the data of a directory listing to an array of {@link FileStatus}
+   * entries. Tombstones are filtered out at this point. If the listing is null
+   * an empty array is returned.
+   * @param dirMeta directory listing -may be null
+   * @return a possibly-empty array of file status entries
+   */
+  public static FileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta)  {
+    if (dirMeta == null) {
+      return EMPTY_LISTING;
+    }
+
+    Collection<PathMetadata> listing = dirMeta.getListing();
+    List<FileStatus> statuses = new ArrayList<>();
+
+    for (PathMetadata pm : listing) {
+      if (!pm.isDeleted()) {
+        statuses.add(pm.getFileStatus());
+      }
+    }
+
+    return statuses.toArray(new FileStatus[0]);
+  }
+
+  /**
+   * Given directory listing metadata from both the backing store and the
+   * MetadataStore, merge the two sources of truth to create a consistent
+   * view of the current directory contents, which can be returned to clients.
+   *
+   * Also update the MetadataStore to reflect the resulting directory listing.
+   *
+   * @param ms MetadataStore to use.
+   * @param path path to directory
+   * @param backingStatuses Directory listing from the backing store.
+   * @param dirMeta  Directory listing from MetadataStore.  May be null.
+   * @param isAuthoritative State of authoritative mode
+   * @return Final result of directory listing.
+   * @throws IOException if metadata store update failed
+   */
+  public static FileStatus[] dirListingUnion(MetadataStore ms, Path path,
+      List<FileStatus> backingStatuses, DirListingMetadata dirMeta,
+      boolean isAuthoritative) throws IOException {
+
+    // Fast-path for NullMetadataStore
+    if (isNullMetadataStore(ms)) {
+      return backingStatuses.toArray(new FileStatus[backingStatuses.size()]);
+    }
+
+    assertQualified(path);
+
+    if (dirMeta == null) {
+      // The metadataStore had zero state for this directory
+      dirMeta = new DirListingMetadata(path, DirListingMetadata.EMPTY_DIR,
+          false);
+    }
+
+    Set<Path> deleted = dirMeta.listTombstones();
+
+    // Since we treat the MetadataStore as a "fresher" or "consistent" view
+    // of metadata, we always use its metadata first.
+
+    // Since the authoritative case is already handled outside this function,
+    // we will basically start with the set of directory entries in the
+    // DirListingMetadata, and add any that only exist in the backingStatuses.
+
+    boolean changed = false;
+    for (FileStatus s : backingStatuses) {
+      if (deleted.contains(s.getPath())) {
+        continue;
+      }
+
+      // Minor race condition here.  Multiple threads could add to this
+      // mutable DirListingMetadata.  Since it is backed by a
+      // ConcurrentHashMap, the last put() wins.
+      // More concerning is two threads racing on listStatus() and delete().
+      // Any FileSystem has similar race conditions, but we could persist
+      // a stale entry longer.  We could expose an atomic
+      // DirListingMetadata#putIfNotPresent()
+      boolean updated = dirMeta.put(s);
+      changed = changed || updated;
+    }
+
+    if (changed && isAuthoritative) {
+      dirMeta.setAuthoritative(true); // This is the full directory contents
+      ms.put(dirMeta);
+    }
+
+    return dirMetaToStatuses(dirMeta);
+  }
+
+  /**
+   * Although NullMetadataStore does nothing, callers may wish to avoid work
+   * (fast path) when the NullMetadataStore is in use.
+   * @param ms The MetadataStore to test
+   * @return true iff the MetadataStore is the null, or no-op, implementation.
+   */
+  public static boolean isNullMetadataStore(MetadataStore ms) {
+    return (ms instanceof NullMetadataStore);
+  }
+
+  /**
+   * Update MetadataStore to reflect creation of the given  directories.
+   *
+   * If an IOException is raised while trying to update the entry, this
+   * operation catches the exception and returns.
+   * @param ms    MetadataStore to update.
+   * @param dirs  null, or an ordered list of directories from leaf to root.
+   *              E.g. if /a/ exists, and  mkdirs(/a/b/c/d) is called, this
+   *              list will contain [/a/b/c/d, /a/b/c, /a/b].   /a/b/c/d is
+   *              an empty, dir, and the other dirs only contain their child
+   *              dir.
+   * @param owner Hadoop user name.
+   * @param authoritative Whether to mark new directories as authoritative.
+   */
+  public static void makeDirsOrdered(MetadataStore ms, List<Path> dirs,
+      String owner, boolean authoritative) {
+    if (dirs == null) {
+      return;
+    }
+
+    /* We discussed atomicity of this implementation.
+     * The concern is that multiple clients could race to write different
+     * cached directories to the MetadataStore.  Two solutions are proposed:
+     * 1. Move mkdirs() into MetadataStore interface and let implementations
+     *    ensure they are atomic.
+     * 2. Specify that the semantics of MetadataStore#putListStatus() is
+     *    always additive,  That is, if MetadataStore has listStatus() state
+     *    for /a/b that contains [/a/b/file0, /a/b/file1], and we then call
+     *    putListStatus(/a/b -> [/a/b/file2, /a/b/file3], isAuthoritative=true),
+     *    then we will end up with final state of
+     *    [/a/b/file0, /a/b/file1, /a/b/file2, /a/b/file3], isAuthoritative =
+     *    true
+     */
+    FileStatus prevStatus = null;
+
+    // Use new batched put to reduce round trips.
+    List<PathMetadata> pathMetas = new ArrayList<>(dirs.size());
+
+    try {
+      // Iterate from leaf to root
+      for (int i = 0; i < dirs.size(); i++) {
+        boolean isLeaf = (prevStatus == null);
+        Path f = dirs.get(i);
+        assertQualified(f);
+        FileStatus status =
+            createUploadFileStatus(f, true, 0, 0, owner);
+
+        // We only need to put a DirListingMetadata if we are setting
+        // authoritative bit
+        DirListingMetadata dirMeta = null;
+        if (authoritative) {
+          Collection<PathMetadata> children;
+          if (isLeaf) {
+            children = DirListingMetadata.EMPTY_DIR;
+          } else {
+            children = new ArrayList<>(1);
+            children.add(new PathMetadata(prevStatus));
+          }
+          dirMeta = new DirListingMetadata(f, children, authoritative);
+          ms.put(dirMeta);
+        }
+
+        pathMetas.add(new PathMetadata(status));
+        prevStatus = status;
+      }
+
+      // Batched put
+      ms.put(pathMetas);
+    } catch (IOException ioe) {
+      LOG.error("MetadataStore#put() failure:", ioe);
+    }
+  }
+
+  /**
+   * Helper function that records the move of directory paths, adding
+   * resulting metadata to the supplied lists.
+   * Does not store in MetadataStore.
+   * @param ms  MetadataStore, used to make this a no-op, when it is
+   *            NullMetadataStore.
+   * @param srcPaths stores the source path here
+   * @param dstMetas stores destination metadata here
+   * @param srcPath  source path to store
+   * @param dstPath  destination path to store
+   * @param owner file owner to use in created records
+   */
+  public static void addMoveDir(MetadataStore ms, Collection<Path> srcPaths,
+      Collection<PathMetadata> dstMetas, Path srcPath, Path dstPath,
+      String owner) {
+    if (isNullMetadataStore(ms)) {
+      return;
+    }
+    assertQualified(srcPath, dstPath);
+
+    FileStatus dstStatus = createUploadFileStatus(dstPath, true, 0, 0, owner);
+    addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
+  }
+
+  /**
+   * Like {@link #addMoveDir(MetadataStore, Collection, Collection, Path,
+   * Path, String)} (), but for files.
+   * @param ms  MetadataStore, used to make this a no-op, when it is
+   *            NullMetadataStore.
+   * @param srcPaths stores the source path here
+   * @param dstMetas stores destination metadata here
+   * @param srcPath  source path to store
+   * @param dstPath  destination path to store
+   * @param size length of file moved
+   * @param blockSize  blocksize to associate with destination file
+   * @param owner file owner to use in created records
+   */
+  public static void addMoveFile(MetadataStore ms, Collection<Path> srcPaths,
+      Collection<PathMetadata> dstMetas, Path srcPath, Path dstPath,
+      long size, long blockSize, String owner) {
+    if (isNullMetadataStore(ms)) {
+      return;
+    }
+    assertQualified(srcPath, dstPath);
+    FileStatus dstStatus = createUploadFileStatus(dstPath, false,
+        size, blockSize, owner);
+    addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
+  }
+
+  /**
+   * Helper method that records the move of all ancestors of a path.
+   *
+   * In S3A, an optimization is to delete unnecessary fake directory objects if
+   * the directory is non-empty. In that case, for a nested child to move, S3A
+   * is not listing and thus moving all its ancestors (up to source root). So we
+   * take care of those inferred directories of this path explicitly.
+   *
+   * As {@link #addMoveFile} and {@link #addMoveDir}, this method adds resulting
+   * metadata to the supplied lists. It does not store in MetadataStore.
+   *
+   * @param ms MetadataStore, no-op if it is NullMetadataStore
+   * @param srcPaths stores the source path here
+   * @param dstMetas stores destination metadata here
+   * @param srcRoot source root up to which (exclusive) should we add ancestors
+   * @param srcPath source path of the child to add ancestors
+   * @param dstPath destination path of the child to add ancestors
+   * @param owner Hadoop user name
+   */
+  public static void addMoveAncestors(MetadataStore ms,
+      Collection<Path> srcPaths, Collection<PathMetadata> dstMetas,
+      Path srcRoot, Path srcPath, Path dstPath, String owner) {
+    if (isNullMetadataStore(ms)) {
+      return;
+    }
+
+    assertQualified(srcRoot, srcPath, dstPath);
+
+    if (srcPath.equals(srcRoot)) {
+      LOG.debug("Skip moving ancestors of source root directory {}", srcRoot);
+      return;
+    }
+
+    Path parentSrc = srcPath.getParent();
+    Path parentDst = dstPath.getParent();
+    while (parentSrc != null
+        && !parentSrc.isRoot()
+        && !parentSrc.equals(srcRoot)
+        && !srcPaths.contains(parentSrc)) {
+      LOG.debug("Renaming non-listed parent {} to {}", parentSrc, parentDst);
+      S3Guard.addMoveDir(ms, srcPaths, dstMetas, parentSrc, parentDst, owner);
+      parentSrc = parentSrc.getParent();
+      parentDst = parentDst.getParent();
+    }
+  }
+
+  public static void addAncestors(MetadataStore metadataStore,
+      Path qualifiedPath, String username) throws IOException {
+    Collection<PathMetadata> newDirs = new ArrayList<>();
+    Path parent = qualifiedPath.getParent();
+    while (!parent.isRoot()) {
+      PathMetadata directory = metadataStore.get(parent);
+      if (directory == null || directory.isDeleted()) {
+        FileStatus status = new FileStatus(0, true, 1, 0, 0, 0, null, username,
+            null, parent);
+        PathMetadata meta = new PathMetadata(status, Tristate.FALSE, false);
+        newDirs.add(meta);
+      } else {
+        break;
+      }
+      parent = parent.getParent();
+    }
+    metadataStore.put(newDirs);
+  }
+
+  private static void addMoveStatus(Collection<Path> srcPaths,
+      Collection<PathMetadata> dstMetas,
+      Path srcPath,
+      FileStatus dstStatus) {
+    srcPaths.add(srcPath);
+    dstMetas.add(new PathMetadata(dstStatus));
+  }
+
+  /**
+   * Assert that the path is qualified with a host and scheme.
+   * @param p path to check
+   * @throws NullPointerException if either argument does not hold
+   */
+  public static void assertQualified(Path p) {
+    URI uri = p.toUri();
+    // Paths must include bucket in case MetadataStore is shared between
+    // multiple S3AFileSystem instances
+    Preconditions.checkNotNull(uri.getHost(), "Null host in " + uri);
+
+    // This should never fail, but is retained for completeness.
+    Preconditions.checkNotNull(uri.getScheme(), "Null scheme in " + uri);
+  }
+
+  /**
+   * Assert that all paths are valid.
+   * @param paths path to check
+   * @throws NullPointerException if either argument does not hold
+   */
+  public static void assertQualified(Path...paths) {
+    for (Path path : paths) {
+      assertQualified(path);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
new file mode 100644
index 0000000..be271ae
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -0,0 +1,924 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * CLI to manage S3Guard Metadata Store.
+ */
+public abstract class S3GuardTool extends Configured implements Tool {
+  private static final Logger LOG = LoggerFactory.getLogger(S3GuardTool.class);
+
+  private static final String NAME = "s3guard";
+  private static final String COMMON_USAGE =
+      "When possible and not overridden by more specific options, metadata\n" +
+      "repository information will be inferred from the S3A URL (if provided)" +
+      "\n\n" +
+      "Generic options supported are:\n" +
+      "  -conf <config file> - specify an application configuration file\n" +
+      "  -D <property=value> - define a value for a given property\n";
+
+  private static final String USAGE = NAME +
+      " [command] [OPTIONS] [s3a://BUCKET]\n\n" +
+      "Commands: \n" +
+      "\t" + Init.NAME + " - " + Init.PURPOSE + "\n" +
+      "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
+      "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
+      "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
+      "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n";
+  private static final String DATA_IN_S3_IS_PRESERVED
+      = "(all data in S3 is preserved";
+
+  abstract public String getUsage();
+
+  // Exit codes
+  static final int SUCCESS = 0;
+  static final int INVALID_ARGUMENT = 1;
+  static final int ERROR = 99;
+
+  private S3AFileSystem filesystem;
+  private MetadataStore store;
+  private final CommandFormat commandFormat;
+
+  private static final String META_FLAG = "meta";
+  private static final String DAYS_FLAG = "days";
+  private static final String HOURS_FLAG = "hours";
+  private static final String MINUTES_FLAG = "minutes";
+  private static final String SECONDS_FLAG = "seconds";
+
+  private static final String REGION_FLAG = "region";
+  private static final String READ_FLAG = "read";
+  private static final String WRITE_FLAG = "write";
+
+  /**
+   * Constructor a S3Guard tool with HDFS configuration.
+   * @param conf Configuration.
+   */
+  protected S3GuardTool(Configuration conf) {
+    super(conf);
+
+    commandFormat = new CommandFormat(0, Integer.MAX_VALUE);
+    // For metadata store URI
+    commandFormat.addOptionWithValue(META_FLAG);
+    // DDB region.
+    commandFormat.addOptionWithValue(REGION_FLAG);
+  }
+
+  /**
+   * Return sub-command name.
+   */
+  abstract String getName();
+
+  /**
+   * Parse DynamoDB region from either -m option or a S3 path.
+   *
+   * This function should only be called from {@link Init} or
+   * {@link Destroy}.
+   *
+   * @param paths remaining parameters from CLI.
+   * @return false for invalid parameters.
+   * @throws IOException on I/O errors.
+   */
+  boolean parseDynamoDBRegion(List<String> paths) throws IOException {
+    Configuration conf = getConf();
+    String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
+    String fromConf = conf.get(S3GUARD_DDB_REGION_KEY);
+    boolean hasS3Path = !paths.isEmpty();
+
+    if (fromCli != null) {
+      if (fromCli.isEmpty()) {
+        System.err.println("No region provided with -" + REGION_FLAG + " flag");
+        return false;
+      }
+      if (hasS3Path) {
+        System.err.println("Providing both an S3 path and the -" + REGION_FLAG
+            + " flag is not supported. If you need to specify a different "
+            + "region than the S3 bucket, configure " + S3GUARD_DDB_REGION_KEY);
+        return false;
+      }
+      conf.set(S3GUARD_DDB_REGION_KEY, fromCli);
+      return true;
+    }
+
+    if (fromConf != null) {
+      if (fromConf.isEmpty()) {
+        System.err.printf("No region provided with config %s, %n",
+            S3GUARD_DDB_REGION_KEY);
+        return false;
+      }
+      return true;
+    }
+
+    if (hasS3Path) {
+      String s3Path = paths.get(0);
+      initS3AFileSystem(s3Path);
+      return true;
+    }
+
+    System.err.println("No region found from -" + REGION_FLAG + " flag, " +
+        "config, or S3 bucket");
+    return false;
+  }
+
+  /**
+   * Parse metadata store from command line option or HDFS configuration.
+   *
+   * @param forceCreate override the auto-creation setting to true.
+   * @return a initialized metadata store.
+   */
+  MetadataStore initMetadataStore(boolean forceCreate) throws IOException {
+    if (getStore() != null) {
+      return getStore();
+    }
+    Configuration conf;
+    if (filesystem == null) {
+      conf = getConf();
+    } else {
+      conf = filesystem.getConf();
+    }
+    String metaURI = getCommandFormat().getOptValue(META_FLAG);
+    if (metaURI != null && !metaURI.isEmpty()) {
+      URI uri = URI.create(metaURI);
+      LOG.info("create metadata store: {}", uri + " scheme: "
+          + uri.getScheme());
+      switch (uri.getScheme().toLowerCase(Locale.ENGLISH)) {
+      case "local":
+        setStore(new LocalMetadataStore());
+        break;
+      case "dynamodb":
+        setStore(new DynamoDBMetadataStore());
+        conf.set(S3GUARD_DDB_TABLE_NAME_KEY, uri.getAuthority());
+        if (forceCreate) {
+          conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+        }
+        break;
+      default:
+        throw new IOException(
+            String.format("Metadata store %s is not supported", uri));
+      }
+    } else {
+      // CLI does not specify metadata store URI, it uses default metadata store
+      // DynamoDB instead.
+      setStore(new DynamoDBMetadataStore());
+      if (forceCreate) {
+        conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+      }
+    }
+
+    if (filesystem == null) {
+      getStore().initialize(conf);
+    } else {
+      getStore().initialize(filesystem);
+    }
+    LOG.info("Metadata store {} is initialized.", getStore());
+    return getStore();
+  }
+
+  /**
+   * Initialize S3A FileSystem instance.
+   *
+   * @param path s3a URI
+   * @throws IOException
+   */
+  void initS3AFileSystem(String path) throws IOException {
+    URI uri;
+    try {
+      uri = new URI(path);
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+    // Make sure that S3AFileSystem does not hold an actual MetadataStore
+    // implementation.
+    Configuration conf = getConf();
+    conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+        MetadataStore.class);
+    FileSystem fs = FileSystem.get(uri, getConf());
+    if (!(fs instanceof S3AFileSystem)) {
+      throw new IOException(
+          String.format("URI %s is not a S3A file system: %s", uri,
+              fs.getClass().getName()));
+    }
+    filesystem = (S3AFileSystem) fs;
+  }
+
+  /**
+   * Parse CLI arguments and returns the position arguments.
+   * The options are stored in {@link #commandFormat}
+   *
+   * @param args command line arguments.
+   * @return the position arguments from CLI.
+   */
+  List<String> parseArgs(String[] args) {
+    return getCommandFormat().parse(args, 1);
+  }
+
+  protected S3AFileSystem getFilesystem() {
+    return filesystem;
+  }
+
+  protected void setFilesystem(S3AFileSystem filesystem) {
+    this.filesystem = filesystem;
+  }
+
+  @VisibleForTesting
+  public MetadataStore getStore() {
+    return store;
+  }
+
+  @VisibleForTesting
+  protected void setStore(MetadataStore store) {
+    Preconditions.checkNotNull(store);
+    this.store = store;
+  }
+
+  protected CommandFormat getCommandFormat() {
+    return commandFormat;
+  }
+
+  /**
+   * Create the metadata store.
+   */
+  static class Init extends S3GuardTool {
+    private static final String NAME = "init";
+    public static final String PURPOSE = "initialize metadata repository";
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+          "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "  -" + READ_FLAG + " UNIT - Provisioned read throughput units\n" +
+        "  -" + WRITE_FLAG + " UNIT - Provisioned write through put units\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    Init(Configuration conf) {
+      super(conf);
+      // read capacity.
+      getCommandFormat().addOptionWithValue(READ_FLAG);
+      // write capacity.
+      getCommandFormat().addOptionWithValue(WRITE_FLAG);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    @Override
+    public int run(String[] args) throws IOException {
+      List<String> paths = parseArgs(args);
+
+      String readCap = getCommandFormat().getOptValue(READ_FLAG);
+      if (readCap != null && !readCap.isEmpty()) {
+        int readCapacity = Integer.parseInt(readCap);
+        getConf().setInt(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, readCapacity);
+      }
+      String writeCap = getCommandFormat().getOptValue(WRITE_FLAG);
+      if (writeCap != null && !writeCap.isEmpty()) {
+        int writeCapacity = Integer.parseInt(writeCap);
+        getConf().setInt(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, writeCapacity);
+      }
+
+      // Validate parameters.
+      if (!parseDynamoDBRegion(paths)) {
+        System.err.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+      initMetadataStore(true);
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Destroy a metadata store.
+   */
+  static class Destroy extends S3GuardTool {
+    private static final String NAME = "destroy";
+    public static final String PURPOSE = "destroy Metadata Store data "
+        + DATA_IN_S3_IS_PRESERVED;
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+          "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    Destroy(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    public int run(String[] args) throws IOException {
+      List<String> paths = parseArgs(args);
+      if (!parseDynamoDBRegion(paths)) {
+        System.err.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+
+      try {
+        initMetadataStore(false);
+      } catch (FileNotFoundException e) {
+        // indication that the table was not found
+        LOG.debug("Failed to bind to store to be destroyed", e);
+        LOG.info("Metadata Store does not exist.");
+        return SUCCESS;
+      }
+
+      Preconditions.checkState(getStore() != null,
+          "Metadata Store is not initialized");
+
+      getStore().destroy();
+      LOG.info("Metadata store is deleted.");
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Import s3 metadata to the metadata store.
+   */
+  static class Import extends S3GuardTool {
+    private static final String NAME = "import";
+    public static final String PURPOSE = "import metadata from existing S3 " +
+        "data";
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+        "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    private final Set<Path> dirCache = new HashSet<>();
+
+    Import(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    /**
+     * Put parents into MS and cache if the parents are not presented.
+     *
+     * @param f the file or an empty directory.
+     * @throws IOException on I/O errors.
+     */
+    private void putParentsIfNotPresent(FileStatus f) throws IOException {
+      Preconditions.checkNotNull(f);
+      Path parent = f.getPath().getParent();
+      while (parent != null) {
+        if (dirCache.contains(parent)) {
+          return;
+        }
+        FileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent,
+            f.getOwner());
+        getStore().put(new PathMetadata(dir));
+        dirCache.add(parent);
+        parent = parent.getParent();
+      }
+    }
+
+    /**
+     * Recursively import every path under path.
+     * @return number of items inserted into MetadataStore
+     * @throws IOException on I/O errors.
+     */
+    private long importDir(FileStatus status) throws IOException {
+      Preconditions.checkArgument(status.isDirectory());
+      RemoteIterator<LocatedFileStatus> it = getFilesystem()
+          .listFilesAndEmptyDirectories(status.getPath(), true);
+      long items = 0;
+
+      while (it.hasNext()) {
+        LocatedFileStatus located = it.next();
+        FileStatus child;
+        if (located.isDirectory()) {
+          child = DynamoDBMetadataStore.makeDirStatus(located.getPath(),
+              located.getOwner());
+          dirCache.add(child.getPath());
+        } else {
+          child = new S3AFileStatus(located.getLen(),
+              located.getModificationTime(),
+              located.getPath(),
+              located.getBlockSize(),
+              located.getOwner());
+        }
+        putParentsIfNotPresent(child);
+        getStore().put(new PathMetadata(child));
+        items++;
+      }
+      return items;
+    }
+
+    @Override
+    public int run(String[] args) throws IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        System.err.println(getUsage());
+        return INVALID_ARGUMENT;
+      }
+      String s3Path = paths.get(0);
+      initS3AFileSystem(s3Path);
+
+      URI uri;
+      try {
+        uri = new URI(s3Path);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+      String filePath = uri.getPath();
+      if (filePath.isEmpty()) {
+        // If they specify a naked S3 URI (e.g. s3a://bucket), we'll consider
+        // root to be the path
+        filePath = "/";
+      }
+      Path path = new Path(filePath);
+      FileStatus status = getFilesystem().getFileStatus(path);
+
+      initMetadataStore(false);
+
+      long items = 1;
+      if (status.isFile()) {
+        PathMetadata meta = new PathMetadata(status);
+        getStore().put(meta);
+      } else {
+        items = importDir(status);
+      }
+
+      System.out.printf("Inserted %d items into Metadata Store%n", items);
+
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Show diffs between the s3 and metadata store.
+   */
+  static class Diff extends S3GuardTool {
+    private static final String NAME = "diff";
+    public static final String PURPOSE = "report on delta between S3 and " +
+        "repository";
+    private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+        "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    private static final String SEP = "\t";
+    static final String S3_PREFIX = "S3";
+    static final String MS_PREFIX = "MS";
+
+    Diff(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    /**
+     * Formats the output of printing a FileStatus in S3guard diff tool.
+     * @param status the status to print.
+     * @return the string of output.
+     */
+    private static String formatFileStatus(FileStatus status) {
+      return String.format("%s%s%d%s%s",
+          status.isDirectory() ? "D" : "F",
+          SEP,
+          status.getLen(),
+          SEP,
+          status.getPath().toString());
+    }
+
+    /**
+     * Compares metadata from 2 S3 FileStatus's to see if they differ.
+     * @param thisOne
+     * @param thatOne
+     * @return true if the metadata is not identical
+     */
+    private static boolean differ(FileStatus thisOne, FileStatus thatOne) {
+      Preconditions.checkArgument(!(thisOne == null && thatOne == null));
+      return (thisOne == null || thatOne == null) ||
+          (thisOne.getLen() != thatOne.getLen()) ||
+          (thisOne.isDirectory() != thatOne.isDirectory()) ||
+          (!thisOne.isDirectory() &&
+              thisOne.getModificationTime() != thatOne.getModificationTime());
+    }
+
+    /**
+     * Print difference, if any, between two file statuses to the output stream.
+     *
+     * @param msStatus file status from metadata store.
+     * @param s3Status file status from S3.
+     * @param out output stream.
+     */
+    private static void printDiff(FileStatus msStatus,
+                                  FileStatus s3Status,
+                                  PrintStream out) {
+      Preconditions.checkArgument(!(msStatus == null && s3Status == null));
+      if (msStatus != null && s3Status != null) {
+        Preconditions.checkArgument(
+            msStatus.getPath().equals(s3Status.getPath()),
+            String.format("The path from metadata store and s3 are different:" +
+            " ms=%s s3=%s", msStatus.getPath(), s3Status.getPath()));
+      }
+
+      if (differ(msStatus, s3Status)) {
+        if (s3Status != null) {
+          out.printf("%s%s%s%n", S3_PREFIX, SEP, formatFileStatus(s3Status));
+        }
+        if (msStatus != null) {
+          out.printf("%s%s%s%n", MS_PREFIX, SEP, formatFileStatus(msStatus));
+        }
+      }
+    }
+
+    /**
+     * Compare the metadata of the directory with the same path, on S3 and
+     * the metadata store, respectively. If one of them is null, consider the
+     * metadata of the directory and all its subdirectories are missing from
+     * the source.
+     *
+     * Pass the FileStatus obtained from s3 and metadata store to avoid one
+     * round trip to fetch the same metadata twice, because the FileStatus
+     * hve already been obtained from listStatus() / listChildren operations.
+     *
+     * @param msDir the directory FileStatus obtained from the metadata store.
+     * @param s3Dir the directory FileStatus obtained from S3.
+     * @param out the output stream to generate diff results.
+     * @throws IOException on I/O errors.
+     */
+    private void compareDir(FileStatus msDir, FileStatus s3Dir,
+                            PrintStream out) throws IOException {
+      Preconditions.checkArgument(!(msDir == null && s3Dir == null));
+      if (msDir != null && s3Dir != null) {
+        Preconditions.checkArgument(msDir.getPath().equals(s3Dir.getPath()),
+            String.format("The path from metadata store and s3 are different:" +
+             " ms=%s s3=%s", msDir.getPath(), s3Dir.getPath()));
+      }
+
+      Map<Path, FileStatus> s3Children = new HashMap<>();
+      if (s3Dir != null && s3Dir.isDirectory()) {
+        for (FileStatus status : getFilesystem().listStatus(s3Dir.getPath())) {
+          s3Children.put(status.getPath(), status);
+        }
+      }
+
+      Map<Path, FileStatus> msChildren = new HashMap<>();
+      if (msDir != null && msDir.isDirectory()) {
+        DirListingMetadata dirMeta =
+            getStore().listChildren(msDir.getPath());
+
+        if (dirMeta != null) {
+          for (PathMetadata meta : dirMeta.getListing()) {
+            FileStatus status = meta.getFileStatus();
+            msChildren.put(status.getPath(), status);
+          }
+        }
+      }
+
+      Set<Path> allPaths = new HashSet<>(s3Children.keySet());
+      allPaths.addAll(msChildren.keySet());
+
+      for (Path path : allPaths) {
+        FileStatus s3Status = s3Children.get(path);
+        FileStatus msStatus = msChildren.get(path);
+        printDiff(msStatus, s3Status, out);
+        if ((s3Status != null && s3Status.isDirectory()) ||
+            (msStatus != null && msStatus.isDirectory())) {
+          compareDir(msStatus, s3Status, out);
+        }
+      }
+      out.flush();
+    }
+
+    /**
+     * Compare both metadata store and S3 on the same path.
+     *
+     * @param path the path to be compared.
+     * @param out  the output stream to display results.
+     * @throws IOException on I/O errors.
+     */
+    private void compareRoot(Path path, PrintStream out) throws IOException {
+      Path qualified = getFilesystem().qualify(path);
+      FileStatus s3Status = null;
+      try {
+        s3Status = getFilesystem().getFileStatus(qualified);
+      } catch (FileNotFoundException e) {
+      }
+      PathMetadata meta = getStore().get(qualified);
+      FileStatus msStatus = (meta != null && !meta.isDeleted()) ?
+          meta.getFileStatus() : null;
+      compareDir(msStatus, s3Status, out);
+    }
+
+    @VisibleForTesting
+    public int run(String[] args, PrintStream out) throws IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        out.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+      String s3Path = paths.get(0);
+      initS3AFileSystem(s3Path);
+      initMetadataStore(true);
+
+      URI uri;
+      try {
+        uri = new URI(s3Path);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+      Path root;
+      if (uri.getPath().isEmpty()) {
+        root = new Path("/");
+      } else {
+        root = new Path(uri.getPath());
+      }
+      root = getFilesystem().qualify(root);
+      compareRoot(root, out);
+      out.flush();
+      return SUCCESS;
+    }
+
+    @Override
+    public int run(String[] args) throws IOException {
+      return run(args, System.out);
+    }
+  }
+
+  /**
+   * Prune metadata that has not been modified recently.
+   */
+  static class Prune extends S3GuardTool {
+    private static final String NAME = "prune";
+    public static final String PURPOSE = "truncate older metadata from " +
+        "repository "
+        + DATA_IN_S3_IS_PRESERVED;;
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+        "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    Prune(Configuration conf) {
+      super(conf);
+
+      CommandFormat format = getCommandFormat();
+      format.addOptionWithValue(DAYS_FLAG);
+      format.addOptionWithValue(HOURS_FLAG);
+      format.addOptionWithValue(MINUTES_FLAG);
+      format.addOptionWithValue(SECONDS_FLAG);
+    }
+
+    @VisibleForTesting
+    void setMetadataStore(MetadataStore ms) {
+      Preconditions.checkNotNull(ms);
+      this.setStore(ms);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    private long getDeltaComponent(TimeUnit unit, String arg) {
+      String raw = getCommandFormat().getOptValue(arg);
+      if (raw == null || raw.isEmpty()) {
+        return 0;
+      }
+      Long parsed = Long.parseLong(raw);
+      return unit.toMillis(parsed);
+    }
+
+    @VisibleForTesting
+    public int run(String[] args, PrintStream out) throws
+        InterruptedException, IOException {
+      List<String> paths = parseArgs(args);
+      if (!parseDynamoDBRegion(paths)) {
+        System.err.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+      initMetadataStore(false);
+
+      Configuration conf = getConf();
+      long confDelta = conf.getLong(Constants.S3GUARD_CLI_PRUNE_AGE, 0);
+
+      long cliDelta = 0;
+      cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
+      cliDelta += getDeltaComponent(TimeUnit.HOURS, "hours");
+      cliDelta += getDeltaComponent(TimeUnit.MINUTES, "minutes");
+      cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
+
+      if (confDelta <= 0 && cliDelta <= 0) {
+        System.err.println(
+            "You must specify a positive age for metadata to prune.");
+      }
+
+      // A delta provided on the CLI overrides if one is configured
+      long delta = confDelta;
+      if (cliDelta > 0) {
+        delta = cliDelta;
+      }
+
+      long now = System.currentTimeMillis();
+      long divide = now - delta;
+
+      getStore().prune(divide);
+
+      out.flush();
+      return SUCCESS;
+    }
+
+    @Override
+    public int run(String[] args) throws InterruptedException, IOException {
+      return run(args, System.out);
+    }
+  }
+
+  private static S3GuardTool command;
+
+  private static void printHelp() {
+    if (command == null) {
+      System.err.println("Usage: hadoop " + USAGE);
+      System.err.println("\tperform S3Guard metadata store " +
+          "administrative commands.");
+    } else {
+      System.err.println("Usage: hadoop " + command.getUsage());
+    }
+    System.err.println();
+    System.err.println(COMMON_USAGE);
+  }
+
+  /**
+   * Execute the command with the given arguments.
+   *
+   * @param args command specific arguments.
+   * @param conf Hadoop configuration.
+   * @return exit code.
+   * @throws Exception on I/O errors.
+   */
+  public static int run(String[] args, Configuration conf) throws
+      Exception {
+    /* ToolRunner.run does this too, but we must do it before looking at
+    subCommand or instantiating the cmd object below */
+    String[] otherArgs = new GenericOptionsParser(conf, args)
+        .getRemainingArgs();
+    if (otherArgs.length == 0) {
+      printHelp();
+      return INVALID_ARGUMENT;
+    }
+    final String subCommand = otherArgs[0];
+    switch (subCommand) {
+    case Init.NAME:
+      command = new Init(conf);
+      break;
+    case Destroy.NAME:
+      command = new Destroy(conf);
+      break;
+    case Import.NAME:
+      command = new Import(conf);
+      break;
+    case Diff.NAME:
+      command = new Diff(conf);
+      break;
+    case Prune.NAME:
+      command = new Prune(conf);
+      break;
+    default:
+      printHelp();
+      return INVALID_ARGUMENT;
+    }
+    return ToolRunner.run(conf, command, otherArgs);
+  }
+
+  /**
+   * Main entry point. Calls {@code System.exit()} on all execution paths.
+   * @param args argument list
+   */
+  public static void main(String[] args) {
+    try {
+      int ret = run(args, new Configuration());
+      System.exit(ret);
+    } catch (CommandFormat.UnknownOptionException e) {
+      System.err.println(e.getMessage());
+      printHelp();
+      System.exit(INVALID_ARGUMENT);
+    } catch (Throwable e) {
+      e.printStackTrace(System.err);
+      System.exit(ERROR);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java
new file mode 100644
index 0000000..d430315
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.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.
+ */
+
+/**
+ * This package contains classes related to S3Guard: a feature of S3A to mask
+ * the eventual consistency behavior of S3 and optimize access patterns by
+ * coordinating with a strongly consistent external store for file system
+ * metadata.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
index 862ce6b..ce79284 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -105,6 +105,10 @@ public final class S3xLoginHelper {
    * @return a login tuple, possibly empty.
    */
   public static Login extractLoginDetails(URI name) {
+    if (name == null) {
+      return Login.EMPTY;
+    }
+
     try {
       String authority = name.getAuthority();
       if (authority == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3guard.sh
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3guard.sh b/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3guard.sh
new file mode 100644
index 0000000..039b077
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3guard.sh
@@ -0,0 +1,37 @@
+#!/usr/bin/env bash
+
+# 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.
+
+if ! declare -f hadoop_subcommand_s3guard >/dev/null 2>/dev/null; then
+
+  if [[ "${HADOOP_SHELL_EXECNAME}" = hadoop ]]; then
+    hadoop_add_subcommand "s3guard" client "manage metadata on S3"
+  fi
+
+  # this can't be indented otherwise shelldocs won't get it
+
+## @description  s3guard command for hadoop
+## @audience     public
+## @stability    stable
+## @replaceable  yes
+function hadoop_subcommand_s3guard
+{
+  # shellcheck disable=SC2034
+  HADOOP_CLASSNAME=org.apache.hadoop.fs.s3a.s3guard.S3GuardTool
+  hadoop_add_to_classpath_tools hadoop-aws
+}
+
+fi

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 182f060..b8d37c6 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -46,6 +46,7 @@ See also:
 
 * [Testing](testing.html)
 * [Troubleshooting S3a](troubleshooting_s3a.html)
+* [S3Guard](s3guard.html)
 
 ### Warning #1: Object Stores are not filesystems
 
@@ -1552,7 +1553,7 @@ for `fs.s3a.server-side-encryption-algorithm` is `AES256`.
 
 SSE-KMS is where the user specifies a Customer Master Key(CMK) that is used to
 encrypt the objects. The user may specify a specific CMK or leave the
-`fs.s3a.server-side-encryption-key` empty to use the default auto-generated key
+`fs.s3a.server-side-encryption.key` empty to use the default auto-generated key
 in AWS IAM.  Each CMK configured in AWS IAM is region specific, and cannot be
 used in a in a S3 bucket in a different region.  There is can also be policies
 assigned to the CMK that prohibit or restrict its use for users causing S3A

http://git-wip-us.apache.org/repos/asf/hadoop/blob/621b43e2/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
new file mode 100644
index 0000000..fe67d69
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -0,0 +1,610 @@
+<!---
+  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.
+-->
+
+# S3Guard: Consistency and Metadata Caching for S3A
+
+**Experimental Feature**
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+## Overview
+
+*S3Guard* is an experimental feature for the S3A client of the S3 object store,
+which can use a (consistent) database as the store of metadata about objects
+in an S3 bucket.
+
+S3Guard
+
+1. May improve performance on directory listing/scanning operations,
+including those which take place during the partitioning period of query
+execution, the process where files are listed and the work divided up amongst
+processes.
+
+1. Permits a consistent view of the object store. Without this, changes in
+objects may not be immediately visible, especially in listing operations.
+
+1. Offers a platform for future performance improvements for running Hadoop
+workloads on top of object stores
+
+The basic idea is that, for each operation in the Hadoop S3 client (s3a) that
+reads or modifies metadata, a shadow copy of that metadata is stored in a
+separate MetadataStore implementation.  Each MetadataStore implementation
+offers HDFS-like consistency for the metadata, and may also provide faster
+lookups for things like file status or directory listings.
+
+For links to early design documents and related patches, see
+[HADOOP-13345](https://issues.apache.org/jira/browse/HADOOP-13345).
+
+*Important*
+
+* S3Guard is experimental and should be considered unstable.
+
+* While all underlying data is persisted in S3, if, for some reason,
+the S3Guard-cached metadata becomes inconsistent with that in S3,
+queries on the data may become incorrect.
+For example, new datasets may be omitted, objects may be overwritten,
+or clients may not be aware that some data has been deleted.
+It is essential for all clients writing to an S3Guard-enabled
+S3 Repository to use the feature. Clients reading the data may work directly
+with the S3A data, in which case the normal S3 consistency guarantees apply.
+
+
+## Setting up S3Guard
+
+The latest configuration parameters are defined in `core-default.xml`.  You
+should consult that file for full information, but a summary is provided here.
+
+
+### 1. Choose the Database
+
+A core concept of S3Guard is that the directory listing data of the object
+store, *the metadata* is replicated in a higher-performance, consistent,
+database. In S3Guard, this database is called *The Metadata Store*
+
+By default, S3Guard is not enabled.
+
+The Metadata Store to use in production is bonded to Amazon's DynamoDB
+database service.  The following setting will enable this Metadata Store:
+
+```xml
+<property>
+    <name>fs.s3a.metadatastore.impl</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</value>
+</property>
+```
+
+Note that the `NullMetadataStore` store can be explicitly requested if desired.
+This offers no metadata storage, and effectively disables S3Guard.
+
+```xml
+<property>
+    <name>fs.s3a.metadatastore.impl</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+</property>
+```
+
+### 2. Configure S3Guard Settings
+
+More settings will may be added in the future.
+Currently the only Metadata Store-independent setting, besides the
+implementation class above, is the *allow authoritative* flag.
+
+It is recommended that you leave the default setting here:
+
+```xml
+<property>
+    <name>fs.s3a.metadatastore.authoritative</name>
+    <value>false</value>
+</property>
+
+```
+
+Setting this to `true` is currently an experimental feature.  When true, the
+S3A client will avoid round-trips to S3 when getting directory listings, if
+there is a fully-cached version of the directory stored in the Metadata Store.
+
+Note that if this is set to true, it may exacerbate or persist existing race
+conditions around multiple concurrent modifications and listings of a given
+directory tree.
+
+In particular: **If the Metadata Store is declared as authoritative,
+all interactions with the S3 bucket(s) must be through S3A clients sharing
+the same Metadata Store**
+
+
+### 3. Configure the Metadata Store.
+
+Here are the `DynamoDBMetadataStore` settings.  Other Metadata Store
+implementations will have their own configuration parameters.
+
+
+### 4. Name Your Table
+
+First, choose the name of the table you wish to use for the S3Guard metadata
+storage in your DynamoDB instance.  If you leave it unset/empty, a
+separate table will be created for each S3 bucket you access, and that
+bucket's name will be used for the name of the DynamoDB table.  For example,
+this sets the table name to `my-ddb-table-name`
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.table</name>
+  <value>my-ddb-table-name</value>
+  <description>
+    The DynamoDB table name to operate. Without this property, the respective
+    S3 bucket names will be used.
+  </description>
+</property>
+```
+
+It is good to share a table across multiple buckets for multiple reasons.
+
+1. You are billed for the I/O capacity allocated to the table,
+*even when the table is not used*. Sharing capacity can reduce costs.
+
+1. You can share the "provision burden" across the buckets. That is, rather
+than allocating for the peak load on a single bucket, you can allocate for
+the peak load *across all the buckets*, which is likely to be significantly
+lower.
+
+1. It's easier to measure and tune the load requirements and cost of
+S3Guard, because there is only one table to review and configure in the
+AWS management console.
+
+When wouldn't you want to share a table?
+
+1. When you do explicitly want to provision I/O capacity to a specific bucket
+and table, isolated from others.
+
+1. When you are using separate billing for specific buckets allocated
+to specific projects.
+
+1. When different users/roles have different access rights to different buckets.
+As S3Guard requires all users to have R/W access to the table, all users will
+be able to list the metadata in all buckets, even those to which they lack
+read access.
+
+### 5. Locate your Table
+
+You may also wish to specify the region to use for DynamoDB.  If a region
+is not configured, S3A will assume that it is in the same region as the S3
+bucket. A list of regions for the DynamoDB service can be found in
+[Amazon's documentation](http://docs.aws.amazon.com/general/latest/gr/rande.html#ddb_region).
+In this example, to use the US West 2 region:
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.region</name>
+  <value>us-west-2</value>
+</property>
+```
+
+When working with S3Guard-managed buckets from EC2 VMs running in AWS
+infrastructure, using a local DynamoDB region ensures the lowest latency
+and highest reliability, as well as avoiding all long-haul network charges.
+The S3Guard tables, and indeed, the S3 buckets, should all be in the same
+region as the VMs.
+
+### 6. Optional: Create your Table
+
+Next, you can choose whether or not the table will be automatically created
+(if it doesn't already exist).  If you want this feature, set the
+`fs.s3a.s3guard.ddb.table.create` option to `true`.
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.table.create</name>
+  <value>true</value>
+  <description>
+    If true, the S3A client will create the table if it does not already exist.
+  </description>
+</property>
+```
+
+### 7. If creating a table: Set your DynamoDB IO Capacity
+
+Next, you need to set the DynamoDB read and write throughput requirements you
+expect to need for your cluster.  Setting higher values will cost you more
+money.  *Note* that these settings only affect table creation when
+`fs.s3a.s3guard.ddb.table.create` is enabled.  To change the throughput for
+an existing table, use the AWS console or CLI tool.
+
+For more details on DynamoDB capacity units, see the AWS page on [Capacity
+Unit Calculations](http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/WorkingWithTables.html#CapacityUnitCalculations).
+
+The charges are incurred per hour for the life of the table, *even when the
+table and the underlying S3 buckets are not being used*.
+
+There are also charges incurred for data storage and for data IO outside of the
+region of the DynamoDB instance. S3Guard only stores metadata in DynamoDB: path names
+and summary details of objects —the actual data is stored in S3, so billed at S3
+rates.
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.read</name>
+  <value>500</value>
+  <description>
+    Provisioned throughput requirements for read operations in terms of capacity
+    units for the DynamoDB table.  This config value will only be used when
+    creating a new DynamoDB table, though later you can manually provision by
+    increasing or decreasing read capacity as needed for existing tables.
+    See DynamoDB documents for more information.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.write</name>
+  <value>100</value>
+  <description>
+    Provisioned throughput requirements for write operations in terms of
+    capacity units for the DynamoDB table.  Refer to related config
+    fs.s3a.s3guard.ddb.table.capacity.read before usage.
+  </description>
+</property>
+```
+
+Attempting to perform more IO than the capacity requested simply throttles the
+IO; small capacity numbers are recommended when initially experimenting
+with S3Guard.
+
+## Authenticating with S3Guard
+
+The DynamoDB metadata store takes advantage of the fact that the DynamoDB
+service uses the same authentication mechanisms as S3. S3Guard
+gets all its credentials from the S3A client that is using it.
+
+All existing S3 authentication mechanisms can be used, except for one
+exception. Credentials placed in URIs are not supported for S3Guard, for security
+reasons.
+
+## Per-bucket S3Guard configuration
+
+In production, it is likely only some buckets will have S3Guard enabled;
+those which are read-only may have disabled, for example. Equally importantly,
+buckets in different regions should have different tables, each
+in the relevant region.
+
+These options can be managed through S3A's [per-bucket configuration
+mechanism](./index.html#Configuring_different_S3_buckets).
+All options with the under `fs.s3a.bucket.BUCKETNAME.KEY` are propagated
+to the options `fs.s3a.KEY` *for that bucket only*.
+
+As an example, here is a configuration to use different metadata stores
+and tables for different buckets
+
+First, we define shortcuts for the metadata store classnames
+
+
+```xml
+<property>
+  <name>s3guard.null</name>
+  <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+</property>
+
+<property>
+  <name>s3guard.dynamo</name>
+  <value>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</value>
+</property>
+```
+
+Next, Amazon's public landsat database is configured with no
+metadata store
+
+```xml
+<property>
+  <name>fs.s3a.bucket.landsat-pds.metadatastore.impl</name>
+  <value>${s3guard.null}</value>
+  <description>The read-only landsat-pds repository isn't
+  managed by S3Guard</description>
+</property>
+```
+
+Next the `ireland-2` and `ireland-offline` buckets are configured with
+DynamoDB as the store, and a shared table `production-table`
+
+
+```xml
+<property>
+  <name>fs.s3a.bucket.ireland-2.metadatastore.impl</name>
+  <value>${s3guard.dynamo}</value>
+</property>
+
+<property>
+  <name>fs.s3a.bucket.ireland-offline.metadatastore.impl</name>
+  <value>${s3guard.dynamo}</value>
+</property>
+
+<property>
+  <name>fs.s3a.bucket.ireland-2.s3guard.ddb.table</name>
+  <value>production-table</value>
+</property>
+```
+
+The region of this table is automatically set to be that of the buckets,
+here `eu-west-1`; the same table name may actually be used in different
+regions.
+
+Together then, this configuration enables the DynamoDB Metadata Store
+for two buckets with a shared table, while disabling it for the public
+bucket.
+
+
+## S3Guard Command Line Interface (CLI)
+
+Note that in some cases an AWS region or `s3a://` URI can be provided.
+
+Metadata store URIs include a scheme that designates the backing store. For
+example (e.g. `dynamodb://table_name`;). As documented above, the
+AWS region can be inferred if the URI to an existing bucket is provided.
+
+
+The S3A URI must also be provided for per-bucket configuration options
+to be picked up. That is: when an s3a URL is provided on the command line,
+all its "resolved" per-bucket settings are used to connect to, authenticate
+with and configure the S3Guard table. If no such URL is provided, then
+the base settings are picked up.
+
+
+### Create a table: `s3guard init`
+
+```bash
+hadoop s3guard init -meta URI ( -region REGION | s3a://BUCKET )
+```
+
+Creates and initializes an empty metadata store.
+
+A DynamoDB metadata store can be initialized with additional parameters
+pertaining to [Provisioned Throughput](http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ProvisionedThroughput.html):
+
+```bash
+[-write PROVISIONED_WRITES] [-read PROVISIONED_READS]
+```
+
+Example 1
+
+```bash
+hadoop s3guard init -meta dynamodb://ireland-team -write 5 -read 10 s3a://ireland-1
+```
+
+Creates a table "ireland-team" with a capacity of 5 for writes, 10 for reads,
+in the same location as the bucket "ireland-1".
+
+
+Example 2
+
+```bash
+hadoop s3guard init -meta dynamodb://ireland-team -region eu-west-1
+```
+
+Creates a table "ireland-team" in the same region "s3-eu-west-1.amazonaws.com"
+
+
+### Import a bucket: `s3guard import`
+
+```bash
+hadoop s3guard import [-meta URI] s3a://BUCKET
+```
+
+Pre-populates a metadata store according to the current contents of an S3
+bucket. If the `-meta` option is omitted, the binding information is taken
+from the `core-site.xml` configuration.
+
+Example
+
+```bash
+hadoop s3guard import s3a://ireland-1
+```
+
+### Audit a table: `s3guard diff`
+
+```bash
+hadoop s3guard diff [-meta URI] s3a://BUCKET
+```
+
+Lists discrepancies between a metadata store and bucket. Note that depending on
+how S3Guard is used, certain discrepancies are to be expected.
+
+Example
+
+```bash
+hadoop s3guard diff s3a://ireland-1
+```
+
+### Delete a table: `s3guard destroy`
+
+
+Deletes a metadata store. With DynamoDB as the store, this means
+the specific DynamoDB table use to store the metadata.
+
+```bash
+hadoop s3guard destroy [-meta URI] ( -region REGION | s3a://BUCKET )
+```
+
+This *does not* delete the bucket, only the S3Guard table which it is bound
+to.
+
+
+Examples
+
+```bash
+hadoop s3guard destroy s3a://ireland-1
+```
+
+Deletes the table which the bucket ireland-1 is configured to use
+as its MetadataStore.
+
+```bash
+hadoop s3guard destroy -meta dynamodb://ireland-team -region eu-west-1
+```
+
+
+
+### Clean up a table, `s3guard prune`
+
+Delete all file entries in the MetadataStore table whose object "modification
+time" is older than the specified age.
+
+```bash
+hadoop s3guard prune [-days DAYS] [-hours HOURS] [-minutes MINUTES]
+    [-seconds SECONDS] [-m URI] ( -region REGION | s3a://BUCKET )
+```
+
+A time value must be supplied.
+
+1. This does not delete the entries in the bucket itself.
+1. The modification time is effectively the creation time of the objects
+in the S3 Bucket.
+1. Even when an S3A URI is supplied, all entries in the table older than
+a specific age are deleted &mdash; even those from other buckets.
+
+Example
+
+```bash
+hadoop s3guard prune -days 7 s3a://ireland-1
+```
+
+Deletes all entries in the S3Guard table for files older than seven days from
+the table associated with `s3a://ireland-1`.
+
+```bash
+hadoop s3guard prune -hours 1 -minutes 30 -meta dynamodb://ireland-team -region eu-west-1
+```
+
+Delete all entries more than 90 minutes old from the table "ireland-team" in
+the region "eu-west-1".
+
+
+
+## Debugging and Error Handling
+
+If you run into network connectivity issues, or have a machine failure in the
+middle of an operation, you may end up with your metadata store having state
+that differs from S3.  The S3Guard CLI commands, covered in the CLI section
+above, can be used to diagnose and repair these issues.
+
+There are some logs whose log level can be increased to provide more
+information.
+
+```properties
+# Log S3Guard classes
+log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG
+
+# Log all S3A classes
+log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+
+# Enable debug logging of AWS DynamoDB client
+log4j.logger.com.amazonaws.services.dynamodbv2.AmazonDynamoDB
+
+# Log all HTTP requests made; includes S3 interaction. This may
+# include sensitive information such as account IDs in HTTP headers.
+log4j.logger.com.amazonaws.request=DEBUG
+
+```
+
+If all else fails, S3Guard is designed to allow for easy recovery by deleting
+the metadata store data. In DynamoDB, this can be accomplished by simply
+deleting the table, and allowing S3Guard to recreate it from scratch.  Note
+that S3Guard tracks recent changes to file metadata to implement consistency.
+Deleting the metadata store table will simply result in a period of eventual
+consistency for any file modifications that were made right before the table
+was deleted.
+
+### Failure Semantics
+
+Operations which modify metadata will make changes to S3 first. If, and only
+if, those operations succeed, the equivalent changes will be made to the
+Metadata Store.
+
+These changes to S3 and Metadata Store are not fully-transactional:  If the S3
+operations succeed, and the subsequent Metadata Store updates fail, the S3
+changes will *not* be rolled back.  In this case, an error message will be
+logged.
+
+### Versioning
+
+S3Guard tables are created with a version marker, an entry with the primary
+key and child entry of `../VERSION`; the use of a relative path guarantees
+that it will not be resolved.
+
+#### Versioning policy.
+
+1. The version number of an S3Guard table will only be incremented when
+an incompatible change is made to the table structure —that is, the structure
+has changed so that it is no longer readable by older versions, or because
+it has added new mandatory fields which older versions do not create.
+1. The version number of S3Guard tables will only be changed by incrementing
+the value.
+1. Updated versions of S3Guard MAY continue to support older version tables.
+1. If an incompatible change is made such that existing tables are not compatible,
+then a means shall be provided to update existing tables. For example:
+an option in the Command Line Interface, or an option to upgrade tables
+during S3Guard initialization.
+
+*Note*: this policy does not indicate any intent to upgrade table structures
+in an incompatible manner. The version marker in tables exists to support
+such an option if it ever becomes necessary, by ensuring that all S3Guard
+client can recognise any version mismatch.
+
+### Security
+
+All users of the DynamoDB table must have write access to it. This
+effectively means they must have write access to the entire object store.
+
+There's not been much testing of using a S3Guard Metadata Store
+with a read-only S3 Bucket. It *should* work, provided all users
+have write access to the DynamoDB table. And, as updates to the Metadata Store
+are only made after successful file creation, deletion and rename, the
+store is *unlikely* to get out of sync, it is still something which
+merits more testing before it could be considered reliable.
+
+### Troubleshooting
+
+#### Error: `S3Guard table lacks version marker.`
+
+The table which was intended to be used as a S3guard metadata store
+does not have any version marker indicating that it is a S3Guard table.
+
+It may be that this is not a S3Guard table.
+
+* Make sure that this is the correct table name.
+* Delete the table, so it can be rebuilt.
+
+#### Error: `Database table is from an incompatible S3Guard version`
+
+This indicates that the version of S3Guard which created (or possibly updated)
+the database table is from a different version that that expected by the S3A
+client.
+
+This error will also include the expected and actual version numbers.
+
+If the expected version is lower than the actual version, then the version
+of the S3A client library is too old to interact with this S3Guard-managed
+bucket. Upgrade the application/library.
+
+If the expected version is higher than the actual version, then the table
+itself will need upgrading.
+
+#### Error `"DynamoDB table TABLE does not exist in region REGION; auto-creation is turned off"`
+
+S3Guard could not find the DynamoDB table for the Metadata Store,
+and it was not configured to create it. Either the table was missing,
+or the configuration is preventing S3Guard from finding the table.
+
+1. Verify that the value of `fs.s3a.s3guard.ddb.table` is correct.
+1. If the region for an existing table has been set in
+`fs.s3a.s3guard.ddb.region`, verify that the value is correct.
+1. If the region is not set, verify that the table exists in the same
+region as the bucket being used.
+1. Create the table if necessary.


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


[31/48] 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/190510f3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/190510f3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/190510f3

Branch: refs/heads/HDFS-10467
Commit: 190510f323364e166a4d3ce61653782b96f32567
Parents: 9176c5a
Author: Inigo <in...@apache.org>
Authored: Thu Apr 6 19:18:52 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:08 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/190510f3/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/190510f3/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/190510f3/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/190510f3/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/190510f3/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/190510f3/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/190510f3/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


[47/48] 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/37449af2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/37449af2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/37449af2

Branch: refs/heads/HDFS-10467
Commit: 37449af266eb03e3f2d8880ca1928a44f4e67e49
Parents: c7b39ef
Author: Inigo Goiri <in...@apache.org>
Authored: Tue Aug 8 14:44:43 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Sat Sep 2 14:20:10 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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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/37449af2/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


[34/48] 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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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/5da7d3f4/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