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 bo...@apache.org on 2010/12/24 01:45:04 UTC

svn commit: r1052420 [3/3] - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/security/ src/java/org/apache/hadoop/security/authorize/ src/java/org/apache/hadoop/util/ src/native/ src/native/src/org/apache/hadoop/security/ src/test/core/org/apach...

Added: hadoop/common/trunk/src/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c?rev=1052420&view=auto
==============================================================================
--- hadoop/common/trunk/src/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c (added)
+++ hadoop/common/trunk/src/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c Fri Dec 24 00:45:03 2010
@@ -0,0 +1,138 @@
+/**
+ * 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.
+ */
+#include <jni.h>
+#include <sys/types.h>
+#include <sys/ipc.h>
+#include <sys/shm.h>
+#include <sys/mman.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <unistd.h>
+#include <stdlib.h>
+#include <errno.h>
+#include <grp.h>
+#include <stdio.h>
+#include <pwd.h>
+#include <string.h>
+
+#include <netdb.h>
+
+#include "org_apache_hadoop_security_JniBasedUnixGroupsNetgroupMapping.h"
+#include "org_apache_hadoop.h"
+
+struct listElement {
+   char * string;
+   struct listElement * next;
+};
+
+typedef struct listElement UserList;
+
+JNIEXPORT jobjectArray JNICALL 
+Java_org_apache_hadoop_security_JniBasedUnixGroupsNetgroupMapping_getUsersForNetgroupJNI
+(JNIEnv *env, jobject jobj, jstring jgroup) {
+
+  // pointers to free at the end
+  const char *cgroup  = NULL;
+  jobjectArray jusers = NULL;
+
+  // do we need to end the group lookup?
+  int setnetgrentCalledFlag = 0;
+
+  // if not NULL then THROW exception
+  char *errorMessage = NULL;
+
+  cgroup = (*env)->GetStringUTFChars(env, jgroup, NULL);
+  if (cgroup == NULL) {
+    goto END;
+  }
+
+  //--------------------------------------------------
+  // get users
+  // see man pages for setnetgrent, getnetgrent and endnetgrent
+
+  UserList *userListHead = NULL;
+  int       userListSize = 0;
+
+  // set the name of the group for subsequent calls to getnetgrent
+  // note that we want to end group lokup regardless whether setnetgrent
+  // was successfull or not (as long as it was called we need to call
+  // endnetgrent)
+  setnetgrentCalledFlag = 1;
+  if(setnetgrent(cgroup) == 1) {
+    UserList *current = NULL;
+    // three pointers are for host, user, domain, we only care
+    // about user now
+    char *p[3];
+    while(getnetgrent(p, p + 1, p + 2)) {
+      if(p[1]) {
+        current = (UserList *)malloc(sizeof(UserList));
+        current->string = malloc(strlen(p[1]) + 1);
+        strcpy(current->string, p[1]);
+        current->next = userListHead;
+        userListHead = current;
+        userListSize++;
+      }
+    }
+  }
+
+  //--------------------------------------------------
+  // build return data (java array)
+
+  jusers = (jobjectArray)(*env)->NewObjectArray(env,
+    userListSize, 
+    (*env)->FindClass(env, "java/lang/String"),
+    NULL);
+  if (jusers == NULL) {
+    errorMessage = "java/lang/OutOfMemoryError";
+    goto END;
+  }
+
+  UserList * current = NULL;
+
+  // note that the loop iterates over list but also over array (i)
+  int i = 0;
+  for(current = userListHead; current != NULL; current = current->next) {
+    jstring juser = (*env)->NewStringUTF(env, current->string);
+    if (juser == NULL) {
+      errorMessage = "java/lang/OutOfMemoryError";
+      goto END;
+    }
+    (*env)->SetObjectArrayElement(env, jusers, i++, juser);
+  }
+
+
+END:
+
+  // cleanup
+  if(cgroup) { (*env)->ReleaseStringUTFChars(env, jgroup, cgroup); }
+  if(setnetgrentCalledFlag) { endnetgrent(); }
+  while(userListHead) {
+    UserList *current = userListHead;
+    userListHead = userListHead->next;
+    if(current->string) { free(current->string); }
+    free(current);
+  }
+
+  // return results or THROW
+  if(errorMessage) {
+    THROW(env, errorMessage, NULL);
+    return NULL;
+  } else {
+    return jusers;
+  }
+}

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java?rev=1052420&r1=1052419&r2=1052420&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/security/authorize/TestAccessControlList.java Fri Dec 24 00:45:03 2010
@@ -19,15 +19,152 @@ package org.apache.hadoop.security.autho
 
 import java.util.Iterator;
 import java.util.Set;
+import java.util.List;
 
+import org.junit.Test;
+import org.junit.Before;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class TestAccessControlList {
 
-import junit.framework.TestCase;
+  private static final Log LOG =
+    LogFactory.getLog(TestAccessControlList.class);
 
-public class TestAccessControlList extends TestCase {
-  
+  /**
+   * Test the netgroups (groups in ACL rules that start with @)
+   *
+   * This is a  manual test because it requires:
+   *   - host setup
+   *   - native code compiled
+   *   - specify the group mapping class
+   *
+   * Host setup:
+   *
+   * /etc/nsswitch.conf should have a line like this:
+   * netgroup: files
+   *
+   * /etc/netgroup should be (the whole file):
+   * lasVegas (,elvis,)
+   * memphis (,elvis,) (,jerryLeeLewis,)
+   *
+   * To run this test:
+   *
+   * export JAVA_HOME='path/to/java'
+   * ant \
+   *   -Dtestcase=TestAccessControlList \
+   *   -Dtest.output=yes \
+   *   -DTestAccessControlListGroupMapping=$className \
+   *   compile-native test
+   *
+   * where $className is one of the classes that provide group
+   * mapping services, i.e. classes that implement
+   * GroupMappingServiceProvider interface, at this time:
+   *   - org.apache.hadoop.security.JniBasedUnixGroupsNetgroupMapping
+   *   - org.apache.hadoop.security.ShellBasedUnixGroupsNetgroupMapping
+   *
+   */
+  @Test
+  public void testNetgroups() throws Exception {
+
+    if(!NativeCodeLoader.isNativeCodeLoaded()) {
+      LOG.info("Not testing netgroups, " +
+        "this test only runs when native code is compiled");
+      return;
+    }
+
+    String groupMappingClassName =
+      System.getProperty("TestAccessControlListGroupMapping");
+
+    if(groupMappingClassName == null) {
+      LOG.info("Not testing netgroups, no group mapping class specified, " +
+        "use -DTestAccessControlListGroupMapping=$className to specify " +
+        "group mapping class (must implement GroupMappingServiceProvider " +
+        "interface and support netgroups)");
+      return;
+    }
+
+    LOG.info("Testing netgroups using: " + groupMappingClassName);
+
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_GROUP_MAPPING,
+      groupMappingClassName);
+
+    Groups groups = Groups.getUserToGroupsMappingService(conf);
+
+    AccessControlList acl;
+
+    // create these ACLs to populate groups cache
+    acl = new AccessControlList("ja my"); // plain
+    acl = new AccessControlList("sinatra ratpack,@lasVegas"); // netgroup
+    acl = new AccessControlList(" somegroup,@someNetgroup"); // no user
+
+    // this ACL will be used for testing ACLs
+    acl = new AccessControlList("carlPerkins ratpack,@lasVegas");
+    acl.addGroup("@memphis");
+
+    // validate the netgroups before and after rehresh to make
+    // sure refresh works correctly
+    validateNetgroups(groups, acl);
+    groups.refresh();
+    validateNetgroups(groups, acl);
+
+  }
+
+  /**
+   * Validate the netgroups, both group membership and ACL
+   * functionality
+   *
+   * Note: assumes a specific acl setup done by testNetgroups
+   *
+   * @param groups group to user mapping service
+   * @param acl ACL set up in a specific way, see testNetgroups
+   */
+  private void validateNetgroups(Groups groups,
+    AccessControlList acl) throws Exception {
+
+    // check that the netgroups are working
+    List<String> elvisGroups = groups.getGroups("elvis");
+    assertTrue(elvisGroups.contains("@lasVegas"));
+    assertTrue(elvisGroups.contains("@memphis"));
+    List<String> jerryLeeLewisGroups = groups.getGroups("jerryLeeLewis");
+    assertTrue(jerryLeeLewisGroups.contains("@memphis"));
+
+    // allowed becuase his netgroup is in ACL
+    UserGroupInformation elvis = 
+      UserGroupInformation.createRemoteUser("elvis");
+    assertUserAllowed(elvis, acl);
+
+    // allowed because he's in ACL
+    UserGroupInformation carlPerkins = 
+      UserGroupInformation.createRemoteUser("carlPerkins");
+    assertUserAllowed(carlPerkins, acl);
+
+    // not allowed because he's not in ACL and has no netgroups
+    UserGroupInformation littleRichard = 
+      UserGroupInformation.createRemoteUser("littleRichard");
+    assertUserNotAllowed(littleRichard, acl);
+  }
+
+  @Test
   public void testWildCardAccessControlList() throws Exception {
     AccessControlList acl;
     
@@ -46,6 +183,7 @@ public class TestAccessControlList exten
 
   // Check if AccessControlList.toString() works as expected.
   // Also validate if getAclString() for various cases.
+  @Test
   public void testAclString() {
     AccessControlList acl;
 
@@ -83,6 +221,7 @@ public class TestAccessControlList exten
         new AccessControlList(acl.getAclString()).toString()));
   }
 
+  @Test
   public void testAccessControlList() throws Exception {
     AccessControlList acl;
     Set<String> users;
@@ -134,6 +273,7 @@ public class TestAccessControlList exten
   /**
    * Test addUser/Group and removeUser/Group api.
    */
+  @Test
   public void testAddRemoveAPI() {
     AccessControlList acl;
     Set<String> users;
@@ -196,6 +336,7 @@ public class TestAccessControlList exten
   /**
    * Tests adding/removing wild card as the user/group.
    */
+  @Test
   public void testAddRemoveWildCard() {
     AccessControlList acl = new AccessControlList("drwho tardis");
     
@@ -237,6 +378,7 @@ public class TestAccessControlList exten
   /**
    * Tests adding user/group to an wild card acl.
    */
+  @Test
   public void testAddRemoveToWildCardACL() {
     AccessControlList acl = new AccessControlList(" * ");
     assertTrue(acl.isAllAllowed());
@@ -266,6 +408,7 @@ public class TestAccessControlList exten
   /**
    * Verify the method isUserAllowed()
    */
+  @Test
   public void testIsUserAllowed() {
     AccessControlList acl;