Browse Source

Reverting HADOOP-8805

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1387697 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 years ago
parent
commit
8bf6b80f2b

+ 0 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -240,8 +240,6 @@ Release 2.0.3-alpha - Unreleased
 
     HADOOP-8812. ExitUtil#terminate should print Exception#toString. (eli)
 
-    HADOOP-8805. Move protocol buffer implementation of GetUserMappingProtocol from HDFS to Common. (bowang via tucu)
-
   OPTIMIZATIONS
 
   BUG FIXES

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -36,6 +36,8 @@ 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.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
@@ -65,8 +67,6 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tools.GetUserMappingsProtocolPB;
-import org.apache.hadoop.tools.impl.pb.client.GetUserMappingsProtocolPBClientImpl;
 
 import com.google.common.base.Preconditions;
 
@@ -218,7 +218,7 @@ public class NameNodeProxies {
       throws IOException {
     GetUserMappingsProtocolPB proxy = (GetUserMappingsProtocolPB)
         createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class, 0);
-    return new GetUserMappingsProtocolPBClientImpl(proxy);
+    return new GetUserMappingsProtocolClientSideTranslatorPB(proxy);
   }
   
   private static NamenodeProtocol createNNProxyWithNamenodeProtocol(

+ 23 - 35
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/impl/pb/client/GetUserMappingsProtocolPBClientImpl.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java

@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -16,66 +16,54 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.tools.impl.pb.client;
+package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
 import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tools.GetUserMappingsProtocolPB;
-import org.apache.hadoop.tools.proto.GetUserMappingsProtocol.GetGroupsForUserRequestProto;
-import org.apache.hadoop.tools.proto.GetUserMappingsProtocol.GetGroupsForUserResponseProto;
 
+import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
-public class GetUserMappingsProtocolPBClientImpl implements
+public class GetUserMappingsProtocolClientSideTranslatorPB implements
     ProtocolMetaInterface, GetUserMappingsProtocol, Closeable {
 
-  private GetUserMappingsProtocolPB proxy;
-  
-  public GetUserMappingsProtocolPBClientImpl(
-      long clientVersion, InetSocketAddress addr, Configuration conf)
-      throws IOException {
-    RPC.setProtocolEngine(conf, GetUserMappingsProtocolPB.class,
-        ProtobufRpcEngine.class);
-    proxy = (GetUserMappingsProtocolPB) RPC.getProxy(
-        GetUserMappingsProtocolPB.class, clientVersion, addr, conf);
-  }
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final GetUserMappingsProtocolPB rpcProxy;
   
-  public GetUserMappingsProtocolPBClientImpl(
-      GetUserMappingsProtocolPB proxy) {
-    this.proxy = proxy;
+  public GetUserMappingsProtocolClientSideTranslatorPB(
+      GetUserMappingsProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
   }
-  
+
   @Override
   public void close() throws IOException {
-    RPC.stopProxy(proxy);
+    RPC.stopProxy(rpcProxy);
   }
-  
+
   @Override
   public String[] getGroupsForUser(String user) throws IOException {
-    GetGroupsForUserRequestProto requestProto = 
-        GetGroupsForUserRequestProto.newBuilder().setUser(user).build();
+    GetGroupsForUserRequestProto request = GetGroupsForUserRequestProto
+        .newBuilder().setUser(user).build();
+    GetGroupsForUserResponseProto resp;
     try {
-      GetGroupsForUserResponseProto responseProto =
-          proxy.getGroupsForUser(null, requestProto);
-      return (String[]) responseProto.getGroupsList().toArray(
-          new String[responseProto.getGroupsCount()]);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
+      resp = rpcProxy.getGroupsForUser(NULL_CONTROLLER, request);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
     }
+    return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]);
   }
 
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
-    return RpcClientUtil.isMethodSupported(proxy,
+    return RpcClientUtil.isMethodSupported(rpcProxy,
         GetUserMappingsProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
   }

+ 5 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocolPB.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java

@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -16,21 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.tools;
+package org.apache.hadoop.hdfs.protocolPB;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.tools.proto.GetUserMappingsProtocol.GetUserMappingsProtocolService;
 
 @KerberosInfo(
     serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
 @ProtocolInfo(
-    protocolName = "org.apache.hadoop.tools.GetUserMappingsProtocol",
+    protocolName = "org.apache.hadoop.tools.GetUserMappingsProtocol", 
     protocolVersion = 1)
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN"})
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public interface GetUserMappingsProtocolPB extends
   GetUserMappingsProtocolService.BlockingInterface {

+ 19 - 20
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/impl/pb/service/GetUserMappingsProtocolPBServiceImpl.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java

@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -16,43 +16,42 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.tools.impl.pb.service;
+package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tools.GetUserMappingsProtocolPB;
-import org.apache.hadoop.tools.proto.GetUserMappingsProtocol.GetGroupsForUserRequestProto;
-import org.apache.hadoop.tools.proto.GetUserMappingsProtocol.GetGroupsForUserResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
-public class GetUserMappingsProtocolPBServiceImpl implements
+public class GetUserMappingsProtocolServerSideTranslatorPB implements
     GetUserMappingsProtocolPB {
 
-  private GetUserMappingsProtocol real;
-  
-  public GetUserMappingsProtocolPBServiceImpl(GetUserMappingsProtocol impl) {
-    this.real = impl;
+  private final GetUserMappingsProtocol impl;
+
+  public GetUserMappingsProtocolServerSideTranslatorPB(
+      GetUserMappingsProtocol impl) {
+    this.impl = impl;
   }
-  
+
   @Override
   public GetGroupsForUserResponseProto getGroupsForUser(
       RpcController controller, GetGroupsForUserRequestProto request)
       throws ServiceException {
-    String user = request.getUser();
+    String[] groups;
     try {
-      String[] groups = real.getGroupsForUser(user);
-      GetGroupsForUserResponseProto.Builder responseBuilder =
-          GetGroupsForUserResponseProto.newBuilder();
-      for (String group : groups) {
-        responseBuilder.addGroups(group);
-      }
-      return responseBuilder.build();
+      groups = impl.getGroupsForUser(request.getUser());
     } catch (IOException e) {
       throw new ServiceException(e);
     }
+    GetGroupsForUserResponseProto.Builder builder = GetGroupsForUserResponseProto
+        .newBuilder();
+    for (String g : groups) {
+      builder.addGroups(g);
+    }
+    return builder.build();
   }
-
 }

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshAuthorizationPolicyProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService;
@@ -73,6 +74,8 @@ import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolPB;
@@ -116,9 +119,6 @@ import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.tools.GetUserMappingsProtocolPB;
-import org.apache.hadoop.tools.impl.pb.service.GetUserMappingsProtocolPBServiceImpl;
-import org.apache.hadoop.tools.proto.GetUserMappingsProtocol.GetUserMappingsProtocolService;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 
@@ -189,8 +189,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
     BlockingService refreshUserMappingService = RefreshUserMappingsProtocolService
         .newReflectiveBlockingService(refreshUserMappingXlator);
 
-    GetUserMappingsProtocolPBServiceImpl getUserMappingXlator = 
-        new GetUserMappingsProtocolPBServiceImpl(this);
+    GetUserMappingsProtocolServerSideTranslatorPB getUserMappingXlator = 
+        new GetUserMappingsProtocolServerSideTranslatorPB(this);
     BlockingService getUserMappingService = GetUserMappingsProtocolService
         .newReflectiveBlockingService(getUserMappingXlator);
     

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto → hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto

@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
- 
-option java_package = "org.apache.hadoop.tools.proto";
-option java_outer_classname = "GetUserMappingsProtocol";
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "GetUserMappingsProtocolProtos";
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 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;
@@ -40,7 +41,6 @@ import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tools.impl.pb.client.GetUserMappingsProtocolPBClientImpl;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -141,8 +141,8 @@ public class TestIsMethodSupported {
   
   @Test
   public void testGetUserMappingsProtocol() throws IOException {
-    GetUserMappingsProtocolPBClientImpl translator = 
-        (GetUserMappingsProtocolPBClientImpl)
+    GetUserMappingsProtocolClientSideTranslatorPB translator = 
+        (GetUserMappingsProtocolClientSideTranslatorPB)
         NameNodeProxies.createNonHAProxy(conf, nnAddress,
             GetUserMappingsProtocol.class, UserGroupInformation.getCurrentUser(),
             true).getProxy();