Browse Source

HDFS-2801. Provide a method in client side translators to check for a methods supported in underlying protocol.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1237192 13f79535-47bb-0310-9956-ffa450edef68
Jitendra Nath Pandey 13 years ago
parent
commit
98302971c2
13 changed files with 267 additions and 24 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  3. 12 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  4. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  5. 14 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  6. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
  7. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  8. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
  9. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
  10. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
  11. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
  13. 154 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -104,6 +104,9 @@ Trunk (unreleased changes)
 
     HDFS-2669 Enable protobuf rpc for ClientNamenodeProtocol
 
+    HDFS-2801. Provide a method in client side translators to check for a 
+    methods supported in underlying protocol. (jitendra)
+
   OPTIMIZATIONS
     HDFS-2477. Optimize computing the diff between a block report and the
     namenode state. (Tomasz Nykiel via hairong)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -73,7 +73,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public RefreshNamenodesResponseProto refreshNamenode(
+  public RefreshNamenodesResponseProto refreshNamenodes(
       RpcController unused, RefreshNamenodesRequestProto request)
       throws ServiceException {
     try {

+ 12 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -43,8 +43,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -60,7 +63,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class ClientDatanodeProtocolTranslatorPB implements
-    ClientDatanodeProtocol, Closeable {
+    ProtocolMetaInterface, ClientDatanodeProtocol, Closeable {
   public static final Log LOG = LogFactory
       .getLog(ClientDatanodeProtocolTranslatorPB.class);
   
@@ -168,7 +171,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
   @Override
   public void refreshNamenodes() throws IOException {
     try {
-      rpcProxy.refreshNamenode(NULL_CONTROLLER, REFRESH_NAMENODES);
+      rpcProxy.refreshNamenodes(NULL_CONTROLLER, REFRESH_NAMENODES);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -201,4 +204,11 @@ public class ClientDatanodeProtocolTranslatorPB implements
     return new BlockLocalPathInfo(PBHelper.convert(resp.getBlock()),
         resp.getLocalPath(), resp.getLocalMetaPath());
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
+  }
 }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -65,9 +65,12 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -138,7 +141,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class ClientNamenodeProtocolTranslatorPB implements
-    ClientProtocol, Closeable {
+    ProtocolMetaInterface, ClientProtocol, Closeable {
   final private ClientNamenodeProtocolPB rpcProxy;
 
   private static ClientNamenodeProtocolPB createNamenode(
@@ -874,4 +877,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+  }
 }

+ 14 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -61,9 +61,12 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -77,14 +80,14 @@ import com.google.protobuf.ServiceException;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
-public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
-    Closeable {
+public class DatanodeProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, DatanodeProtocol, Closeable {
   
   /** RpcController is not used and hence is set to null */
-  private final static RpcController NULL_CONTROLLER = null;
   private final DatanodeProtocolPB rpcProxy;
   private static final VersionRequestProto VERSION_REQUEST = 
       VersionRequestProto.newBuilder().build();
+  private final static RpcController NULL_CONTROLLER = null;
   
   public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
       Configuration conf) throws IOException {
@@ -299,4 +302,12 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
       throw ProtobufHelper.getRemoteException(se);
     }
   }
+
+  @Override // ProtocolMetaInterface
+  public boolean isMethodSupported(String methodName)
+      throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, DatanodeProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(DatanodeProtocolPB.class), methodName);
+  }
 }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java

@@ -29,8 +29,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
@@ -39,7 +42,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
 public class GetUserMappingsProtocolClientSideTranslatorPB implements
-    GetUserMappingsProtocol, Closeable {
+    ProtocolMetaInterface, GetUserMappingsProtocol, Closeable {
 
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
@@ -86,4 +89,11 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
     }
     return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]);
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        GetUserMappingsProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
+  }
 }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -37,8 +37,11 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import com.google.protobuf.RpcController;
@@ -52,7 +55,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class InterDatanodeProtocolTranslatorPB implements
-    InterDatanodeProtocol, Closeable {
+    ProtocolMetaInterface, InterDatanodeProtocol, Closeable {
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   final private InterDatanodeProtocolPB rpcProxy;
@@ -116,4 +119,11 @@ public class InterDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        InterDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(InterDatanodeProtocolPB.class), methodName);
+  }
 }

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java

@@ -31,8 +31,11 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -44,7 +47,8 @@ import com.google.protobuf.ServiceException;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
-public class JournalProtocolTranslatorPB implements JournalProtocol, Closeable {
+public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
+    JournalProtocol, Closeable {
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final JournalProtocolPB rpcProxy;
@@ -103,4 +107,11 @@ public class JournalProtocolTranslatorPB implements JournalProtocol, Closeable {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, JournalProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(JournalProtocolPB.class), methodName);
+  }
 }

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -20,17 +20,12 @@ package org.apache.hadoop.hdfs.protocolPB;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
@@ -45,21 +40,19 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpo
 import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-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.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -74,7 +67,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
-    Closeable {
+    ProtocolMetaInterface, Closeable {
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   
@@ -242,4 +235,11 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, NamenodeProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(NamenodeProtocolPB.class), methodName);
+  }
 }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java

@@ -28,8 +28,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
@@ -38,7 +41,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
 public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
-    RefreshAuthorizationPolicyProtocol, Closeable {
+    ProtocolMetaInterface, RefreshAuthorizationPolicyProtocol, Closeable {
 
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
@@ -83,4 +86,13 @@ public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
       throw ProtobufHelper.getRemoteException(se);
     }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        RefreshAuthorizationPolicyProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
+        methodName);
+  }
 }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java

@@ -29,8 +29,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -39,7 +42,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
 public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
-    RefreshUserMappingsProtocol, Closeable {
+    ProtocolMetaInterface, RefreshUserMappingsProtocol, Closeable {
 
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
@@ -95,4 +98,13 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
       throw ProtobufHelper.getRemoteException(se);
     }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil
+        .isMethodSupported(rpcProxy, RefreshUserMappingsProtocolPB.class,
+            RpcKind.RPC_PROTOCOL_BUFFER,
+            RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
+            methodName);
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto

@@ -104,7 +104,7 @@ service ClientDatanodeProtocolService {
    * Refresh the list of federated namenodes from updated configuration.
    * Adds new namenodes and stops the deleted namenodes.
    */
-  rpc refreshNamenode(RefreshNamenodesRequestProto)
+  rpc refreshNamenodes(RefreshNamenodesRequestProto)
       returns(RefreshNamenodesResponseProto);
 
   /**

+ 154 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java

@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.net.InetSocketAddress;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test cases to verify that client side translators correctly implement the
+ * isMethodSupported method in ProtocolMetaInterface.
+ */
+public class TestIsMethodSupported {
+  private static MiniDFSCluster cluster = null;
+  private static HdfsConfiguration conf = new HdfsConfiguration();
+  private static InetSocketAddress nnAddress = null;
+  private static InetSocketAddress dnAddress = null;
+  
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = (new MiniDFSCluster.Builder(conf)).numNameNodes(1)
+        .numDataNodes(1).build();
+    nnAddress = cluster.getNameNode().getNameNodeAddress();
+    dnAddress = new InetSocketAddress(cluster.getDataNodes().get(0)
+        .getDatanodeId().getHost(), cluster.getDataNodes().get(0).getIpcPort());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testNamenodeProtocol() throws IOException {
+    NamenodeProtocolTranslatorPB translator =
+        new NamenodeProtocolTranslatorPB(nnAddress, conf,
+            UserGroupInformation.getCurrentUser());
+    boolean exists = translator.isMethodSupported("rollEditLog");
+    Assert.assertTrue(exists);
+    exists = translator.isMethodSupported("bogusMethod");
+    Assert.assertFalse(exists);
+  }
+  
+  @Test
+  public void testDatanodeProtocol() throws IOException {
+    DatanodeProtocolClientSideTranslatorPB translator = 
+        new DatanodeProtocolClientSideTranslatorPB(nnAddress, conf);
+    Assert.assertTrue(translator.isMethodSupported("sendHeartbeat"));
+  }
+  
+  @Test
+  public void testClientDatanodeProtocol() throws IOException {
+    ClientDatanodeProtocolTranslatorPB translator = 
+        new ClientDatanodeProtocolTranslatorPB(nnAddress, 
+            UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+    //Namenode doesn't implement ClientDatanodeProtocol
+    Assert.assertFalse(translator.isMethodSupported("refreshNamenodes"));
+    
+    translator = new ClientDatanodeProtocolTranslatorPB(
+        dnAddress, UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+    Assert.assertTrue(translator.isMethodSupported("refreshNamenodes"));
+  }
+  
+  @Test
+  public void testClientNamenodeProtocol() throws IOException {
+    ClientNamenodeProtocolTranslatorPB translator = 
+        new ClientNamenodeProtocolTranslatorPB(nnAddress, conf, 
+            UserGroupInformation.getCurrentUser()); 
+    Assert.assertTrue(translator.isMethodSupported("mkdirs"));
+  }
+  
+  @Test
+  public void tesJournalProtocol() throws IOException {
+    JournalProtocolTranslatorPB translator = 
+        new JournalProtocolTranslatorPB(nnAddress, conf);
+    //Nameode doesn't implement JournalProtocol
+    Assert.assertFalse(translator.isMethodSupported("startLogSegment"));
+  }
+  
+  @Test
+  public void testInterDatanodeProtocol() throws IOException {
+    InterDatanodeProtocolTranslatorPB translator = 
+        new InterDatanodeProtocolTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf,
+            NetUtils.getDefaultSocketFactory(conf), 0);
+    //Not supported at namenode
+    Assert.assertFalse(translator.isMethodSupported("initReplicaRecovery"));
+    
+    translator = new InterDatanodeProtocolTranslatorPB(
+        dnAddress, UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf), 0);
+    Assert.assertTrue(translator.isMethodSupported("initReplicaRecovery"));
+  }
+  
+  @Test
+  public void testGetUserMappingsProtocol() throws IOException {
+    GetUserMappingsProtocolClientSideTranslatorPB translator = 
+        new GetUserMappingsProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(translator.isMethodSupported("getGroupsForUser"));
+  }
+  
+  @Test
+  public void testRefreshAuthorizationPolicyProtocol() throws IOException {
+    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB translator =
+        new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(translator.isMethodSupported("refreshServiceAcl"));
+  }
+  
+  @Test
+  public void testRefreshUserMappingsProtocol() throws IOException {
+    RefreshUserMappingsProtocolClientSideTranslatorPB translator =
+        new RefreshUserMappingsProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(
+        translator.isMethodSupported("refreshUserToGroupsMappings"));
+  }
+}