You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by vi...@apache.org on 2012/10/08 23:21:39 UTC

svn commit: r1395781 [2/2] - in /hadoop/common/trunk/hadoop-yarn-project: ./ hadoop-yarn/bin/ hadoop-yarn/hadoop-yarn-api/ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop...

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java?rev=1395781&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/GetGroupsForTesting.java Mon Oct  8 21:21:38 2012
@@ -0,0 +1,77 @@
+/**
+ * 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.yarn.client;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.GetGroupsBase;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.RMAdminProtocol;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+
+public class GetGroupsForTesting extends GetGroupsBase {
+  
+  public GetGroupsForTesting(Configuration conf) {
+    super(conf);
+  }
+
+  public GetGroupsForTesting(Configuration conf, PrintStream out) {
+    super(conf, out);
+  }
+  
+  @Override
+  protected InetSocketAddress getProtocolAddress(Configuration conf)
+      throws IOException {
+    return conf.getSocketAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
+  }
+  
+  @Override
+  public void setConf(Configuration conf) {
+    conf = new YarnConfiguration(conf);
+    super.setConf(conf);
+  }
+  
+  @Override
+  protected GetUserMappingsProtocol getUgmProtocol() throws IOException {
+    Configuration conf = getConf();
+    
+    final InetSocketAddress addr = conf.getSocketAddr(
+        YarnConfiguration.RM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
+    final YarnRPC rpc = YarnRPC.create(conf);
+    
+    RMAdminProtocol adminProtocol = (RMAdminProtocol) rpc.getProxy(
+        RMAdminProtocol.class, addr, getConf());
+
+    return adminProtocol;
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int res = ToolRunner.run(new GetGroupsForTesting(new YarnConfiguration()), argv);
+    System.exit(res);
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java?rev=1395781&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java Mon Oct  8 21:21:38 2012
@@ -0,0 +1,95 @@
+/**
+ * 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.yarn.client;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.GetGroupsTestBase;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+public class TestGetGroups extends GetGroupsTestBase {
+  
+  private static final Log LOG = LogFactory.getLog(TestGetGroups.class);
+  
+  private static ResourceManager resourceManager;
+  
+  private static Configuration conf;
+  
+  @BeforeClass
+  public static void setUpResourceManager() throws IOException, InterruptedException {
+    conf = new YarnConfiguration();
+    Store store = StoreFactory.getStore(conf);
+    resourceManager = new ResourceManager(store) {
+      @Override
+      protected void doSecureLogin() throws IOException {
+      };
+    };
+    resourceManager.init(conf);
+    new Thread() {
+      public void run() {
+        resourceManager.start();
+      };
+    }.start();
+    int waitCount = 0;
+    while (resourceManager.getServiceState() == STATE.INITED
+        && waitCount++ < 10) {
+      LOG.info("Waiting for RM to start...");
+      Thread.sleep(1000);
+    }
+    if (resourceManager.getServiceState() != STATE.STARTED) {
+      throw new IOException(
+          "ResourceManager failed to start. Final state is "
+              + resourceManager.getServiceState());
+    }
+    LOG.info("ResourceManager RMAdmin address: " +
+        conf.get(YarnConfiguration.RM_ADMIN_ADDRESS));
+  }
+  
+  @SuppressWarnings("static-access")
+  @Before
+  public void setUpConf() {
+    super.conf = this.conf;
+  }
+  
+  @AfterClass
+  public static void tearDownResourceManager() throws InterruptedException {
+    if (resourceManager != null) {
+      LOG.info("Stopping ResourceManager...");
+      resourceManager.stop();
+    }
+  }
+  
+  @Override
+  protected Tool getTool(PrintStream o) {
+    return new GetGroupsForTesting(conf, o);
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java?rev=1395781&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java Mon Oct  8 21:21:38 2012
@@ -0,0 +1,173 @@
+/**
+* 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.yarn.api.impl.pb.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.api.RMAdminProtocol;
+import org.apache.hadoop.yarn.api.RMAdminProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshServiceAclsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshServiceAclsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshUserToGroupsMappingsRequestProto;
+
+import com.google.protobuf.ServiceException;
+
+
+public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
+
+  private RMAdminProtocolPB proxy;
+  
+  public RMAdminProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, 
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, RMAdminProtocolPB.class, 
+        ProtobufRpcEngine.class);
+    proxy = (RMAdminProtocolPB)RPC.getProxy(
+        RMAdminProtocolPB.class, clientVersion, addr, conf);
+  }
+
+  @Override
+  public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
+      throws YarnRemoteException {
+    RefreshQueuesRequestProto requestProto = 
+      ((RefreshQueuesRequestPBImpl)request).getProto();
+    try {
+      return new RefreshQueuesResponsePBImpl(
+          proxy.refreshQueues(null, requestProto));
+    } catch (ServiceException e) {
+      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+    }
+  }
+
+  @Override
+  public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
+  throws YarnRemoteException {
+    RefreshNodesRequestProto requestProto = 
+      ((RefreshNodesRequestPBImpl)request).getProto();
+    try {
+      return new RefreshNodesResponsePBImpl(
+          proxy.refreshNodes(null, requestProto));
+    } catch (ServiceException e) {
+      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+    }
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
+      RefreshSuperUserGroupsConfigurationRequest request)
+      throws YarnRemoteException {
+    RefreshSuperUserGroupsConfigurationRequestProto requestProto = 
+      ((RefreshSuperUserGroupsConfigurationRequestPBImpl)request).getProto();
+    try {
+      return new RefreshSuperUserGroupsConfigurationResponsePBImpl(
+          proxy.refreshSuperUserGroupsConfiguration(null, requestProto));
+    } catch (ServiceException e) {
+      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+    }
+  }
+
+  @Override
+  public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
+      RefreshUserToGroupsMappingsRequest request) throws YarnRemoteException {
+    RefreshUserToGroupsMappingsRequestProto requestProto = 
+      ((RefreshUserToGroupsMappingsRequestPBImpl)request).getProto();
+    try {
+      return new RefreshUserToGroupsMappingsResponsePBImpl(
+          proxy.refreshUserToGroupsMappings(null, requestProto));
+    } catch (ServiceException e) {
+      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+    }
+  }
+
+  @Override
+  public RefreshAdminAclsResponse refreshAdminAcls(
+      RefreshAdminAclsRequest request) throws YarnRemoteException {
+    RefreshAdminAclsRequestProto requestProto = 
+      ((RefreshAdminAclsRequestPBImpl)request).getProto();
+    try {
+      return new RefreshAdminAclsResponsePBImpl(
+          proxy.refreshAdminAcls(null, requestProto));
+    } catch (ServiceException e) {
+      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+    }
+  }
+
+  @Override
+  public RefreshServiceAclsResponse refreshServiceAcls(
+      RefreshServiceAclsRequest request) throws YarnRemoteException {
+    RefreshServiceAclsRequestProto requestProto = 
+        ((RefreshServiceAclsRequestPBImpl)request).getProto();
+    try {
+      return new RefreshServiceAclsResponsePBImpl(proxy.refreshServiceAcls(
+          null, requestProto));
+    } catch (ServiceException e) {
+      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+    }
+  }
+
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    GetGroupsForUserRequestProto requestProto = 
+        GetGroupsForUserRequestProto.newBuilder().setUser(user).build();
+    try {
+      GetGroupsForUserResponseProto responseProto =
+          proxy.getGroupsForUser(null, requestProto);
+      return (String[]) responseProto.getGroupsList().toArray(
+          new String[responseProto.getGroupsCount()]);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java?rev=1395781&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/RMAdminProtocolPBServiceImpl.java Mon Oct  8 21:21:38 2012
@@ -0,0 +1,160 @@
+/**
+* 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.yarn.api.impl.pb.service;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.api.RMAdminProtocol;
+import org.apache.hadoop.yarn.api.RMAdminProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshServiceAclsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshServiceAclsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.*;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
+
+  private RMAdminProtocol real;
+  
+  public RMAdminProtocolPBServiceImpl(RMAdminProtocol impl) {
+    this.real = impl;
+  }
+  
+  @Override
+  public RefreshQueuesResponseProto refreshQueues(RpcController controller,
+      RefreshQueuesRequestProto proto) throws ServiceException {
+    RefreshQueuesRequestPBImpl request = new RefreshQueuesRequestPBImpl(proto);
+    try {
+      RefreshQueuesResponse response = real.refreshQueues(request);
+      return ((RefreshQueuesResponsePBImpl)response).getProto();
+    } catch (YarnRemoteException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RefreshAdminAclsResponseProto refreshAdminAcls(
+      RpcController controller, RefreshAdminAclsRequestProto proto)
+      throws ServiceException {
+    RefreshAdminAclsRequestPBImpl request = 
+      new RefreshAdminAclsRequestPBImpl(proto);
+    try {
+      RefreshAdminAclsResponse response = real.refreshAdminAcls(request);
+      return ((RefreshAdminAclsResponsePBImpl)response).getProto();
+    } catch (YarnRemoteException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RefreshNodesResponseProto refreshNodes(RpcController controller,
+      RefreshNodesRequestProto proto) throws ServiceException {
+    RefreshNodesRequestPBImpl request = new RefreshNodesRequestPBImpl(proto);
+    try {
+      RefreshNodesResponse response = real.refreshNodes(request);
+      return ((RefreshNodesResponsePBImpl)response).getProto();
+    } catch (YarnRemoteException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponseProto 
+  refreshSuperUserGroupsConfiguration(
+      RpcController controller,
+      RefreshSuperUserGroupsConfigurationRequestProto proto)
+      throws ServiceException {
+    RefreshSuperUserGroupsConfigurationRequestPBImpl request = 
+      new RefreshSuperUserGroupsConfigurationRequestPBImpl(proto);
+    try {
+      RefreshSuperUserGroupsConfigurationResponse response = 
+        real.refreshSuperUserGroupsConfiguration(request);
+      return ((RefreshSuperUserGroupsConfigurationResponsePBImpl)response).getProto();
+    } catch (YarnRemoteException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RefreshUserToGroupsMappingsResponseProto refreshUserToGroupsMappings(
+      RpcController controller, RefreshUserToGroupsMappingsRequestProto proto)
+      throws ServiceException {
+    RefreshUserToGroupsMappingsRequestPBImpl request = 
+      new RefreshUserToGroupsMappingsRequestPBImpl(proto);
+    try {
+      RefreshUserToGroupsMappingsResponse response = 
+        real.refreshUserToGroupsMappings(request);
+      return ((RefreshUserToGroupsMappingsResponsePBImpl)response).getProto();
+    } catch (YarnRemoteException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RefreshServiceAclsResponseProto refreshServiceAcls(
+      RpcController controller, RefreshServiceAclsRequestProto proto)
+      throws ServiceException {
+    RefreshServiceAclsRequestPBImpl request = 
+        new RefreshServiceAclsRequestPBImpl(proto);
+      try {
+        RefreshServiceAclsResponse response = 
+          real.refreshServiceAcls(request);
+        return ((RefreshServiceAclsResponsePBImpl)response).getProto();
+      } catch (YarnRemoteException e) {
+        throw new ServiceException(e);
+      }
+  }
+
+  @Override
+  public GetGroupsForUserResponseProto getGroupsForUser(
+      RpcController controller, GetGroupsForUserRequestProto request)
+      throws ServiceException {
+    String user = request.getUser();
+    try {
+      String[] groups = real.getGroupsForUser(user);
+      GetGroupsForUserResponseProto.Builder responseBuilder =
+          GetGroupsForUserResponseProto.newBuilder();
+      for (String group : groups) {
+        responseBuilder.addGroups(group);
+      }
+      return responseBuilder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java?rev=1395781&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java Mon Oct  8 21:21:38 2012
@@ -0,0 +1,61 @@
+/**
+ * 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.yarn.security.admin;
+
+import java.lang.annotation.Annotation;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.SecurityInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+import org.apache.hadoop.yarn.api.RMAdminProtocolPB;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+public class AdminSecurityInfo extends SecurityInfo {
+
+  @Override
+  public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
+    if (!protocol.equals(RMAdminProtocolPB.class)) {
+      return null;
+    }
+    return new KerberosInfo() {
+
+      @Override
+      public Class<? extends Annotation> annotationType() {
+        return null;
+      }
+
+      @Override
+      public String serverPrincipal() {
+        return YarnConfiguration.RM_PRINCIPAL;
+      }
+
+      @Override
+      public String clientPrincipal() {
+        return null;
+      }
+    };
+  }
+
+  @Override
+  public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
+    return null;
+  }
+
+}

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo?rev=1395781&r1=1395780&r2=1395781&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo Mon Oct  8 21:21:38 2012
@@ -1,4 +1,4 @@
 org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo
 org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo
 org.apache.hadoop.yarn.security.SchedulerSecurityInfo
-
+org.apache.hadoop.yarn.security.admin.AdminSecurityInfo

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml?rev=1395781&r1=1395780&r2=1395781&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml Mon Oct  8 21:21:38 2012
@@ -58,66 +58,6 @@
           </execution>
         </executions>
       </plugin>
-
-      <plugin>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>create-protobuf-generated-sources-directory</id>
-            <phase>initialize</phase>
-            <configuration>
-              <target>
-                <mkdir dir="target/generated-sources/proto" />
-              </target>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>generate-sources</id>
-            <phase>generate-sources</phase>
-            <configuration>
-              <executable>protoc</executable>
-              <arguments>
-                <argument>-Isrc/main/proto/</argument>
-                <argument>--java_out=target/generated-sources/proto</argument>
-                <argument>src/main/proto/yarn_server_resourcemanager_service_protos.proto</argument>
-                <argument>src/main/proto/RMAdminProtocol.proto</argument>
-              </arguments>
-            </configuration>
-            <goals>
-              <goal>exec</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>build-helper-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>add-source</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>add-source</goal>
-            </goals>
-            <configuration>
-              <sources>
-                <source>target/generated-sources/proto</source>
-              </sources>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
     </plugins>
   </build>
 

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java?rev=1395781&r1=1395780&r2=1395781&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java Mon Oct  8 21:21:38 2012
@@ -32,6 +32,19 @@ import org.apache.hadoop.security.UserGr
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.yarn.api.RMAdminProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -39,19 +52,6 @@ import org.apache.hadoop.yarn.factory.pr
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshServiceAclsResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.service.AbstractService;

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java?rev=1395781&r1=1395780&r2=1395781&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/authorize/RMPolicyProvider.java Mon Oct  8 21:21:38 2012
@@ -24,9 +24,9 @@ import org.apache.hadoop.security.author
 import org.apache.hadoop.yarn.api.AMRMProtocolPB;
 import org.apache.hadoop.yarn.api.ClientRMProtocolPB;
 import org.apache.hadoop.yarn.api.ContainerManagerPB;
+import org.apache.hadoop.yarn.api.RMAdminProtocolPB;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.ResourceTrackerPB;
-import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocolPB;
 
 /**
  * {@link PolicyProvider} for YARN ResourceManager protocols.