Selaa lähdekoodia

HADOOP-16125. Support multiple bind users in LdapGroupsMapping. Contributed by Lukas Majercak.

Inigo Goiri 6 vuotta sitten
vanhempi
commit
ba4e7bd192

+ 121 - 24
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java

@@ -31,6 +31,7 @@ import java.util.HashSet;
 import java.util.Collection;
 import java.util.Set;
 
+import javax.naming.AuthenticationException;
 import javax.naming.Context;
 import javax.naming.NamingEnumeration;
 import javax.naming.NamingException;
@@ -134,23 +135,40 @@ public class LdapGroupsMapping
   public static final String LDAP_TRUSTSTORE_PASSWORD_FILE_KEY =
       LDAP_TRUSTSTORE_PASSWORD_KEY + ".file";
 
+  /*
+   * User aliases to bind to the LDAP server with. Each alias will have
+   * to have its username and password configured, see core-default.xml
+   * and GroupsMapping.md for details.
+   */
+  public static final String BIND_USERS_KEY = LDAP_CONFIG_PREFIX +
+      ".bind.users";
+
   /*
    * 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_SUFFIX = ".bind.user";
+  public static final String BIND_USER_KEY = LDAP_CONFIG_PREFIX +
+      BIND_USER_SUFFIX;
   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_SUFFIX = ".bind.password";
+  public static final String BIND_PASSWORD_KEY = LDAP_CONFIG_PREFIX +
+      BIND_PASSWORD_SUFFIX;
   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_SUFFIX =
+      BIND_PASSWORD_SUFFIX + ".file";
+  public static final String BIND_PASSWORD_FILE_KEY = LDAP_CONFIG_PREFIX +
+      BIND_PASSWORD_FILE_SUFFIX;
   public static final String BIND_PASSWORD_FILE_DEFAULT = "";
 
+  public static final String BIND_PASSWORD_ALIAS_SUFFIX =
+      BIND_PASSWORD_SUFFIX + ".alias";
   public static final String BIND_PASSWORD_ALIAS_KEY =
-      BIND_PASSWORD_KEY + ".alias";
+      LDAP_CONFIG_PREFIX + BIND_PASSWORD_ALIAS_SUFFIX;
   public static final String BIND_PASSWORD_ALIAS_DEFAULT = "";
 
   /*
@@ -273,8 +291,14 @@ public class LdapGroupsMapping
   private String keystorePass;
   private String truststore;
   private String truststorePass;
-  private String bindUser;
-  private String bindPassword;
+
+  /*
+   * Users to bind to when connecting to LDAP. This will be a rotating
+   * iterator, cycling back to the first user if necessary.
+   */
+  private Iterator<BindUserInfo> bindUsers;
+  private BindUserInfo currentBindUser;
+
   private String userbaseDN;
   private String groupbaseDN;
   private String groupSearchFilter;
@@ -318,6 +342,8 @@ public class LdapGroupsMapping
         atemptsBeforeFailover++) {
       try {
         return doGetGroups(user, groupHierarchyLevels);
+      } catch (AuthenticationException e) {
+        switchBindUser(e);
       } catch (NamingException e) {
         LOG.warn("Failed to get groups for user {} (attempt={}/{}) using {}. " +
             "Exception: ", user, attempt, numAttempts, currentLdapUrl, e);
@@ -589,6 +615,19 @@ public class LdapGroupsMapping
     return false;
   }
 
+  /**
+   * Switch to the next available user to bind to.
+   * @param e AuthenticationException encountered when contacting LDAP
+   */
+  protected void switchBindUser(AuthenticationException e) {
+    BindUserInfo oldBindUser = this.currentBindUser;
+    currentBindUser = this.bindUsers.next();
+    if (!oldBindUser.equals(currentBindUser)) {
+      LOG.info("Switched from {} to {} after an AuthenticationException: {}",
+          oldBindUser, currentBindUser, e.getMessage());
+    }
+  }
+
   private DirContext getDirContext() throws NamingException {
     if (ctx == null) {
       // Set up the initial environment for LDAP connectivity
@@ -615,8 +654,8 @@ public class LdapGroupsMapping
         }
       }
 
-      env.put(Context.SECURITY_PRINCIPAL, bindUser);
-      env.put(Context.SECURITY_CREDENTIALS, bindPassword);
+      env.put(Context.SECURITY_PRINCIPAL, currentBindUser.username);
+      env.put(Context.SECURITY_CREDENTIALS, currentBindUser.password);
 
       env.put("com.sun.jndi.ldap.connect.timeout", conf.get(CONNECTION_TIMEOUT,
           String.valueOf(CONNECTION_TIMEOUT_DEFAULT)));
@@ -653,6 +692,7 @@ public class LdapGroupsMapping
 
   @Override
   public synchronized void setConf(Configuration conf) {
+    this.conf = conf;
     String[] urls = conf.getStrings(LDAP_URL_KEY, LDAP_URL_DEFAULT);
     if (urls == null || urls.length == 0) {
       throw new RuntimeException("LDAP URL(s) are not configured");
@@ -664,20 +704,8 @@ public class LdapGroupsMapping
     if (useSsl) {
       loadSslConf(conf);
     }
-    
-    bindUser = conf.get(BIND_USER_KEY, BIND_USER_DEFAULT);
 
-    String alias = conf.get(BIND_PASSWORD_ALIAS_KEY,
-        BIND_PASSWORD_ALIAS_DEFAULT);
-    bindPassword = getPasswordFromCredentialProviders(conf, alias, "");
-    if (bindPassword.isEmpty()) {
-      bindPassword = getPassword(conf, BIND_PASSWORD_KEY,
-          BIND_PASSWORD_DEFAULT);
-      if (bindPassword.isEmpty()) {
-        bindPassword = extractPassword(
-            conf.get(BIND_PASSWORD_FILE_KEY, BIND_PASSWORD_FILE_DEFAULT));
-      }
-    }
+    initializeBindUsers();
 
     String baseDN = conf.getTrimmed(BASE_DN_KEY, BASE_DN_DEFAULT);
 
@@ -734,8 +762,6 @@ public class LdapGroupsMapping
     this.numAttemptsBeforeFailover = conf.getInt(
         LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY,
         LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_DEFAULT);
-
-    this.conf = conf;
   }
 
   /**
@@ -821,4 +847,75 @@ public class LdapGroupsMapping
       throw new RuntimeException("Could not read password file: " + pwFile, ioe);
     }
   }
+
+  private void initializeBindUsers() {
+    List<BindUserInfo> bindUsersConfigured = new ArrayList<>();
+
+    String[] bindUserAliases = conf.getStrings(BIND_USERS_KEY);
+    if (bindUserAliases != null && bindUserAliases.length > 0) {
+
+      for (String bindUserAlias : bindUserAliases) {
+        String userConfPrefix = BIND_USERS_KEY + "." + bindUserAlias;
+        String bindUsername = conf.get(userConfPrefix + BIND_USER_SUFFIX);
+        String bindPassword = getPasswordForBindUser(userConfPrefix);
+
+        if (bindUsername == null || bindPassword == null) {
+          throw new RuntimeException("Bind username or password not " +
+              "configured for user: " + bindUserAlias);
+        }
+        bindUsersConfigured.add(new BindUserInfo(bindUsername, bindPassword));
+      }
+    } else {
+      String bindUsername = conf.get(BIND_USER_KEY, BIND_USER_DEFAULT);
+      String bindPassword = getPasswordForBindUser(LDAP_CONFIG_PREFIX);
+      bindUsersConfigured.add(new BindUserInfo(bindUsername, bindPassword));
+    }
+
+    this.bindUsers = Iterators.cycle(bindUsersConfigured);
+    this.currentBindUser = this.bindUsers.next();
+  }
+
+  private String getPasswordForBindUser(String keyPrefix) {
+    String password;
+    String alias = conf.get(keyPrefix + BIND_PASSWORD_ALIAS_SUFFIX,
+        BIND_PASSWORD_ALIAS_DEFAULT);
+    password = getPasswordFromCredentialProviders(conf, alias, "");
+    if (password.isEmpty()) {
+      password = getPassword(conf, keyPrefix + BIND_PASSWORD_SUFFIX,
+          BIND_PASSWORD_DEFAULT);
+      if (password.isEmpty()) {
+        password = extractPassword(conf.get(
+            keyPrefix + BIND_PASSWORD_FILE_SUFFIX, BIND_PASSWORD_FILE_DEFAULT));
+      }
+    }
+    return password;
+  }
+
+  private final static class BindUserInfo {
+    private final String username;
+    private final String password;
+
+    private BindUserInfo(String username, String password) {
+      this.username = username;
+      this.password = password;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof BindUserInfo)) {
+        return false;
+      }
+      return this.username.equals(((BindUserInfo) o).username);
+    }
+
+    @Override
+    public int hashCode() {
+      return this.username.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return this.username;
+    }
+  }
 }

+ 24 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -361,6 +361,30 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.group.mapping.ldap.bind.users</name>
+  <value></value>
+  <description>
+    Aliases of users to be used to bind as when connecting to the LDAP
+    server(s). Each alias will have to have its distinguished name and
+    password specified through:
+    hadoop.security.group.mapping.ldap.bind.user
+    and a password configuration such as:
+    hadoop.security.group.mapping.ldap.bind.password.alias
+
+    For example, if:
+    hadoop.security.group.mapping.ldap.bind.users=alias1,alias2
+
+    then the following configuration is valid:
+    hadoop.security.group.mapping.ldap.bind.users.alias1.bind.user=bindUser1
+    hadoop.security.group.mapping.ldap.bind.users.alias1.bind.password.alias=
+    bindPasswordAlias1
+    hadoop.security.group.mapping.ldap.bind.users.alias2.bind.user=bindUser2
+    hadoop.security.group.mapping.ldap.bind.users.alias2.bind.password.alias=
+    bindPasswordAlias2
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.group.mapping.ldap.bind.user</name>
   <value></value>

+ 21 - 5
hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md

@@ -78,11 +78,6 @@ This provider supports LDAP with simple password authentication using JNDI API.
 `hadoop.security.group.mapping.ldap.base` configures the search base for the LDAP connection. This is a distinguished name, and will typically be the root of the LDAP directory.
 Get groups for a given username first looks up the user and then looks up the groups for the user result. If the directory setup has different user and group search bases, use `hadoop.security.group.mapping.ldap.userbase` and `hadoop.security.group.mapping.ldap.groupbase` configs.
 
-If the LDAP server does not support anonymous binds,
-set the distinguished name of the user to bind in `hadoop.security.group.mapping.ldap.bind.user`.
-The path to the file containing the bind user's password is specified in `hadoop.security.group.mapping.ldap.bind.password.file`.
-This file should be readable only by the Unix user running the daemons.
-
 It is possible to set a maximum time limit when searching and awaiting a result.
 Set `hadoop.security.group.mapping.ldap.directory.search.timeout` to 0 if infinite wait period is desired. Default is 10,000 milliseconds (10 seconds).
 This is the limit for each ldap query.  If `hadoop.security.group.mapping.ldap.search.group.hierarchy.levels` is set to a positive value, then the total latency will be bounded by max(Recur Depth in LDAP, `hadoop.security.group.mapping.ldap.search.group.hierarchy.levels` ) * `hadoop.security.group.mapping.ldap.directory.search.timeout`.
@@ -90,6 +85,27 @@ This is the limit for each ldap query.  If `hadoop.security.group.mapping.ldap.s
 `hadoop.security.group.mapping.ldap.base` configures how far to walk up the groups hierarchy when resolving groups.
 By default, with a limit of 0, in order to be considered a member of a group, the user must be an explicit member in LDAP.  Otherwise, it will traverse the group hierarchy `hadoop.security.group.mapping.ldap.search.group.hierarchy.levels` levels up.
 
+### Bind user(s) ###
+If the LDAP server does not support anonymous binds,
+set the distinguished name of the user to bind in `hadoop.security.group.mapping.ldap.bind.user`.
+The path to the file containing the bind user's password is specified in `hadoop.security.group.mapping.ldap.bind.password.file`.
+This file should be readable only by the Unix user running the daemons.
+
+Multiple bind users
+--------
+If multiple bind users are required, they can be specified through `hadoop.security.group.mapping.ldap.bind.users`.
+These will represent the aliases of users to be used to bind as when connecting to the LDAP.
+Each alias will then have to have its distinguished name and password configured.
+This is useful if the bind user's password has to be reset.
+If AuthenticationException is encountered when connecting to LDAP, LDAPGroupsMapping will switch to the next bind user information and cycle back if necessary.
+
+For example, if:
+`hadoop.security.group.mapping.ldap.bind.users=alias1,alias2`
+, then the following configuration is valid:
+`hadoop.security.group.mapping.ldap.bind.users.alias1.bind.user=bindUser1`
+`hadoop.security.group.mapping.ldap.bind.users.alias1.bind.password.alias=bindPasswordAlias1`
+`hadoop.security.group.mapping.ldap.bind.users.alias2.bind.user=bindUser2`
+`hadoop.security.group.mapping.ldap.bind.users.alias2.bind.password.alias=bindPasswordAlias2`
 
 ### Active Directory ###
 The default configuration supports LDAP group name resolution with an Active Directory server.

+ 26 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java

@@ -69,6 +69,7 @@ public class TestLdapGroupsMappingBase {
 
   @Before
   public void setupMocksBase() throws NamingException {
+    DummyLdapCtxFactory.reset();
     MockitoAnnotations.initMocks(this);
     DirContext ctx = getContext();
 
@@ -189,6 +190,8 @@ public class TestLdapGroupsMappingBase {
 
     private static DirContext contextToReturn;
     private static String expectedLdapUrl;
+    private static String expectedBindUser;
+    private static String expectedBindPassword;
 
     public DummyLdapCtxFactory() {
     }
@@ -201,6 +204,20 @@ public class TestLdapGroupsMappingBase {
       expectedLdapUrl = url;
     }
 
+    public static void setExpectedBindUser(String bindUser) {
+      expectedBindUser = bindUser;
+    }
+
+    public static void setExpectedBindPassword(String bindPassword) {
+      expectedBindPassword = bindPassword;
+    }
+
+    public static void reset() {
+      expectedLdapUrl = null;
+      expectedBindUser = null;
+      expectedBindPassword = null;
+    }
+
     @Override
     public Context getInitialContext(Hashtable<?, ?> env)
         throws NamingException {
@@ -208,6 +225,15 @@ public class TestLdapGroupsMappingBase {
         String actualLdapUrl = (String) env.get(Context.PROVIDER_URL);
         assertEquals(expectedLdapUrl, actualLdapUrl);
       }
+      if (expectedBindUser != null) {
+        String actualBindUser = (String) env.get(Context.SECURITY_PRINCIPAL);
+        assertEquals(expectedBindUser, actualBindUser);
+      }
+      if (expectedBindPassword != null) {
+        String actualBindPassword = (String) env.get(
+            Context.SECURITY_CREDENTIALS);
+        assertEquals(expectedBindPassword, actualBindPassword);
+      }
       if (contextToReturn == null) {
         InitialContextFactory defaultFactory = null;
         try {

+ 247 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithBindUserSwitch.java

@@ -0,0 +1,247 @@
+/**
+ * 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 com.google.common.collect.Iterators;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import javax.naming.AuthenticationException;
+import javax.naming.NamingException;
+import javax.naming.directory.SearchControls;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.security.LdapGroupsMapping.BIND_PASSWORD_ALIAS_SUFFIX;
+import static org.apache.hadoop.security.LdapGroupsMapping.BIND_PASSWORD_FILE_SUFFIX;
+import static org.apache.hadoop.security.LdapGroupsMapping.BIND_PASSWORD_SUFFIX;
+import static org.apache.hadoop.security.LdapGroupsMapping.BIND_USERS_KEY;
+import static org.apache.hadoop.security.LdapGroupsMapping.BIND_USER_SUFFIX;
+import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_KEY;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test functionality for switching bind user information if
+ * AuthenticationExceptions are encountered.
+ */
+public class TestLdapGroupsMappingWithBindUserSwitch
+    extends TestLdapGroupsMappingBase {
+
+  private static final String TEST_USER_NAME = "some_user";
+
+  @Test
+  public void testIncorrectConfiguration() {
+    // No bind user configured for user2
+    Configuration conf = getBaseConf();
+    conf.set(BIND_USERS_KEY, "user1,user2");
+    conf.set(BIND_USERS_KEY + ".user1" + BIND_USER_SUFFIX, "bindUsername1");
+
+    LdapGroupsMapping groupsMapping = new LdapGroupsMapping();
+    try {
+      groupsMapping.setConf(conf);
+      groupsMapping.getGroups(TEST_USER_NAME);
+      fail("Should have failed with RuntimeException");
+    } catch (RuntimeException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Bind username or password not configured for user: user2", e);
+    }
+  }
+
+  @Test
+  public void testBindUserSwitchPasswordPlaintext() throws Exception {
+    Configuration conf = getBaseConf();
+    conf.set(BIND_USERS_KEY, "user1,user2");
+    conf.set(BIND_USERS_KEY + ".user1" + BIND_USER_SUFFIX, "bindUsername1");
+    conf.set(BIND_USERS_KEY + ".user2" + BIND_USER_SUFFIX, "bindUsername2");
+
+    conf.set(BIND_USERS_KEY + ".user1" + BIND_PASSWORD_SUFFIX, "bindPassword1");
+    conf.set(BIND_USERS_KEY + ".user2" + BIND_PASSWORD_SUFFIX, "bindPassword2");
+
+    doTestBindUserSwitch(conf, 1,
+        Arrays.asList("bindUsername1", "bindUsername2"),
+        Arrays.asList("bindPassword1", "bindPassword2"));
+  }
+
+  @Test
+  public void testBindUserSwitchPasswordFromAlias() throws Exception {
+    Configuration conf = getBaseConf();
+    conf.set(BIND_USERS_KEY, "joe,lukas");
+    conf.set(BIND_USERS_KEY + ".joe" + BIND_USER_SUFFIX, "joeBindUsername");
+    conf.set(BIND_USERS_KEY + ".lukas" + BIND_USER_SUFFIX, "lukasBindUsername");
+
+    conf.set(BIND_USERS_KEY + ".joe" + BIND_PASSWORD_ALIAS_SUFFIX,
+        "joeBindPasswordAlias");
+    conf.set(BIND_USERS_KEY + ".lukas" + BIND_PASSWORD_ALIAS_SUFFIX,
+        "lukasBindPasswordAlias");
+
+    setupCredentialProvider(conf);
+    createCredentialForAlias(conf, "joeBindPasswordAlias", "joeBindPassword");
+    createCredentialForAlias(conf, "lukasBindPasswordAlias",
+        "lukasBindPassword");
+
+    // Simulate 2 failures to test cycling through the bind users
+    List<String> expectedBindUsers = Arrays.asList("joeBindUsername",
+        "lukasBindUsername", "joeBindUsername");
+    List<String> expectedBindPasswords = Arrays.asList("joeBindPassword",
+        "lukasBindPassword", "joeBindPassword");
+
+    doTestBindUserSwitch(conf, 2, expectedBindUsers, expectedBindPasswords);
+  }
+
+  @Test
+  public void testBindUserSwitchPasswordFromFile() throws Exception {
+    Configuration conf = getBaseConf();
+    conf.setInt(LDAP_NUM_ATTEMPTS_KEY, 10);
+
+    conf.set(BIND_USERS_KEY, "bob,alice");
+    conf.set(BIND_USERS_KEY + ".bob" + BIND_USER_SUFFIX, "bobUsername");
+    conf.set(BIND_USERS_KEY + ".alice" + BIND_USER_SUFFIX, "aliceUsername");
+
+    conf.set(BIND_USERS_KEY + ".bob" + BIND_PASSWORD_FILE_SUFFIX,
+        createPasswordFile("bobPasswordFile1.txt", "bobBindPassword"));
+    conf.set(BIND_USERS_KEY + ".alice" + BIND_PASSWORD_FILE_SUFFIX,
+        createPasswordFile("alicePasswordFile2.txt", "aliceBindPassword"));
+
+    // Simulate 4 failures to test cycling through the bind users
+    List<String> expectedBindUsers = Arrays.asList("bobUsername",
+        "aliceUsername", "bobUsername", "aliceUsername", "bobUsername");
+    List<String> expectedBindPasswords = Arrays.asList("bobBindPassword",
+        "aliceBindPassword", "bobBindPassword", "aliceBindPassword",
+        "bobBindPassword");
+
+    doTestBindUserSwitch(conf, 4, expectedBindUsers, expectedBindPasswords);
+  }
+
+  private void setupCredentialProvider(Configuration conf) {
+    File testDir = GenericTestUtils.getTestDir();
+    final Path jksPath = new Path(testDir.toString(), "test.jks");
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+    File file = new File(testDir, "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+  }
+
+  private void createCredentialForAlias(
+      Configuration conf, String alias, String password) throws Exception {
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+    char[] bindpass = password.toCharArray();
+
+    // Ensure that we get null when the key isn't there
+    assertNull(provider.getCredentialEntry(alias));
+
+    // Create credential for the alias
+    provider.createCredentialEntry(alias, bindpass);
+    provider.flush();
+
+    // Make sure we get back the right key
+    assertArrayEquals(bindpass, provider.getCredentialEntry(
+        alias).getCredential());
+  }
+
+  private String createPasswordFile(String filename, String password)
+      throws Exception {
+    File testDir = GenericTestUtils.getTestDir();
+    testDir.mkdirs();
+    File secretFile = new File(testDir, filename);
+    Writer writer = new FileWriter(secretFile);
+    writer.write(password);
+    writer.close();
+    return secretFile.getPath();
+  }
+
+  private void doTestBindUserSwitch(
+      Configuration conf, Integer numFailures,
+      List<String> expectedBindUsers,
+      List<String> expectedBindPasswords) throws NamingException {
+    doTestBindUserSwitch(conf, numFailures, Iterators.cycle(expectedBindUsers),
+        Iterators.cycle(expectedBindPasswords));
+  }
+
+  /**
+   *
+   * @param conf Configuration to be used
+   * @param numFailures number of AuthenticationException failures to simulate
+   * @param expectedBindUsers expected sequence of distinguished user names
+   *                          when binding to LDAP
+   * @param expectedBindPasswords expected sequence of passwords to be used when
+   *                              binding to LDAP
+   * @throws NamingException from DirContext.search()
+   */
+  private void doTestBindUserSwitch(
+      Configuration conf, Integer numFailures,
+      Iterator<String> expectedBindUsers,
+      Iterator<String> expectedBindPasswords) throws NamingException {
+
+    DummyLdapCtxFactory.setExpectedBindUser(expectedBindUsers.next());
+    DummyLdapCtxFactory.setExpectedBindPassword(expectedBindPasswords.next());
+
+    final AtomicInteger failuresLeft = new AtomicInteger(numFailures);
+
+    when(getContext().search(anyString(), anyString(), any(Object[].class),
+        any(SearchControls.class))).thenAnswer(invocationOnMock -> {
+          if (failuresLeft.get() > 0) {
+            DummyLdapCtxFactory.setExpectedBindUser(expectedBindUsers.next());
+            DummyLdapCtxFactory.setExpectedBindPassword(
+                expectedBindPasswords.next());
+            failuresLeft.decrementAndGet();
+            throw new AuthenticationException();
+          }
+          // Return userNames for the first successful search()
+          if (failuresLeft.getAndDecrement() == 0) {
+            return getUserNames();
+          }
+          // Return groupNames for the second successful search()
+          return getGroupNames();
+        });
+
+    LdapGroupsMapping groupsMapping = new LdapGroupsMapping();
+    groupsMapping.setConf(conf);
+
+    List<String> groups = groupsMapping.getGroups(TEST_USER_NAME);
+    assertEquals(Arrays.asList("group1", "group2"), groups);
+
+    // There will be one search() call for each failure and
+    // 2 calls for the successful case; one for retrieving the
+    // user and one for retrieving their groups.
+    int numExpectedSearchCalls = numFailures + 2;
+    verify(getContext(), times(numExpectedSearchCalls)).search(anyString(),
+        anyString(), any(Object[].class), any(SearchControls.class));
+  }
+}