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

Branch: refs/heads/HDFS-10467
Commit: ca78fcb4a59c63b3b23b1da55ab91fd832c90f6c
Parents: 0c23c8c
Author: Inigo Goiri <inigo...@apache.org>
Authored: Fri Aug 4 18:00:12 2017 -0700
Committer: Inigo Goiri <inigo...@apache.org>
Committed: Sat Aug 12 09:36:24 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/ca78fcb4/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 acd4790..f156fdb 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;
@@ -1160,8 +1162,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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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/ca78fcb4/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-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to