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 at...@apache.org on 2012/03/20 02:00:15 UTC

svn commit: r1302740 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/security/ src/main/resources/ src/test/java/org/apache/hadoop/security/

Author: atm
Date: Tue Mar 20 01:00:14 2012
New Revision: 1302740

URL: http://svn.apache.org/viewvc?rev=1302740&view=rev
Log:
HADOOP-8121. Active Directory Group Mapping Service. Contributed by Jonathan Natkins.

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.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=1302740&r1=1302739&r2=1302740&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Tue Mar 20 01:00:14 2012
@@ -130,6 +130,9 @@ Release 0.23.3 - UNRELEASED 
     Bikas Saha, Suresh Srinivas, Jitendra Nath Pandey, Hari Mankude, Brandon Li,
     Sanjay Radia, Mingjie Lai, and Gregory Chanan
 
+    HADOOP-8121. Active Directory Group Mapping Service. (Jonathan Natkins via
+    atm)
+
   IMPROVEMENTS
 
     HADOOP-7524. Change RPC to allow multiple protocols including multuple

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java?rev=1302740&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java Tue Mar 20 01:00:14 2012
@@ -0,0 +1,321 @@
+/**
+ * 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;
+
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Hashtable;
+import java.util.List;
+
+import javax.naming.Context;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.InitialDirContext;
+import javax.naming.directory.SearchControls;
+import javax.naming.directory.SearchResult;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * An implementation of {@link GroupMappingServiceProvider} which
+ * connects directly to an LDAP server for determining group membership.
+ * 
+ * This provider should be used only if it is necessary to map users to
+ * groups that reside exclusively in an Active Directory or LDAP installation.
+ * The common case for a Hadoop installation will be that LDAP users and groups
+ * materialized on the Unix servers, and for an installation like that,
+ * ShellBasedUnixGroupsMapping is preferred. However, in cases where
+ * those users and groups aren't materialized in Unix, but need to be used for
+ * access control, this class may be used to communicate directly with the LDAP
+ * server.
+ * 
+ * It is important to note that resolving group mappings will incur network
+ * traffic, and may cause degraded performance, although user-group mappings
+ * will be cached via the infrastructure provided by {@link Groups}.
+ * 
+ * This implementation does not support configurable search limits. If a filter
+ * is used for searching users or groups which returns more results than are
+ * allowed by the server, an exception will be thrown.
+ * 
+ * The implementation also does not attempt to resolve group hierarchies. In
+ * order to be considered a member of a group, the user must be an explicit
+ * member in LDAP.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class LdapGroupsMapping
+    implements GroupMappingServiceProvider, Configurable {
+  
+  public static final String LDAP_CONFIG_PREFIX = "hadoop.security.group.mapping.ldap";
+
+  /*
+   * URL of the LDAP server
+   */
+  public static final String LDAP_URL_KEY = LDAP_CONFIG_PREFIX + ".url";
+  public static final String LDAP_URL_DEFAULT = "";
+
+  /*
+   * Should SSL be used to connect to the server
+   */
+  public static final String LDAP_USE_SSL_KEY = LDAP_CONFIG_PREFIX + ".ssl";
+  public static final Boolean LDAP_USE_SSL_DEFAULT = false;
+
+  /*
+   * File path to the location of the SSL keystore to use
+   */
+  public static final String LDAP_KEYSTORE_KEY = LDAP_CONFIG_PREFIX + ".ssl.keystore";
+  public static final String LDAP_KEYSTORE_DEFAULT = "";
+
+  /*
+   * Password for the keystore
+   */
+  public static final String LDAP_KEYSTORE_PASSWORD_KEY = LDAP_CONFIG_PREFIX + ".ssl.keystore.password";
+  public static final String LDAP_KEYSTORE_PASSWORD_DEFAULT = "";
+  
+  public static final String LDAP_KEYSTORE_PASSWORD_FILE_KEY = LDAP_KEYSTORE_PASSWORD_KEY + ".file";
+  public static final String LDAP_KEYSTORE_PASSWORD_FILE_DEFAULT = "";
+
+  /*
+   * User to bind to the LDAP server with
+   */
+  public static final String BIND_USER_KEY = LDAP_CONFIG_PREFIX + ".bind.user";
+  public static final String BIND_USER_DEFAULT = "";
+
+  /*
+   * Password for the bind user
+   */
+  public static final String BIND_PASSWORD_KEY = LDAP_CONFIG_PREFIX + ".bind.password";
+  public static final String BIND_PASSWORD_DEFAULT = "";
+  
+  public static final String BIND_PASSWORD_FILE_KEY = BIND_PASSWORD_KEY + ".file";
+  public static final String BIND_PASSWORD_FILE_DEFAULT = "";
+
+  /*
+   * Base distinguished name to use for searches
+   */
+  public static final String BASE_DN_KEY = LDAP_CONFIG_PREFIX + ".base";
+  public static final String BASE_DN_DEFAULT = "";
+
+  /*
+   * Any additional filters to apply when searching for users
+   */
+  public static final String USER_SEARCH_FILTER_KEY = LDAP_CONFIG_PREFIX + ".search.filter.user";
+  public static final String USER_SEARCH_FILTER_DEFAULT = "(&(objectClass=user)(sAMAccountName={0}))";
+
+  /*
+   * Any additional filters to apply when finding relevant groups
+   */
+  public static final String GROUP_SEARCH_FILTER_KEY = LDAP_CONFIG_PREFIX + ".search.filter.group";
+  public static final String GROUP_SEARCH_FILTER_DEFAULT = "(objectClass=group)";
+
+  /*
+   * LDAP attribute to use for determining group membership
+   */
+  public static final String GROUP_MEMBERSHIP_ATTR_KEY = LDAP_CONFIG_PREFIX + ".search.attr.member";
+  public static final String GROUP_MEMBERSHIP_ATTR_DEFAULT = "member";
+
+  /*
+   * LDAP attribute to use for identifying a group's name
+   */
+  public static final String GROUP_NAME_ATTR_KEY = LDAP_CONFIG_PREFIX + ".search.attr.group.name";
+  public static final String GROUP_NAME_ATTR_DEFAULT = "cn";
+  
+  private static final Log LOG = LogFactory.getLog(LdapGroupsMapping.class);
+
+  private static final SearchControls SEARCH_CONTROLS = new SearchControls();
+  static {
+    SEARCH_CONTROLS.setSearchScope(SearchControls.SUBTREE_SCOPE);
+  }
+
+  private DirContext ctx;
+  private Configuration conf;
+  
+  private String ldapUrl;
+  private boolean useSsl;
+  private String keystore;
+  private String keystorePass;
+  private String bindUser;
+  private String bindPassword;
+  private String baseDN;
+  private String groupSearchFilter;
+  private String userSearchFilter;
+  private String groupMemberAttr;
+  private String groupNameAttr;
+
+  /**
+   * Returns list of groups for a user.
+   * 
+   * The LdapCtx which underlies the DirContext object is not thread-safe, so
+   * we need to block around this whole method. The caching infrastructure will
+   * ensure that performance stays in an acceptable range.
+   *
+   * @param user get groups for this user
+   * @return list of groups for a given user
+   */
+  @Override
+  public synchronized List<String> getGroups(String user) throws IOException {
+    List<String> groups = new ArrayList<String>();
+
+    try {
+      DirContext ctx = getDirContext();
+
+      // Search for the user. We'll only ever need to look at the first result
+      NamingEnumeration<SearchResult> results = ctx.search(baseDN,
+                                                           userSearchFilter,
+                                                           new Object[]{user},
+                                                           SEARCH_CONTROLS);
+      if (results.hasMoreElements()) {
+        SearchResult result = results.nextElement();
+        String userDn = result.getNameInNamespace();
+
+        NamingEnumeration<SearchResult> groupResults =
+          ctx.search(baseDN,
+                     "(&" + groupSearchFilter + "(" + groupMemberAttr + "={0}))",
+                     new Object[]{userDn},
+                     SEARCH_CONTROLS);
+        while (groupResults.hasMoreElements()) {
+          SearchResult groupResult = groupResults.nextElement();
+          Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
+          groups.add(groupName.get().toString());
+        }
+      }
+    } catch (NamingException e) {
+      LOG.warn("Exception trying to get groups for user " + user, e);
+      return new ArrayList<String>();
+    }
+
+    return groups;
+  }
+
+  @SuppressWarnings("deprecation")
+  DirContext getDirContext() throws NamingException {
+    if (ctx == null) {
+      // Set up the initial environment for LDAP connectivity
+      Hashtable<String, String> env = new Hashtable<String, String>();
+      env.put(Context.INITIAL_CONTEXT_FACTORY,
+          com.sun.jndi.ldap.LdapCtxFactory.class.getName());
+      env.put(Context.PROVIDER_URL, ldapUrl);
+      env.put(Context.SECURITY_AUTHENTICATION, "simple");
+
+      // Set up SSL security, if necessary
+      if (useSsl) {
+        env.put(Context.SECURITY_PROTOCOL, "ssl");
+        System.setProperty("javax.net.ssl.keyStore", keystore);
+        System.setProperty("javax.net.ssl.keyStorePassword", keystorePass);
+      }
+
+      env.put(Context.SECURITY_PRINCIPAL, bindUser);
+      env.put(Context.SECURITY_CREDENTIALS, bindPassword);
+
+      ctx = new InitialDirContext(env);
+    }
+
+    return ctx;
+  }
+
+  /**
+   * Caches groups, no need to do that for this provider
+   */
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    // does nothing in this provider of user to groups mapping
+  }
+
+  /** 
+   * Adds groups to cache, no need to do that for this provider
+   *
+   * @param groups unused
+   */
+  @Override
+  public void cacheGroupsAdd(List<String> groups) throws IOException {
+    // does nothing in this provider of user to groups mapping
+  }
+
+  @Override
+  public synchronized Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public synchronized void setConf(Configuration conf) {
+    ldapUrl = conf.get(LDAP_URL_KEY, LDAP_URL_DEFAULT);
+    if (ldapUrl == null || ldapUrl.isEmpty()) {
+      throw new RuntimeException("LDAP URL is not configured");
+    }
+    
+    useSsl = conf.getBoolean(LDAP_USE_SSL_KEY, LDAP_USE_SSL_DEFAULT);
+    keystore = conf.get(LDAP_KEYSTORE_KEY, LDAP_KEYSTORE_DEFAULT);
+    
+    keystorePass =
+        conf.get(LDAP_KEYSTORE_PASSWORD_KEY, LDAP_KEYSTORE_PASSWORD_DEFAULT);
+    if (keystorePass.isEmpty()) {
+      keystorePass = extractPassword(
+        conf.get(LDAP_KEYSTORE_PASSWORD_KEY, LDAP_KEYSTORE_PASSWORD_DEFAULT));
+    }
+    
+    bindUser = conf.get(BIND_USER_KEY, BIND_USER_DEFAULT);
+    bindPassword = conf.get(BIND_PASSWORD_KEY, BIND_PASSWORD_DEFAULT);
+    if (bindPassword.isEmpty()) {
+      bindPassword = extractPassword(
+          conf.get(BIND_PASSWORD_FILE_KEY, BIND_PASSWORD_FILE_DEFAULT));
+    }
+    
+    baseDN = conf.get(BASE_DN_KEY, BASE_DN_DEFAULT);
+    groupSearchFilter =
+        conf.get(GROUP_SEARCH_FILTER_KEY, GROUP_SEARCH_FILTER_DEFAULT);
+    userSearchFilter =
+        conf.get(USER_SEARCH_FILTER_KEY, USER_SEARCH_FILTER_DEFAULT);
+    groupMemberAttr =
+        conf.get(GROUP_MEMBERSHIP_ATTR_KEY, GROUP_MEMBERSHIP_ATTR_DEFAULT);
+    groupNameAttr =
+        conf.get(GROUP_NAME_ATTR_KEY, GROUP_NAME_ATTR_DEFAULT);
+
+    this.conf = conf;
+  }
+  
+  String extractPassword(String pwFile) {
+    if (pwFile.isEmpty()) {
+      // If there is no password file defined, we'll assume that we should do
+      // an anonymous bind
+      return "";
+    }
+    
+    try {
+      StringBuilder password = new StringBuilder();
+      Reader reader = new FileReader(pwFile);
+      int c = reader.read();
+      while (c > -1) {
+        password.append((char)c);
+        c = reader.read();
+      }
+      reader.close();
+      return password.toString();
+    } catch (IOException ex) {
+      throw new RuntimeException("Could not read password file: " + pwFile);
+    }
+  }
+}

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml?rev=1302740&r1=1302739&r2=1302740&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml Tue Mar 20 01:00:14 2012
@@ -89,6 +89,113 @@
 </property>
 
 <property>
+  <name>hadoop.security.group.mapping.ldap.url</name>
+  <value></value>
+  <description>
+    The URL of the LDAP server to use for resolving user groups when using
+    the LdapGroupsMapping user to group mapping.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.ssl</name>
+  <value>false</value>
+  <description>
+    Whether or not to use SSL when connecting to the LDAP server.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.ssl.keystore</name>
+  <value></value>
+  <description>
+    File path to the SSL keystore that contains the SSL certificate required
+    by the LDAP server.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.ssl.keystore.password.file</name>
+  <value></value>
+  <description>
+    The path to a file containing the password of the LDAP SSL keystore.
+
+    IMPORTANT: This file should be readable only by the Unix user running
+    the daemons.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.bind.user</name>
+  <value></value>
+  <description>
+    The distinguished name of the user to bind as when connecting to the LDAP
+    server. This may be left blank if the LDAP server supports anonymous binds.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.bind.password.file</name>
+  <value></value>
+  <description>
+    The path to a file containing the password of the bind user.
+
+    IMPORTANT: This file should be readable only by the Unix user running
+    the daemons.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.base</name>
+  <value></value>
+  <description>
+    The search base for the LDAP connection. This is a distinguished name,
+    and will typically be the root of the LDAP directory.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.search.filter.user</name>
+  <value>(&amp;(objectClass=user)(sAMAccountName={0})</value>
+  <description>
+    An additional filter to use when searching for LDAP users. The default will
+    usually be appropriate for Active Directory installations. If connecting to
+    an LDAP server with a non-AD schema, this should be replaced with
+    (&amp;(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to
+    denote where the username fits into the filter.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.search.filter.group</name>
+  <value>(objectClass=group)</value>
+  <description>
+    An additional filter to use when searching for LDAP groups. This should be
+    changed when resolving groups against a non-Active Directory installation.
+    posixGroups are currently not a supported group class.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.search.attr.member</name>
+  <value>member</value>
+  <description>
+    The attribute of the group object that identifies the users that are
+    members of the group. The default will usually be appropriate for
+    any LDAP installation.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.search.attr.group.name</name>
+  <value>cn</value>
+  <description>
+    The attribute of the group object that identifies the group name. The
+    default will usually be appropriate for all LDAP systems.
+  </description>
+</property>
+
+<property>
   <name>hadoop.security.service.user.name.key</name>
   <value></value>
   <description>

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java?rev=1302740&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java Tue Mar 20 01:00:14 2012
@@ -0,0 +1,128 @@
+/**
+ * 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;
+
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.List;
+
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.Attributes;
+import javax.naming.directory.BasicAttribute;
+import javax.naming.directory.BasicAttributes;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.SearchControls;
+import javax.naming.directory.SearchResult;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+@SuppressWarnings("unchecked")
+public class TestLdapGroupsMapping {
+  private DirContext mockContext;
+  
+  private LdapGroupsMapping mappingSpy = spy(new LdapGroupsMapping());
+  
+  @Before
+  public void setupMocks() throws NamingException {
+    mockContext = mock(DirContext.class);
+    doReturn(mockContext).when(mappingSpy).getDirContext();
+    
+    NamingEnumeration mockUserNamingEnum = mock(NamingEnumeration.class);
+    NamingEnumeration mockGroupNamingEnum = mock(NamingEnumeration.class);
+    
+    // The search functionality of the mock context is reused, so we will
+    // return the user NamingEnumeration first, and then the group
+    when(mockContext.search(anyString(), anyString(), any(Object[].class),
+        any(SearchControls.class)))
+        .thenReturn(mockUserNamingEnum, mockGroupNamingEnum);
+    
+    SearchResult mockUserResult = mock(SearchResult.class);
+    // We only ever call hasMoreElements once for the user NamingEnum, so 
+    // we can just have one return value
+    when(mockUserNamingEnum.hasMoreElements()).thenReturn(true);
+    when(mockUserNamingEnum.nextElement()).thenReturn(mockUserResult);
+    when(mockUserResult.getNameInNamespace()).thenReturn("CN=some_user,DC=test,DC=com");
+    
+    SearchResult mockGroupResult = mock(SearchResult.class);
+    // We're going to have to define the loop here. We want two iterations,
+    // to get both the groups
+    when(mockGroupNamingEnum.hasMoreElements()).thenReturn(true, true, false);
+    when(mockGroupNamingEnum.nextElement()).thenReturn(mockGroupResult);
+    
+    // Define the attribute for the name of the first group
+    Attribute group1Attr = new BasicAttribute("cn");
+    group1Attr.add("group1");
+    Attributes group1Attrs = new BasicAttributes();
+    group1Attrs.put(group1Attr);
+    
+    // Define the attribute for the name of the second group
+    Attribute group2Attr = new BasicAttribute("cn");
+    group2Attr.add("group2");
+    Attributes group2Attrs = new BasicAttributes();
+    group2Attrs.put(group2Attr);
+    
+    // This search result gets reused, so return group1, then group2
+    when(mockGroupResult.getAttributes()).thenReturn(group1Attrs, group2Attrs);
+    
+  }
+  
+  @Test
+  public void testGetGroups() throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    
+    mappingSpy.setConf(conf);
+    // Username is arbitrary, since the spy is mocked to respond the same,
+    // regardless of input
+    List<String> groups = mappingSpy.getGroups("some_user");
+    
+    Assert.assertEquals(Arrays.asList("group1", "group2"), groups);
+    
+    // We should have searched for a user, and then two groups
+    verify(mockContext, times(2)).search(anyString(),
+                                         anyString(),
+                                         any(Object[].class),
+                                         any(SearchControls.class));
+  }
+  
+  @Test
+  public void testExtractPassword() throws IOException {
+    File testDir = new File(System.getProperty("test.build.data", 
+                                               "target/test-dir"));
+    testDir.mkdirs();
+    File secretFile = new File(testDir, "secret.txt");
+    Writer writer = new FileWriter(secretFile);
+    writer.write("hadoop");
+    writer.close();
+    
+    LdapGroupsMapping mapping = new LdapGroupsMapping();
+    Assert.assertEquals("hadoop",
+        mapping.extractPassword(secretFile.getPath()));
+  }
+}