You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by to...@apache.org on 2011/05/13 02:01:12 UTC

svn commit: r1102515 - in /hadoop/mapreduce/trunk: ./ bin/ src/java/org/apache/hadoop/mapred/ src/java/org/apache/hadoop/mapred/tools/ src/test/mapred/org/apache/hadoop/mapred/tools/

Author: todd
Date: Fri May 13 00:01:11 2011
New Revision: 1102515

URL: http://svn.apache.org/viewvc?rev=1102515&view=rev
Log:
MAPREDUCE-2473. Add "mapred groups" command to query the server-side groups resolved for a user. Contributed by Aaron T. Myers.

Added:
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/tools/GetGroups.java
    hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/tools/
    hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/tools/TestGetGroups.java
Modified:
    hadoop/mapreduce/trunk/CHANGES.txt
    hadoop/mapreduce/trunk/bin/mapred
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
    hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java

Modified: hadoop/mapreduce/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/CHANGES.txt?rev=1102515&r1=1102514&r2=1102515&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/CHANGES.txt (original)
+++ hadoop/mapreduce/trunk/CHANGES.txt Fri May 13 00:01:11 2011
@@ -6,6 +6,9 @@ Trunk (unreleased changes)
 
   NEW FEATURES
 
+    MAPREDUCE-2473. Add "mapred groups" command to query the server-side groups
+    resolved for a user. (Aaron T. Myers via todd)
+
   IMPROVEMENTS
 
     MAPREDUCE-2153. Bring in more job configuration properties in to the trace 

Modified: hadoop/mapreduce/trunk/bin/mapred
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/bin/mapred?rev=1102515&r1=1102514&r2=1102515&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/bin/mapred (original)
+++ hadoop/mapreduce/trunk/bin/mapred Fri May 13 00:01:11 2011
@@ -29,6 +29,7 @@ function print_usage(){
   echo "  pipes                run a Pipes job"
   echo "  job                  manipulate MapReduce jobs"
   echo "  queue                get information regarding JobQueues"
+  echo "  groups               get the groups which users belong to"
   echo ""
   echo "Most commands print help when invoked w/o parameters."
 }
@@ -60,6 +61,9 @@ elif [ "$COMMAND" = "pipes" ] ; then
 elif [ "$COMMAND" = "sampler" ] ; then
   CLASS=org.apache.hadoop.mapred.lib.InputSampler
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
+elif [ "$COMMAND" = "groups" ] ; then
+  CLASS=org.apache.hadoop.mapred.tools.GetGroups
+  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
 else
   echo $COMMAND - invalid command
   print_usage

Modified: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=1102515&r1=1102514&r2=1102515&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java Fri May 13 00:01:11 2011
@@ -114,6 +114,7 @@ import org.apache.hadoop.security.author
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -128,7 +129,8 @@ import org.apache.hadoop.util.VersionInf
 @InterfaceStability.Unstable
 public class JobTracker implements MRConstants, InterTrackerProtocol,
     ClientProtocol, TaskTrackerManager, RefreshUserMappingsProtocol,
-    RefreshAuthorizationPolicyProtocol, AdminOperationsProtocol, JTConfig {
+    RefreshAuthorizationPolicyProtocol, AdminOperationsProtocol, 
+    GetUserMappingsProtocol, JTConfig {
 
   static{
     ConfigUtil.loadResources();
@@ -321,6 +323,8 @@ public class JobTracker implements MRCon
       return AdminOperationsProtocol.versionID;
     } else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
       return RefreshUserMappingsProtocol.versionID;
+    } else if (protocol.equals(GetUserMappingsProtocol.class.getName())){
+      return GetUserMappingsProtocol.versionID;
     } else {
       throw new IOException("Unknown protocol to job tracker: " + protocol);
     }
@@ -4491,6 +4495,14 @@ public class JobTracker implements MRCon
 
     ProxyUsers.refreshSuperUserGroupsConfiguration();
   }
+  
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Getting groups for user " + user);
+    }
+    return UserGroupInformation.createRemoteUser(user).getGroupNames();
+  }
 
   @Override
   public void refreshUserToGroupsMappings() throws IOException {

Modified: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java?rev=1102515&r1=1102514&r2=1102515&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java (original)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java Fri May 13 00:01:11 2011
@@ -24,6 +24,7 @@ import org.apache.hadoop.security.Refres
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.Service;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
 
 /**
  * {@link PolicyProvider} for Map-Reduce protocols.
@@ -45,6 +46,8 @@ public class MapReducePolicyProvider ext
                   RefreshUserMappingsProtocol.class),
       new Service("security.admin.operations.protocol.acl", 
                   AdminOperationsProtocol.class),
+      new Service("security.get.user.mappings.protocol.acl",
+                  GetUserMappingsProtocol.class)
   };
   
   @Override

Added: hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/tools/GetGroups.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/tools/GetGroups.java?rev=1102515&view=auto
==============================================================================
--- hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/tools/GetGroups.java (added)
+++ hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapred/tools/GetGroups.java Fri May 13 00:01:11 2011
@@ -0,0 +1,58 @@
+/**
+ * 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.mapred.tools;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.tools.GetGroupsBase;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * MR implementation of a tool for getting the groups which a given user
+ * belongs to.
+ */
+public class GetGroups extends GetGroupsBase {
+
+  static {
+    Configuration.addDefaultResource("mapred-default.xml");
+    Configuration.addDefaultResource("mapred-site.xml");
+  }
+  
+  GetGroups(Configuration conf) {
+    super(conf);
+  }
+  
+  GetGroups(Configuration conf, PrintStream out) {
+    super(conf, out);
+  }
+
+  @Override
+  protected InetSocketAddress getProtocolAddress(Configuration conf)
+      throws IOException {
+    return JobTracker.getAddress(conf);
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int res = ToolRunner.run(new GetGroups(new Configuration()), argv);
+    System.exit(res);
+  }
+}
\ No newline at end of file

Added: hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/tools/TestGetGroups.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/tools/TestGetGroups.java?rev=1102515&view=auto
==============================================================================
--- hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/tools/TestGetGroups.java (added)
+++ hadoop/mapreduce/trunk/src/test/mapred/org/apache/hadoop/mapred/tools/TestGetGroups.java Fri May 13 00:01:11 2011
@@ -0,0 +1,53 @@
+/**
+ * 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.mapred.tools;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.tools.GetGroups;
+import org.apache.hadoop.tools.GetGroupsTestBase;
+import org.apache.hadoop.util.Tool;
+import org.junit.After;
+import org.junit.Before;
+
+/**
+ * Tests for the MR implementation of {@link GetGroups}
+ */
+public class TestGetGroups extends GetGroupsTestBase {
+  
+  private MiniMRCluster cluster;
+
+  @Before
+  public void setUpJobTracker() throws IOException, InterruptedException {
+    cluster = new MiniMRCluster(0, "file:///", 1);
+    conf = cluster.createJobConf();
+  }
+  
+  @After
+  public void tearDownJobTracker() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Override
+  protected Tool getTool(PrintStream o) {
+    return new GetGroups(conf, o);
+  }
+
+}