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 jl...@apache.org on 2013/07/17 02:06:48 UTC

svn commit: r1503954 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ dev-support/ src/main/java/org/apache/hadoop/security/protocolPB/ src/main/java/org/apache/hadoop/tools/protocolPB/ src/main/proto/

Author: jlowe
Date: Wed Jul 17 00:06:48 2013
New Revision: 1503954

URL: http://svn.apache.org/r1503954
Log:
HADOOP-9734. Common protobuf definitions for GetUserMappingsProtocol, RefreshAuthorizationPolicyProtocol and RefreshUserMappingsProtocol. Contributed by Jason Lowe

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java   (with props)
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
    hadoop/common/trunk/hadoop-common-project/hadoop-common/pom.xml

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1503954&r1=1503953&r2=1503954&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Wed Jul 17 00:06:48 2013
@@ -474,6 +474,9 @@ Release 2.1.0-beta - 2013-07-02
     HADOOP-9720. Rename Client#uuid to Client#clientId. (Arpit Agarwal via
     suresh)
 
+    HADOOP-9734. Common protobuf definitions for GetUserMappingsProtocol,
+    RefreshAuthorizationPolicyProtocol and RefreshUserMappingsProtocol (jlowe)
+
   OPTIMIZATIONS
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml?rev=1503954&r1=1503953&r2=1503954&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml Wed Jul 17 00:06:48 2013
@@ -19,6 +19,12 @@
        <Package name="org.apache.hadoop.record.compiler.generated" />
      </Match>
      <Match>
+       <Package name="org.apache.hadoop.security.proto" />
+     </Match>
+     <Match>
+       <Package name="org.apache.hadoop.tools.proto" />
+     </Match>
+     <Match>
        <Bug pattern="EI_EXPOSE_REP" />
      </Match>
      <Match>

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/pom.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/pom.xml?rev=1503954&r1=1503953&r2=1503954&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/pom.xml (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/pom.xml Wed Jul 17 00:06:48 2013
@@ -321,6 +321,9 @@
                   <include>ZKFCProtocol.proto</include>
                   <include>ProtobufRpcEngine.proto</include>
                   <include>Security.proto</include>
+                  <include>GetUserMappingsProtocol.proto</include>
+                  <include>RefreshAuthorizationPolicyProtocol.proto</include>
+                  <include>RefreshUserMappingsProtocol.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
+import org.apache.hadoop.security.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclRequestProto;
+import org.apache.hadoop.security.protocolPB.RefreshAuthorizationPolicyProtocolPB;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, RefreshAuthorizationPolicyProtocol, Closeable {
+
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final RefreshAuthorizationPolicyProtocolPB rpcProxy;
+  
+  private final static RefreshServiceAclRequestProto
+  VOID_REFRESH_SERVICE_ACL_REQUEST =
+      RefreshServiceAclRequestProto.newBuilder().build();
+
+  public RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
+      RefreshAuthorizationPolicyProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public void refreshServiceAcl() throws IOException {
+    try {
+      rpcProxy.refreshServiceAcl(NULL_CONTROLLER,
+          VOID_REFRESH_SERVICE_ACL_REQUEST);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        RefreshAuthorizationPolicyProtocolPB.class,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
+        methodName);
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshAuthorizationPolicyProtocolService;
+
+@KerberosInfo(
+    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol", 
+    protocolVersion = 1)
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public interface RefreshAuthorizationPolicyProtocolPB extends
+  RefreshAuthorizationPolicyProtocolService.BlockingInterface {
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java Wed Jul 17 00:06:48 2013
@@ -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.security.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
+import org.apache.hadoop.security.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclRequestProto;
+import org.apache.hadoop.security.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclResponseProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RefreshAuthorizationPolicyProtocolServerSideTranslatorPB implements
+    RefreshAuthorizationPolicyProtocolPB {
+
+  private final RefreshAuthorizationPolicyProtocol impl;
+
+  private final static RefreshServiceAclResponseProto
+  VOID_REFRESH_SERVICE_ACL_RESPONSE = RefreshServiceAclResponseProto
+      .newBuilder().build();
+
+  public RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(
+      RefreshAuthorizationPolicyProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public RefreshServiceAclResponseProto refreshServiceAcl(
+      RpcController controller, RefreshServiceAclRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshServiceAcl();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_REFRESH_SERVICE_ACL_RESPONSE;
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, RefreshUserMappingsProtocol, Closeable {
+
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final RefreshUserMappingsProtocolPB rpcProxy;
+  
+  private final static RefreshUserToGroupsMappingsRequestProto 
+  VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST = 
+      RefreshUserToGroupsMappingsRequestProto.newBuilder().build();
+
+  private final static RefreshSuperUserGroupsConfigurationRequestProto
+  VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST = 
+      RefreshSuperUserGroupsConfigurationRequestProto.newBuilder().build();
+
+  public RefreshUserMappingsProtocolClientSideTranslatorPB(
+      RefreshUserMappingsProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public void refreshUserToGroupsMappings() throws IOException {
+    try {
+      rpcProxy.refreshUserToGroupsMappings(NULL_CONTROLLER,
+          VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public void refreshSuperUserGroupsConfiguration() throws IOException {
+    try {
+      rpcProxy.refreshSuperUserGroupsConfiguration(NULL_CONTROLLER,
+          VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil
+        .isMethodSupported(rpcProxy, RefreshUserMappingsProtocolPB.class,
+            RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+            RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
+            methodName);
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService;
+
+@KerberosInfo(
+    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.security.RefreshUserMappingsProtocol", 
+    protocolVersion = 1)
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public interface RefreshUserMappingsProtocolPB extends
+    RefreshUserMappingsProtocolService.BlockingInterface {
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationResponseProto;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
+import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsResponseProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RefreshUserMappingsProtocolServerSideTranslatorPB implements RefreshUserMappingsProtocolPB {
+
+  private final RefreshUserMappingsProtocol impl;
+  
+  private final static RefreshUserToGroupsMappingsResponseProto 
+  VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE =
+      RefreshUserToGroupsMappingsResponseProto.newBuilder().build();
+
+  private final static RefreshSuperUserGroupsConfigurationResponseProto
+  VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE = 
+      RefreshSuperUserGroupsConfigurationResponseProto.newBuilder()
+      .build();
+
+  public RefreshUserMappingsProtocolServerSideTranslatorPB(RefreshUserMappingsProtocol impl) {
+    this.impl = impl;
+  }
+  
+  @Override
+  public RefreshUserToGroupsMappingsResponseProto 
+      refreshUserToGroupsMappings(RpcController controller, 
+      RefreshUserToGroupsMappingsRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshUserToGroupsMappings();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE;
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponseProto 
+      refreshSuperUserGroupsConfiguration(RpcController controller,
+      RefreshSuperUserGroupsConfigurationRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshSuperUserGroupsConfiguration();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE;
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,70 @@
+/**
+ * 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.tools.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import org.apache.hadoop.ipc.ProtobufHelper;
+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.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
+import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class GetUserMappingsProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, GetUserMappingsProtocol, Closeable {
+
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final GetUserMappingsProtocolPB rpcProxy;
+  
+  public GetUserMappingsProtocolClientSideTranslatorPB(
+      GetUserMappingsProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    GetGroupsForUserRequestProto request = GetGroupsForUserRequestProto
+        .newBuilder().setUser(user).build();
+    GetGroupsForUserResponseProto resp;
+    try {
+      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(rpcProxy,
+        GetUserMappingsProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.tools.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
+
+@KerberosInfo(
+    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.tools.GetUserMappingsProtocol", 
+    protocolVersion = 1)
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public interface GetUserMappingsProtocolPB extends
+  GetUserMappingsProtocolService.BlockingInterface {
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java Wed Jul 17 00:06:48 2013
@@ -0,0 +1,57 @@
+/**
+ * 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.tools.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
+import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
+import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class GetUserMappingsProtocolServerSideTranslatorPB implements
+    GetUserMappingsProtocolPB {
+
+  private final GetUserMappingsProtocol impl;
+
+  public GetUserMappingsProtocolServerSideTranslatorPB(
+      GetUserMappingsProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public GetGroupsForUserResponseProto getGroupsForUser(
+      RpcController controller, GetGroupsForUserRequestProto request)
+      throws ServiceException {
+    String[] groups;
+    try {
+      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();
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto Wed Jul 17 00:06:48 2013
@@ -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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.tools.proto";
+option java_outer_classname = "GetUserMappingsProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.common;
+
+/**
+ *  Get groups for user request.
+ */
+message GetGroupsForUserRequestProto {
+  required string user = 1;
+}
+
+/**
+ * Response for get groups.
+ */
+message GetGroupsForUserResponseProto {
+  repeated string groups = 1;
+}
+
+
+/**
+ * Protocol which maps users to groups.
+ */
+service GetUserMappingsProtocolService {
+  /**
+   * Get the groups which are mapped to the given user.
+   */
+  rpc getGroupsForUser(GetGroupsForUserRequestProto)
+      returns(GetGroupsForUserResponseProto);
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto Wed Jul 17 00:06:48 2013
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.security.proto";
+option java_outer_classname = "RefreshAuthorizationPolicyProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.common;
+
+/**
+ *  Refresh service acl request.
+ */
+message RefreshServiceAclRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshServiceAclResponseProto {
+}
+
+/**
+ * Protocol which is used to refresh the authorization policy in use currently.
+ */
+service RefreshAuthorizationPolicyProtocolService {
+  /**
+   * Refresh the service-level authorization policy in-effect.
+   */
+  rpc refreshServiceAcl(RefreshServiceAclRequestProto)
+      returns(RefreshServiceAclResponseProto);
+}

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto?rev=1503954&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto Wed Jul 17 00:06:48 2013
@@ -0,0 +1,70 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.security.proto";
+option java_outer_classname = "RefreshUserMappingsProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.common;
+
+/**
+ *  Refresh user to group mappings request.
+ */
+message RefreshUserToGroupsMappingsRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshUserToGroupsMappingsResponseProto {
+}
+
+/**
+ * Refresh superuser configuration request.
+ */
+message RefreshSuperUserGroupsConfigurationRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshSuperUserGroupsConfigurationResponseProto {
+}
+
+/**
+ * Protocol to refresh the user mappings.
+ */
+service RefreshUserMappingsProtocolService {
+  /**
+   * Refresh user to group mappings.
+   */
+  rpc refreshUserToGroupsMappings(RefreshUserToGroupsMappingsRequestProto)
+      returns(RefreshUserToGroupsMappingsResponseProto);
+
+  /**
+   * Refresh superuser proxy group list.
+   */
+  rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto)
+      returns(RefreshSuperUserGroupsConfigurationResponseProto);
+}