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 wa...@apache.org on 2015/09/23 22:32:55 UTC

[03/21] hadoop git commit: HDFS-9039. Separate client and server side methods of o.a.h.hdfs.NameNodeProxies. Contributed by Mingliang Liu.

HDFS-9039. Separate client and server side methods of o.a.h.hdfs.NameNodeProxies. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-1197
Commit: 63d9f1596c92206cce3b72e3214d2fb5f6242b90
Parents: cc2b473
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 22 20:52:37 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 22 20:52:37 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/NameNodeProxiesClient.java      |  366 +++++
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 +
 .../ClientNamenodeProtocolTranslatorPB.java     | 1531 ++++++++++++++++++
 .../ha/AbstractNNFailoverProxyProvider.java     |   55 +
 .../ha/WrappedFailoverProxyProvider.java        |   71 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   13 +-
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |    9 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  275 +---
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 -
 .../ClientNamenodeProtocolTranslatorPB.java     | 1531 ------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |    2 +-
 .../ha/AbstractNNFailoverProxyProvider.java     |   51 -
 .../ha/ConfiguredFailoverProxyProvider.java     |    2 +-
 .../ha/WrappedFailoverProxyProvider.java        |   80 -
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    2 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |    4 +-
 17 files changed, 2096 insertions(+), 1991 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
new file mode 100644
index 0000000..223c40d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
@@ -0,0 +1,366 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+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.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
+import org.apache.hadoop.hdfs.server.namenode.ha.WrappedFailoverProxyProvider;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+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.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Create proxy objects with {@link ClientProtocol} to communicate with a remote
+ * NN. Generally use {@link NameNodeProxiesClient#createProxyWithClientProtocol(
+ * Configuration, URI, AtomicBoolean)}, which will create either an HA- or
+ * non-HA-enabled client proxy as appropriate.
+ *
+ * For creating proxy objects with other protocols, please see
+ * {@link NameNodeProxies#createProxy(Configuration, URI, Class)}.
+ */
+@InterfaceAudience.Private
+public class NameNodeProxiesClient {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      NameNodeProxiesClient.class);
+
+  /**
+   * Wrapper for a client proxy as well as its associated service ID.
+   * This is simply used as a tuple-like return type for created NN proxy.
+   */
+  public static class ProxyAndInfo<PROXYTYPE> {
+    private final PROXYTYPE proxy;
+    private final Text dtService;
+    private final InetSocketAddress address;
+
+    public ProxyAndInfo(PROXYTYPE proxy, Text dtService,
+                        InetSocketAddress address) {
+      this.proxy = proxy;
+      this.dtService = dtService;
+      this.address = address;
+    }
+
+    public PROXYTYPE getProxy() {
+      return proxy;
+    }
+
+    public Text getDelegationTokenService() {
+      return dtService;
+    }
+
+    public InetSocketAddress getAddress() {
+      return address;
+    }
+  }
+
+  /**
+   * Creates the namenode proxy with the ClientProtocol. This will handle
+   * creation of either HA- or non-HA-enabled proxy objects, depending upon
+   * if the provided URI is a configured logical URI.
+   *
+   * @param conf the configuration containing the required IPC
+   *        properties, client failover configurations, etc.
+   * @param nameNodeUri the URI pointing either to a specific NameNode
+   *        or to a logical nameservice.
+   * @param fallbackToSimpleAuth set to true or false during calls to indicate
+   *        if a secure client falls back to simple auth
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to
+   * @throws IOException if there is an error creating the proxy
+   * @see {@link NameNodeProxies#createProxy(Configuration, URI, Class)}.
+   */
+  public static ProxyAndInfo<ClientProtocol> createProxyWithClientProtocol(
+      Configuration conf, URI nameNodeUri, AtomicBoolean fallbackToSimpleAuth)
+      throws IOException {
+    AbstractNNFailoverProxyProvider<ClientProtocol> failoverProxyProvider =
+        createFailoverProxyProvider(conf, nameNodeUri, ClientProtocol.class,
+            true, fallbackToSimpleAuth);
+
+    if (failoverProxyProvider == null) {
+      InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
+      Text dtService = SecurityUtil.buildTokenService(nnAddr);
+      ClientProtocol proxy = createNonHAProxyWithClientProtocol(nnAddr, conf,
+          UserGroupInformation.getCurrentUser(), true, fallbackToSimpleAuth);
+      return new ProxyAndInfo<>(proxy, dtService, nnAddr);
+    } else {
+      return createHAProxy(conf, nameNodeUri, ClientProtocol.class,
+          failoverProxyProvider);
+    }
+  }
+
+  /**
+   * Generate a dummy namenode proxy instance that utilizes our hacked
+   * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
+   * method will proactively drop RPC responses. Currently this method only
+   * support HA setup. null will be returned if the given configuration is not
+   * for HA.
+   *
+   * @param config the configuration containing the required IPC
+   *        properties, client failover configurations, etc.
+   * @param nameNodeUri the URI pointing either to a specific NameNode
+   *        or to a logical nameservice.
+   * @param xface the IPC interface which should be created
+   * @param numResponseToDrop The number of responses to drop for each RPC call
+   * @param fallbackToSimpleAuth set to true or false during calls to indicate
+   *        if a secure client falls back to simple auth
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to. Will return null of the
+   *         given configuration does not support HA.
+   * @throws IOException if there is an error creating the proxy
+   */
+  public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler(
+      Configuration config, URI nameNodeUri, Class<T> xface,
+      int numResponseToDrop, AtomicBoolean fallbackToSimpleAuth)
+      throws IOException {
+    Preconditions.checkArgument(numResponseToDrop > 0);
+    AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
+        createFailoverProxyProvider(config, nameNodeUri, xface, true,
+            fallbackToSimpleAuth);
+
+    if (failoverProxyProvider != null) { // HA case
+      int delay = config.getInt(
+          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
+          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
+      int maxCap = config.getInt(
+          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
+          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
+      int maxFailoverAttempts = config.getInt(
+          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
+      int maxRetryAttempts = config.getInt(
+          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+      InvocationHandler dummyHandler = new LossyRetryInvocationHandler<>(
+              numResponseToDrop, failoverProxyProvider,
+              RetryPolicies.failoverOnNetworkException(
+                  RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts,
+                  Math.max(numResponseToDrop + 1, maxRetryAttempts), delay,
+                  maxCap));
+
+      @SuppressWarnings("unchecked")
+      T proxy = (T) Proxy.newProxyInstance(
+          failoverProxyProvider.getInterface().getClassLoader(),
+          new Class[]{xface}, dummyHandler);
+      Text dtService;
+      if (failoverProxyProvider.useLogicalURI()) {
+        dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
+            HdfsConstants.HDFS_URI_SCHEME);
+      } else {
+        dtService = SecurityUtil.buildTokenService(
+            DFSUtilClient.getNNAddress(nameNodeUri));
+      }
+      return new ProxyAndInfo<>(proxy, dtService,
+          DFSUtilClient.getNNAddress(nameNodeUri));
+    } else {
+      LOG.warn("Currently creating proxy using " +
+          "LossyRetryInvocationHandler requires NN HA setup");
+      return null;
+    }
+  }
+
+  /** Creates the Failover proxy provider instance*/
+  @VisibleForTesting
+  public static <T> AbstractNNFailoverProxyProvider<T> createFailoverProxyProvider(
+      Configuration conf, URI nameNodeUri, Class<T> xface, boolean checkPort,
+      AtomicBoolean fallbackToSimpleAuth) throws IOException {
+    Class<FailoverProxyProvider<T>> failoverProxyProviderClass = null;
+    AbstractNNFailoverProxyProvider<T> providerNN;
+    try {
+      // Obtain the class of the proxy provider
+      failoverProxyProviderClass = getFailoverProxyProviderClass(conf,
+          nameNodeUri);
+      if (failoverProxyProviderClass == null) {
+        return null;
+      }
+      // Create a proxy provider instance.
+      Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass
+          .getConstructor(Configuration.class, URI.class, Class.class);
+      FailoverProxyProvider<T> provider = ctor.newInstance(conf, nameNodeUri,
+          xface);
+
+      // If the proxy provider is of an old implementation, wrap it.
+      if (!(provider instanceof AbstractNNFailoverProxyProvider)) {
+        providerNN = new WrappedFailoverProxyProvider<>(provider);
+      } else {
+        providerNN = (AbstractNNFailoverProxyProvider<T>)provider;
+      }
+    } catch (Exception e) {
+      final String message = "Couldn't create proxy provider " +
+          failoverProxyProviderClass;
+      LOG.debug(message, e);
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      } else {
+        throw new IOException(message, e);
+      }
+    }
+
+    // Check the port in the URI, if it is logical.
+    if (checkPort && providerNN.useLogicalURI()) {
+      int port = nameNodeUri.getPort();
+      if (port > 0 &&
+          port != HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) {
+        // Throwing here without any cleanup is fine since we have not
+        // actually created the underlying proxies yet.
+        throw new IOException("Port " + port + " specified in URI "
+            + nameNodeUri + " but host '" + nameNodeUri.getHost()
+            + "' is a logical (HA) namenode"
+            + " and does not use port information.");
+      }
+    }
+    providerNN.setFallbackToSimpleAuth(fallbackToSimpleAuth);
+    return providerNN;
+  }
+
+  /** Gets the configured Failover proxy provider's class */
+  @VisibleForTesting
+  public static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
+      Configuration conf, URI nameNodeUri) throws IOException {
+    if (nameNodeUri == null) {
+      return null;
+    }
+    String host = nameNodeUri.getHost();
+    String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+        + "." + host;
+    try {
+      @SuppressWarnings("unchecked")
+      Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>)
+          conf.getClass(configKey, null, FailoverProxyProvider.class);
+      return ret;
+    } catch (RuntimeException e) {
+      if (e.getCause() instanceof ClassNotFoundException) {
+        throw new IOException("Could not load failover proxy provider class "
+            + conf.get(configKey) + " which is configured for authority "
+            + nameNodeUri, e);
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * Creates an explicitly HA-enabled proxy object.
+   *
+   * @param conf the configuration object
+   * @param nameNodeUri the URI pointing either to a specific NameNode or to a
+   *        logical nameservice.
+   * @param xface the IPC interface which should be created
+   * @param failoverProxyProvider Failover proxy provider
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> ProxyAndInfo<T> createHAProxy(
+      Configuration conf, URI nameNodeUri, Class<T> xface,
+      AbstractNNFailoverProxyProvider<T> failoverProxyProvider)
+      throws IOException {
+    Preconditions.checkNotNull(failoverProxyProvider);
+    // HA case
+    DfsClientConf config = new DfsClientConf(conf);
+    T proxy = (T) RetryProxy.create(xface, failoverProxyProvider,
+        RetryPolicies.failoverOnNetworkException(
+            RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),
+            config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),
+            config.getFailoverSleepMaxMillis()));
+
+    Text dtService;
+    if (failoverProxyProvider.useLogicalURI()) {
+      dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
+          HdfsConstants.HDFS_URI_SCHEME);
+    } else {
+      dtService = SecurityUtil.buildTokenService(
+          DFSUtilClient.getNNAddress(nameNodeUri));
+    }
+    return new ProxyAndInfo<>(proxy, dtService,
+        DFSUtilClient.getNNAddress(nameNodeUri));
+  }
+
+  public static ClientProtocol createNonHAProxyWithClientProtocol(
+      InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
+      boolean withRetries, AtomicBoolean fallbackToSimpleAuth)
+      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,
+            SafeModeException.class.getName());
+
+    final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
+    ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
+        ClientNamenodeProtocolPB.class, version, address, ugi, conf,
+        NetUtils.getDefaultSocketFactory(conf),
+        org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy,
+        fallbackToSimpleAuth).getProxy();
+
+    if (withRetries) { // create the proxy with retries
+      Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<>();
+      ClientProtocol translatorProxy =
+          new ClientNamenodeProtocolTranslatorPB(proxy);
+      return (ClientProtocol) RetryProxy.create(
+          ClientProtocol.class,
+          new DefaultFailoverProxyProvider<>(ClientProtocol.class,
+              translatorProxy),
+          methodNameToPolicyMap,
+          defaultPolicy);
+    } else {
+      return new ClientNamenodeProtocolTranslatorPB(proxy);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
new file mode 100644
index 0000000..e7ce44b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.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.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
+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;
+
+
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@KerberosInfo(
+    serverPrincipal = HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
+    protocolVersion = 1)
+/**
+ * Protocol that a clients use to communicate with the NameNode.
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+public interface ClientNamenodeProtocolPB extends 
+  ClientNamenodeProtocol.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
new file mode 100644
index 0000000..f4ce46d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -0,0 +1,1531 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
+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.Rename;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+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.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+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.DSQuotaExceededException;
+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.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.NSQuotaExceededException;
+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.AclProtos.GetAclStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+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.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 org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
+import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
+import org.apache.hadoop.security.token.Token;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
+
+import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
+    .EncryptionZoneProto;
+
+/**
+ * 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 ClientNamenodeProtocolTranslatorPB implements
+    ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
+  final private ClientNamenodeProtocolPB rpcProxy;
+
+  static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = 
+  GetServerDefaultsRequestProto.newBuilder().build();
+
+  private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
+  GetFsStatusRequestProto.newBuilder().build();
+
+  private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
+  SaveNamespaceRequestProto.newBuilder().build();
+
+  private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = 
+  RollEditsRequestProto.getDefaultInstance();
+
+  private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
+  RefreshNodesRequestProto.newBuilder().build();
+
+  private final static FinalizeUpgradeRequestProto
+  VOID_FINALIZE_UPGRADE_REQUEST =
+      FinalizeUpgradeRequestProto.newBuilder().build();
+
+  private final static GetDataEncryptionKeyRequestProto
+  VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
+      GetDataEncryptionKeyRequestProto.newBuilder().build();
+
+  private final static GetStoragePoliciesRequestProto
+  VOID_GET_STORAGE_POLICIES_REQUEST =
+      GetStoragePoliciesRequestProto.newBuilder().build();
+
+  public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
+    rpcProxy = proxy;
+  }
+  
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public LocatedBlocks getBlockLocations(String src, long offset, long length)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
+        .newBuilder()
+        .setSrc(src)
+        .setOffset(offset)
+        .setLength(length)
+        .build();
+    try {
+      GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
+          req);
+      return resp.hasLocations() ?
+        PBHelperClient.convert(resp.getLocations()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST;
+    try {
+      return PBHelperClient
+          .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize, 
+      CryptoProtocolVersion[] supportedVersions)
+      throws AccessControlException, AlreadyBeingCreatedException,
+      DSQuotaExceededException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
+        .setSrc(src)
+        .setMasked(PBHelperClient.convert(masked))
+        .setClientName(clientName)
+        .setCreateFlag(PBHelperClient.convertCreateFlag(flag))
+        .setCreateParent(createParent)
+        .setReplication(replication)
+        .setBlockSize(blockSize);
+    builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
+    CreateRequestProto req = builder.build();
+    try {
+      CreateResponseProto res = rpcProxy.create(null, req);
+      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public boolean truncate(String src, long newLength, String clientName)
+      throws IOException, UnresolvedLinkException {
+    TruncateRequestProto req = TruncateRequestProto.newBuilder()
+        .setSrc(src)
+        .setNewLength(newLength)
+        .setClientName(clientName)
+        .build();
+    try {
+      return rpcProxy.truncate(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LastBlockWithStatus append(String src, String clientName,
+      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
+      DSQuotaExceededException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
+        .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
+        .build();
+    try {
+      AppendResponseProto res = rpcProxy.append(null, req);
+      LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
+          .convert(res.getBlock()) : null;
+      HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
+          : null;
+      return new LastBlockWithStatus(lastBlock, stat);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean setReplication(String src, short replication)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
+        .setSrc(src)
+        .setReplication(replication)
+        .build();
+    try {
+      return rpcProxy.setReplication(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setPermission(String src, FsPermission permission)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
+        .setSrc(src)
+        .setPermission(PBHelperClient.convert(permission))
+        .build();
+    try {
+      rpcProxy.setPermission(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setOwner(String src, String username, String groupname)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
+        .setSrc(src);
+    if (username != null)
+        req.setUsername(username);
+    if (groupname != null)
+        req.setGroupname(groupname);
+    try {
+      rpcProxy.setOwner(null, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void abandonBlock(ExtendedBlock b, long fileId, String src,
+      String holder) throws AccessControlException, FileNotFoundException,
+        UnresolvedLinkException, IOException {
+    AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
+        .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
+            .setFileId(fileId).build();
+    try {
+      rpcProxy.abandonBlock(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
+      String[] favoredNodes)
+      throws AccessControlException, FileNotFoundException,
+      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
+        .setSrc(src).setClientName(clientName).setFileId(fileId);
+    if (previous != null) 
+      req.setPrevious(PBHelperClient.convert(previous));
+    if (excludeNodes != null)
+      req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
+    if (favoredNodes != null) {
+      req.addAllFavoredNodes(Arrays.asList(favoredNodes));
+    }
+    try {
+      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock getAdditionalDatanode(String src, long fileId,
+      ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
+      DatanodeInfo[] excludes,
+      int numAdditionalNodes, String clientName) throws AccessControlException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
+        .newBuilder()
+        .setSrc(src)
+        .setFileId(fileId)
+        .setBlk(PBHelperClient.convert(blk))
+        .addAllExistings(PBHelperClient.convert(existings))
+        .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
+        .addAllExcludes(PBHelperClient.convert(excludes))
+        .setNumAdditionalNodes(numAdditionalNodes)
+        .setClientName(clientName)
+        .build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
+          .getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean complete(String src, String clientName,
+                          ExtendedBlock last, long fileId)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName)
+        .setFileId(fileId);
+    if (last != null)
+      req.setLast(PBHelperClient.convert(last));
+    try {
+      return rpcProxy.complete(null, req.build()).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
+        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
+        .build();
+    try {
+      rpcProxy.reportBadBlocks(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean rename(String src, String dst) throws UnresolvedLinkException,
+      IOException {
+    RenameRequestProto req = RenameRequestProto.newBuilder()
+        .setSrc(src)
+        .setDst(dst).build();
+    try {
+      return rpcProxy.rename(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+
+  @Override
+  public void rename2(String src, String dst, Rename... options)
+      throws AccessControlException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    boolean overwrite = false;
+    if (options != null) {
+      for (Rename option : options) {
+        if (option == Rename.OVERWRITE) {
+          overwrite = true;
+        }
+      }
+    }
+    Rename2RequestProto req = Rename2RequestProto.newBuilder().
+        setSrc(src).
+        setDst(dst).setOverwriteDest(overwrite).
+        build();
+    try {
+      rpcProxy.rename2(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public void concat(String trg, String[] srcs) throws IOException,
+      UnresolvedLinkException {
+    ConcatRequestProto req = ConcatRequestProto.newBuilder().
+        setTrg(trg).
+        addAllSrcs(Arrays.asList(srcs)).build();
+    try {
+      rpcProxy.concat(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+
+  @Override
+  public boolean delete(String src, boolean recursive)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
+    try {
+      return rpcProxy.delete(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
+        .setSrc(src)
+        .setMasked(PBHelperClient.convert(masked))
+        .setCreateParent(createParent).build();
+
+    try {
+      return rpcProxy.mkdirs(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DirectoryListing getListing(String src, byte[] startAfter,
+      boolean needLocation) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    GetListingRequestProto req = GetListingRequestProto.newBuilder()
+        .setSrc(src)
+        .setStartAfter(ByteString.copyFrom(startAfter))
+        .setNeedLocation(needLocation).build();
+    try {
+      GetListingResponseProto result = rpcProxy.getListing(null, req);
+      
+      if (result.hasDirList()) {
+        return PBHelperClient.convert(result.getDirList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void renewLease(String clientName) throws AccessControlException,
+      IOException {
+    RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
+        .setClientName(clientName).build();
+    try {
+      rpcProxy.renewLease(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean recoverLease(String src, String clientName)
+      throws IOException {
+    RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName).build();
+    try {
+      return rpcProxy.recoverLease(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }  
+  }
+
+  @Override
+  public long[] getStats() throws IOException {
+    try {
+      return PBHelperClient.convert(rpcProxy.getFsStats(null,
+          VOID_GET_FSSTATUS_REQUEST));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+      throws IOException {
+    GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
+        .newBuilder()
+        .setType(PBHelperClient.convert(type)).build();
+    try {
+      return PBHelperClient.convert(
+          rpcProxy.getDatanodeReport(null, req).getDiList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
+      throws IOException {
+    final GetDatanodeStorageReportRequestProto req
+        = GetDatanodeStorageReportRequestProto.newBuilder()
+            .setType(PBHelperClient.convert(type)).build();
+    try {
+      return PBHelperClient.convertDatanodeStorageReports(
+          rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long getPreferredBlockSize(String filename) throws IOException,
+      UnresolvedLinkException {
+    GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
+        .newBuilder()
+        .setFilename(filename)
+        .build();
+    try {
+      return rpcProxy.getPreferredBlockSize(null, req).getBsize();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
+    SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
+        .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
+    try {
+      return rpcProxy.setSafeMode(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+    try {
+      SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder()
+          .setTimeWindow(timeWindow).setTxGap(txGap).build();
+      return rpcProxy.saveNamespace(null, req).getSaved();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public long rollEdits() throws AccessControlException, IOException {
+    try {
+      RollEditsResponseProto resp = rpcProxy.rollEdits(null,
+          VOID_ROLLEDITS_REQUEST);
+      return resp.getNewSegmentTxId();
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException, IOException{
+    RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
+        .newBuilder()
+        .setArg(arg).build();
+    try {
+      return rpcProxy.restoreFailedStorage(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void refreshNodes() throws IOException {
+    try {
+      rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void finalizeUpgrade() throws IOException {
+    try {
+      rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+    final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
+        .setAction(PBHelperClient.convert(action)).build();
+    try {
+      final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
+      if (proto.hasRollingUpgradeInfo()) {
+        return PBHelperClient.convert(proto.getRollingUpgradeInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException {
+    ListCorruptFileBlocksRequestProto.Builder req = 
+        ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
+    if (cookie != null) 
+      req.setCookie(cookie);
+    try {
+      return PBHelperClient.convert(
+          rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void metaSave(String filename) throws IOException {
+    MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder()
+        .setFilename(filename).build();
+    try {
+      rpcProxy.metaSave(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
+      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public HdfsFileStatus getFileLinkInfo(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException {
+    GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
+      return result.hasFs() ?  
+          PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public ContentSummary getContentSummary(String path)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    GetContentSummaryRequestProto req = GetContentSummaryRequestProto
+        .newBuilder()
+        .setPath(path)
+        .build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getContentSummary(null, req)
+          .getSummary());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+                       StorageType type)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    final SetQuotaRequestProto.Builder builder
+        = SetQuotaRequestProto.newBuilder()
+        .setPath(path)
+        .setNamespaceQuota(namespaceQuota)
+        .setStoragespaceQuota(storagespaceQuota);
+    if (type != null) {
+      builder.setStorageType(PBHelperClient.convertStorageType(type));
+    }
+    final SetQuotaRequestProto req = builder.build();
+    try {
+      rpcProxy.setQuota(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void fsync(String src, long fileId, String client,
+                    long lastBlockLength)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
+        .setClient(client).setLastBlockLength(lastBlockLength)
+            .setFileId(fileId).build();
+    try {
+      rpcProxy.fsync(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setTimes(String src, long mtime, long atime)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
+        .setSrc(src)
+        .setMtime(mtime)
+        .setAtime(atime)
+        .build();
+    try {
+      rpcProxy.setTimes(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
+        .setTarget(target)
+        .setLink(link)
+        .setDirPerm(PBHelperClient.convert(dirPerm))
+        .setCreateParent(createParent)
+        .build();
+    try {
+      rpcProxy.createSymlink(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public String getLinkTarget(String path) throws AccessControlException,
+      FileNotFoundException, IOException {
+    GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
+        .setPath(path).build();
+    try {
+      GetLinkTargetResponseProto rsp = rpcProxy.getLinkTarget(null, req);
+      return rsp.hasTargetPath() ? rsp.getTargetPath() : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+      String clientName) throws IOException {
+    UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto
+        .newBuilder()
+        .setBlock(PBHelperClient.convert(block))
+        .setClientName(clientName)
+        .build();
+    try {
+      return PBHelperClient.convert(
+          rpcProxy.updateBlockForPipeline(null, req).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
+    UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
+        .setClientName(clientName)
+        .setOldBlock(PBHelperClient.convert(oldBlock))
+        .setNewBlock(PBHelperClient.convert(newBlock))
+        .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes)))
+        .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
+        .build();
+    try {
+      rpcProxy.updatePipeline(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
+        .newBuilder()
+        .setRenewer(renewer == null ? "" : renewer.toString())
+        .build();
+    try {
+      GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
+      return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
+          : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
+        setToken(PBHelperClient.convert(token)).
+        build();
+    try {
+      return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto
+        .newBuilder()
+        .setToken(PBHelperClient.convert(token))
+        .build();
+    try {
+      rpcProxy.cancelDelegationToken(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setBalancerBandwidth(long bandwidth) throws IOException {
+    SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
+        .setBandwidth(bandwidth)
+        .build();
+    try {
+      rpcProxy.setBalancerBandwidth(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+  }
+  
+  @Override
+  public DataEncryptionKey getDataEncryptionKey() throws IOException {
+    try {
+      GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
+          null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
+     return rsp.hasDataEncryptionKey() ? 
+          PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+
+  @Override
+  public boolean isFileClosed(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      return rpcProxy.isFileClosed(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Object getUnderlyingProxyObject() {
+    return rpcProxy;
+  }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    final CreateSnapshotRequestProto.Builder builder
+        = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot);
+    if (snapshotName != null) {
+      builder.setSnapshotName(snapshotName);
+    }
+    final CreateSnapshotRequestProto req = builder.build();
+    try {
+      return rpcProxy.createSnapshot(null, req).getSnapshotPath();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build();
+    try {
+      rpcProxy.deleteSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).build();
+    try {
+      rpcProxy.allowSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void disallowSnapshot(String snapshotRoot) throws IOException {
+    DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto
+        .newBuilder().setSnapshotRoot(snapshotRoot).build();
+    try {
+      rpcProxy.disallowSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName)
+        .setSnapshotNewName(snapshotNewName).build();
+    try {
+      rpcProxy.renameSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    GetSnapshottableDirListingRequestProto req = 
+        GetSnapshottableDirListingRequestProto.newBuilder().build();
+    try {
+      GetSnapshottableDirListingResponseProto result = rpcProxy
+          .getSnapshottableDirListing(null, req);
+      
+      if (result.hasSnapshottableDirList()) {
+        return PBHelperClient.convert(result.getSnapshottableDirList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto
+        .newBuilder().setSnapshotRoot(snapshotRoot)
+        .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
+    try {
+      GetSnapshotDiffReportResponseProto result = 
+          rpcProxy.getSnapshotDiffReport(null, req);
+    
+      return PBHelperClient.convert(result.getDiffReport());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    try {
+      AddCacheDirectiveRequestProto.Builder builder =
+          AddCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelperClient.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+      }
+      return rpcProxy.addCacheDirective(null, builder.build()).getId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    try {
+      ModifyCacheDirectiveRequestProto.Builder builder =
+          ModifyCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelperClient.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+      }
+      rpcProxy.modifyCacheDirective(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeCacheDirective(long id)
+      throws IOException {
+    try {
+      rpcProxy.removeCacheDirective(null,
+          RemoveCacheDirectiveRequestProto.newBuilder().
+              setId(id).build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private static class BatchedCacheEntries
+      implements BatchedEntries<CacheDirectiveEntry> {
+    private final ListCacheDirectivesResponseProto response;
+
+    BatchedCacheEntries(
+        ListCacheDirectivesResponseProto response) {
+      this.response = response;
+    }
+
+    @Override
+    public CacheDirectiveEntry get(int i) {
+      return PBHelperClient.convert(response.getElements(i));
+    }
+
+    @Override
+    public int size() {
+      return response.getElementsCount();
+    }
+    
+    @Override
+    public boolean hasMore() {
+      return response.getHasMore();
+    }
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry>
+      listCacheDirectives(long prevId,
+          CacheDirectiveInfo filter) throws IOException {
+    if (filter == null) {
+      filter = new CacheDirectiveInfo.Builder().build();
+    }
+    try {
+      return new BatchedCacheEntries(
+        rpcProxy.listCacheDirectives(null,
+          ListCacheDirectivesRequestProto.newBuilder().
+            setPrevId(prevId).
+            setFilter(PBHelperClient.convert(filter)).
+            build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    AddCachePoolRequestProto.Builder builder = 
+        AddCachePoolRequestProto.newBuilder();
+    builder.setInfo(PBHelperClient.convert(info));
+    try {
+      rpcProxy.addCachePool(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyCachePool(CachePoolInfo req) throws IOException {
+    ModifyCachePoolRequestProto.Builder builder = 
+        ModifyCachePoolRequestProto.newBuilder();
+    builder.setInfo(PBHelperClient.convert(req));
+    try {
+      rpcProxy.modifyCachePool(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeCachePool(String cachePoolName) throws IOException {
+    try {
+      rpcProxy.removeCachePool(null, 
+          RemoveCachePoolRequestProto.newBuilder().
+            setPoolName(cachePoolName).build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private static class BatchedCachePoolEntries
+    implements BatchedEntries<CachePoolEntry> {
+      private final ListCachePoolsResponseProto proto;
+    
+    public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
+      this.proto = proto;
+    }
+      
+    @Override
+    public CachePoolEntry get(int i) {
+      CachePoolEntryProto elem = proto.getEntries(i);
+      return PBHelperClient.convert(elem);
+    }
+
+    @Override
+    public int size() {
+      return proto.getEntriesCount();
+    }
+    
+    @Override
+    public boolean hasMore() {
+      return proto.getHasMore();
+    }
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    try {
+      return new BatchedCachePoolEntries(
+        rpcProxy.listCachePools(null,
+          ListCachePoolsRequestProto.newBuilder().
+            setPrevPoolName(prevKey).build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto
+        .newBuilder().setSrc(src)
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+    try {
+      rpcProxy.modifyAclEntries(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto
+        .newBuilder().setSrc(src)
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+    try {
+      rpcProxy.removeAclEntries(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeDefaultAcl(String src) throws IOException {
+    RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto
+        .newBuilder().setSrc(src).build();
+    try {
+      rpcProxy.removeDefaultAcl(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeAcl(String src) throws IOException {
+    RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      rpcProxy.removeAcl(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+    SetAclRequestProto req = SetAclRequestProto.newBuilder()
+        .setSrc(src)
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
+        .build();
+    try {
+      rpcProxy.setAcl(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public AclStatus getAclStatus(String src) throws IOException {
+    GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void createEncryptionZone(String src, String keyName)
+    throws IOException {
+    final CreateEncryptionZoneRequestProto.Builder builder =
+      CreateEncryptionZoneRequestProto.newBuilder();
+    builder.setSrc(src);
+    if (keyName != null && !keyName.isEmpty()) {
+      builder.setKeyName(keyName);
+    }
+    CreateEncryptionZoneRequestProto req = builder.build();
+    try {
+      rpcProxy.createEncryptionZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public EncryptionZone getEZForPath(String src)
+      throws IOException {
+    final GetEZForPathRequestProto.Builder builder =
+        GetEZForPathRequestProto.newBuilder();
+    builder.setSrc(src);
+    final GetEZForPathRequestProto req = builder.build();
+    try {
+      final EncryptionZonesProtos.GetEZForPathResponseProto response =
+          rpcProxy.getEZForPath(null, req);
+      if (response.hasZone()) {
+        return PBHelperClient.convert(response.getZone());
+      } else {
+        return null;
+      }
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
+      throws IOException {
+    final ListEncryptionZonesRequestProto req =
+      ListEncryptionZonesRequestProto.newBuilder()
+          .setId(id)
+          .build();
+    try {
+      EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
+          rpcProxy.listEncryptionZones(null, req);
+      List<EncryptionZone> elements =
+          Lists.newArrayListWithCapacity(response.getZonesCount());
+      for (EncryptionZoneProto p : response.getZonesList()) {
+        elements.add(PBHelperClient.convert(p));
+      }
+      return new BatchedListEntries<EncryptionZone>(elements,
+          response.getHasMore());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+      throws IOException {
+    SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
+        .setSrc(src)
+        .setXAttr(PBHelperClient.convertXAttrProto(xAttr))
+        .setFlag(PBHelperClient.convert(flag))
+        .build();
+    try {
+      rpcProxy.setXAttr(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+      throws IOException {
+    GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder();
+    builder.setSrc(src);
+    if (xAttrs != null) {
+      builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
+    }
+    GetXAttrsRequestProto req = builder.build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getXAttrs(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public List<XAttr> listXAttrs(String src)
+      throws IOException {
+    ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
+    builder.setSrc(src);
+    ListXAttrsRequestProto req = builder.build();
+    try {
+      return PBHelperClient.convert(rpcProxy.listXAttrs(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeXAttr(String src, XAttr xAttr) throws IOException {
+    RemoveXAttrRequestProto req = RemoveXAttrRequestProto
+        .newBuilder().setSrc(src)
+        .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build();
+    try {
+      rpcProxy.removeXAttr(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void checkAccess(String path, FsAction mode) throws IOException {
+    CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder()
+        .setPath(path).setMode(PBHelperClient.convert(mode)).build();
+    try {
+      rpcProxy.checkAccess(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
+        .newBuilder().setSrc(src).setPolicyName(policyName).build();
+    try {
+      rpcProxy.setStoragePolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+    GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto
+        .newBuilder().setPath(path).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getStoragePolicy(null, request)
+          .getStoragePolicy());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    try {
+      GetStoragePoliciesResponseProto response = rpcProxy
+          .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
+      return PBHelperClient.convertStoragePolicies(response.getPoliciesList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  public long getCurrentEditLogTxid() throws IOException {
+    GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
+        .getDefaultInstance();
+    try {
+      return rpcProxy.getCurrentEditLogTxid(null, req).getTxid();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public EventBatchList getEditsFromTxid(long txid) throws IOException {
+    GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder()
+        .setTxid(txid).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getEditsFromTxid(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
new file mode 100644
index 0000000..78cd160
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.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.namenode.ha;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+
+public abstract class AbstractNNFailoverProxyProvider<T> implements
+   FailoverProxyProvider <T> {
+
+  private AtomicBoolean fallbackToSimpleAuth;
+
+  /**
+   * Inquire whether logical HA URI is used for the implementation. If it is
+   * used, a special token handling may be needed to make sure a token acquired 
+   * from a node in the HA pair can be used against the other node. 
+   *
+   * @return true if logical HA URI is used. false, if not used.
+   */
+  public abstract boolean useLogicalURI(); 
+
+  /**
+   * Set for tracking if a secure client falls back to simple auth.  This method
+   * is synchronized only to stifle a Findbugs warning.
+   *
+   * @param fallbackToSimpleAuth - set to true or false during this method to
+   *   indicate if a secure client falls back to simple auth
+   */
+  public synchronized void setFallbackToSimpleAuth(
+      AtomicBoolean fallbackToSimpleAuth) {
+    this.fallbackToSimpleAuth = fallbackToSimpleAuth;
+  }
+
+  public synchronized AtomicBoolean getFallbackToSimpleAuth() {
+    return fallbackToSimpleAuth;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
new file mode 100644
index 0000000..0b387b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.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.namenode.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+
+/**
+ * A NNFailoverProxyProvider implementation which wrapps old implementations
+ * directly implementing the {@link FailoverProxyProvider} interface.
+ *
+ * It is assumed that the old impelmentation is using logical URI.
+ */
+public class WrappedFailoverProxyProvider<T> extends
+    AbstractNNFailoverProxyProvider<T> {
+  private final FailoverProxyProvider<T> proxyProvider;
+  
+  /**
+   * Wrap the given instance of an old FailoverProxyProvider.
+   */
+  public WrappedFailoverProxyProvider(FailoverProxyProvider<T> provider) {
+    proxyProvider = provider;
+  }
+    
+  @Override
+  public Class<T> getInterface() {
+    return proxyProvider.getInterface();
+  }
+
+  @Override
+  public synchronized ProxyInfo<T> getProxy() {
+    return proxyProvider.getProxy();
+  }
+
+  @Override
+  public void performFailover(T currentProxy) {
+    proxyProvider.performFailover(currentProxy);
+  }
+
+  /**
+   * Close the proxy,
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    proxyProvider.close();
+  }
+
+  /**
+   * Assume logical URI is used for old proxy provider implementations.
+   */
+  @Override
+  public boolean useLogicalURI() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index cf54cd2..0718a3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -944,6 +944,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12428. Fix inconsistency between log-level guards and statements.
     (Jagadesh Kiran N and Jackie Chang via ozawa)
 
+    HDFS-9039. Separate client and server side methods of o.a.h.hdfs.
+    NameNodeProxies. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than