瀏覽代碼

Revert "HADOOP-15950. Failover for LdapGroupsMapping. Contributed by Lukas Majercak."

This reverts commit b98ffbe3f228b706b8e547f82e134233ab691443.
Inigo Goiri 6 年之前
父節點
當前提交
a29eb49f85

+ 50 - 113
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java

@@ -25,7 +25,6 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Hashtable;
-import java.util.Iterator;
 import java.util.List;
 import java.util.HashSet;
 import java.util.Collection;
@@ -41,10 +40,7 @@ import javax.naming.directory.SearchControls;
 import javax.naming.directory.SearchResult;
 import javax.naming.ldap.LdapName;
 import javax.naming.ldap.Rdn;
-import javax.naming.spi.InitialContextFactory;
 
-import com.google.common.collect.Iterators;
-import com.sun.jndi.ldap.LdapCtxFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
@@ -87,7 +83,7 @@ public class LdapGroupsMapping
   public static final String LDAP_CONFIG_PREFIX = "hadoop.security.group.mapping.ldap";
 
   /*
-   * URL of the LDAP server(s)
+   * URL of the LDAP server
    */
   public static final String LDAP_URL_KEY = LDAP_CONFIG_PREFIX + ".url";
   public static final String LDAP_URL_DEFAULT = "";
@@ -236,20 +232,6 @@ public class LdapGroupsMapping
       LDAP_CONFIG_PREFIX + ".read.timeout.ms";
   public static final int READ_TIMEOUT_DEFAULT = 60 * 1000; // 60 seconds
 
-  public static final String LDAP_NUM_ATTEMPTS_KEY =
-      LDAP_CONFIG_PREFIX + ".num.attempts";
-  public static final int LDAP_NUM_ATTEMPTS_DEFAULT = 3;
-
-  public static final String LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY =
-      LDAP_CONFIG_PREFIX + ".num.attempts.before.failover";
-  public static final int LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_DEFAULT =
-      LDAP_NUM_ATTEMPTS_DEFAULT;
-
-  public static final String LDAP_CTX_FACTORY_CLASS_KEY =
-      LDAP_CONFIG_PREFIX + ".ctx.factory.class";
-  public static final Class<? extends LdapCtxFactory>
-      LDAP_CTX_FACTORY_CLASS_DEFAULT = LdapCtxFactory.class;
-
   private static final Logger LOG =
       LoggerFactory.getLogger(LdapGroupsMapping.class);
 
@@ -260,10 +242,8 @@ public class LdapGroupsMapping
 
   private DirContext ctx;
   private Configuration conf;
-
-  private Iterator<String> ldapUrls;
-  private String currentLdapUrl;
-
+  
+  private String ldapUrl;
   private boolean useSsl;
   private String keystore;
   private String keystorePass;
@@ -278,15 +258,14 @@ public class LdapGroupsMapping
   private String memberOfAttr;
   private String groupMemberAttr;
   private String groupNameAttr;
-  private int groupHierarchyLevels;
+  private int    groupHierarchyLevels;
   private String posixUidAttr;
   private String posixGidAttr;
   private boolean isPosix;
   private boolean useOneQuery;
-  private int numAttempts;
-  private int numAttemptsBeforeFailover;
-  private Class<? extends InitialContextFactory> ldapCxtFactoryClass;
 
+  public static final int RECONNECT_RETRY_COUNT = 3;
+  
   /**
    * Returns list of groups for a user.
    * 
@@ -300,31 +279,20 @@ public class LdapGroupsMapping
   @Override
   public synchronized List<String> getGroups(String user) {
     /*
-     * Normal garbage collection takes care of removing Context instances when
-     * they are no longer in use. Connections used by Context instances being
-     * garbage collected will be closed automatically. So in case connection is
-     * closed and gets CommunicationException, retry some times with new new
-     * DirContext/connection.
+     * Normal garbage collection takes care of removing Context instances when they are no longer in use. 
+     * Connections used by Context instances being garbage collected will be closed automatically.
+     * So in case connection is closed and gets CommunicationException, retry some times with new new DirContext/connection. 
      */
-
-    // Tracks the number of attempts made using the same LDAP server
-    int atemptsBeforeFailover = 1;
-
-    for (int attempt = 1; attempt <= numAttempts; attempt++,
-        atemptsBeforeFailover++) {
+    for(int retry = 0; retry < RECONNECT_RETRY_COUNT; retry++) {
       try {
         return doGetGroups(user, groupHierarchyLevels);
       } catch (NamingException e) {
-        LOG.warn("Failed to get groups for user {} (attempt={}/{}) using {}. " +
-            "Exception: ", user, attempt, numAttempts, currentLdapUrl, e);
+        LOG.warn("Failed to get groups for user " + user + " (retry=" + retry
+            + ") by " + e);
         LOG.trace("TRACE", e);
-
-        if (failover(atemptsBeforeFailover, numAttemptsBeforeFailover)) {
-          atemptsBeforeFailover = 0;
-        }
       }
 
-      // Reset ctx so that new DirContext can be created with new connection
+      //reset ctx so that new DirContext can be created with new connection
       this.ctx = null;
     }
     
@@ -410,10 +378,10 @@ public class LdapGroupsMapping
   private List<String> lookupGroup(SearchResult result, DirContext c,
       int goUpHierarchy)
       throws NamingException {
-    List<String> groups = new ArrayList<>();
-    Set<String> groupDNs = new HashSet<>();
+    List<String> groups = new ArrayList<String>();
+    Set<String> groupDNs = new HashSet<String>();
 
-    NamingEnumeration<SearchResult> groupResults;
+    NamingEnumeration<SearchResult> groupResults = null;
     // perform the second LDAP query
     if (isPosix) {
       groupResults = lookupPosixGroup(result, c);
@@ -434,10 +402,10 @@ public class LdapGroupsMapping
       }
       if (goUpHierarchy > 0 && !isPosix) {
         // convert groups to a set to ensure uniqueness
-        Set<String> groupset = new HashSet<>(groups);
+        Set<String> groupset = new HashSet<String>(groups);
         goUpGroupHierarchy(groupDNs, goUpHierarchy, groupset);
         // convert set back to list for compatibility
-        groups = new ArrayList<>(groupset);
+        groups = new ArrayList<String>(groupset);
       }
     }
     return groups;
@@ -465,9 +433,11 @@ public class LdapGroupsMapping
         userSearchFilter, new Object[]{user}, SEARCH_CONTROLS);
     // return empty list if the user can not be found.
     if (!results.hasMoreElements()) {
-      LOG.debug("doGetGroups({}) returned no groups because the " +
-          "user is not found.", user);
-      return new ArrayList<>();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("doGetGroups(" + user + ") returned no groups because the " +
+            "user is not found.");
+      }
+      return new ArrayList<String>();
     }
     SearchResult result = results.nextElement();
 
@@ -485,7 +455,7 @@ public class LdapGroupsMapping
               memberOfAttr + "' attribute." +
               "Returned user object: " + result.toString());
         }
-        groups = new ArrayList<>();
+        groups = new ArrayList<String>();
         NamingEnumeration groupEnumeration = groupDNAttr.getAll();
         while (groupEnumeration.hasMore()) {
           String groupDN = groupEnumeration.next().toString();
@@ -500,7 +470,9 @@ public class LdapGroupsMapping
     if (groups == null || groups.isEmpty() || goUpHierarchy > 0) {
       groups = lookupGroup(result, c, goUpHierarchy);
     }
-    LOG.debug("doGetGroups({}) returned {}", user, groups);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("doGetGroups(" + user + ") returned " + groups);
+    }
     return groups;
   }
 
@@ -508,7 +480,7 @@ public class LdapGroupsMapping
   */
   void getGroupNames(SearchResult groupResult, Collection<String> groups,
                      Collection<String> groupDNs, boolean doGetDNs)
-                     throws NamingException {
+                     throws NamingException  {
     Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
     if (groupName == null) {
       throw new NamingException("The group object does not have " +
@@ -545,7 +517,7 @@ public class LdapGroupsMapping
       return;
     }
     DirContext context = getDirContext();
-    Set<String> nextLevelGroups = new HashSet<>();
+    Set<String> nextLevelGroups = new HashSet<String>();
     StringBuilder filter = new StringBuilder();
     filter.append("(&").append(groupSearchFilter).append("(|");
     for (String dn : groupDNs) {
@@ -565,32 +537,13 @@ public class LdapGroupsMapping
     goUpGroupHierarchy(nextLevelGroups, goUpHierarchy - 1, groups);
   }
 
-  /**
-   * Check whether we should fail over to the next LDAP server.
-   * @param attemptsMadeWithSameLdap current number of attempts made
-   *                                 with using same LDAP instance
-   * @param maxAttemptsBeforeFailover maximum number of attempts
-   *                                  before failing over
-   * @return true if we should fail over to the next LDAP server
-   */
-  protected boolean failover(
-      int attemptsMadeWithSameLdap, int maxAttemptsBeforeFailover) {
-    if (attemptsMadeWithSameLdap >= maxAttemptsBeforeFailover) {
-      String previousLdapUrl = currentLdapUrl;
-      currentLdapUrl = ldapUrls.next();
-      LOG.info("Reached {} attempts on {}, failing over to {}",
-          attemptsMadeWithSameLdap, previousLdapUrl, currentLdapUrl);
-      return true;
-    }
-    return false;
-  }
-
-  private DirContext getDirContext() throws NamingException {
+  DirContext getDirContext() throws NamingException {
     if (ctx == null) {
       // Set up the initial environment for LDAP connectivity
-      Hashtable<String, String> env = new Hashtable<>();
-      env.put(Context.INITIAL_CONTEXT_FACTORY, ldapCxtFactoryClass.getName());
-      env.put(Context.PROVIDER_URL, currentLdapUrl);
+      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
@@ -628,7 +581,7 @@ public class LdapGroupsMapping
    * Caches groups, no need to do that for this provider
    */
   @Override
-  public void cacheGroupsRefresh() {
+  public void cacheGroupsRefresh() throws IOException {
     // does nothing in this provider of user to groups mapping
   }
 
@@ -638,7 +591,7 @@ public class LdapGroupsMapping
    * @param groups unused
    */
   @Override
-  public void cacheGroupsAdd(List<String> groups) {
+  public void cacheGroupsAdd(List<String> groups) throws IOException {
     // does nothing in this provider of user to groups mapping
   }
 
@@ -649,12 +602,10 @@ public class LdapGroupsMapping
 
   @Override
   public synchronized void setConf(Configuration 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");
+    ldapUrl = conf.get(LDAP_URL_KEY, LDAP_URL_DEFAULT);
+    if (ldapUrl == null || ldapUrl.isEmpty()) {
+      throw new RuntimeException("LDAP URL is not configured");
     }
-    ldapUrls = Iterators.cycle(urls);
-    currentLdapUrl = ldapUrls.next();
 
     useSsl = conf.getBoolean(LDAP_USE_SSL_KEY, LDAP_USE_SSL_DEFAULT);
     if (useSsl) {
@@ -670,13 +621,17 @@ public class LdapGroupsMapping
     
     String baseDN = conf.getTrimmed(BASE_DN_KEY, BASE_DN_DEFAULT);
 
-    // User search base which defaults to base dn.
+    //User search base which defaults to base dn.
     userbaseDN = conf.getTrimmed(USER_BASE_DN_KEY, baseDN);
-    LOG.debug("Usersearch baseDN: {}", userbaseDN);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Usersearch baseDN: " + userbaseDN);
+    }
 
-    // Group search base which defaults to base dn.
+    //Group search base which defaults to base dn.
     groupbaseDN = conf.getTrimmed(GROUP_BASE_DN_KEY, baseDN);
-    LOG.debug("Groupsearch baseDN: {}", groupbaseDN);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Groupsearch baseDN: " + userbaseDN);
+    }
 
     groupSearchFilter =
         conf.get(GROUP_SEARCH_FILTER_KEY, GROUP_SEARCH_FILTER_DEFAULT);
@@ -700,8 +655,7 @@ public class LdapGroupsMapping
     posixGidAttr =
         conf.get(POSIX_GID_ATTR_KEY, POSIX_GID_ATTR_DEFAULT);
 
-    int dirSearchTimeout = conf.getInt(DIRECTORY_SEARCH_TIMEOUT,
-        DIRECTORY_SEARCH_TIMEOUT_DEFAULT);
+    int dirSearchTimeout = conf.getInt(DIRECTORY_SEARCH_TIMEOUT, DIRECTORY_SEARCH_TIMEOUT_DEFAULT);
     SEARCH_CONTROLS.setTimeLimit(dirSearchTimeout);
     // Limit the attributes returned to only those required to speed up the search.
     // See HADOOP-10626 and HADOOP-12001 for more details.
@@ -715,26 +669,9 @@ public class LdapGroupsMapping
     }
     SEARCH_CONTROLS.setReturningAttributes(returningAttributes);
 
-    ldapCxtFactoryClass = conf.getClass(LDAP_CTX_FACTORY_CLASS_KEY,
-        LDAP_CTX_FACTORY_CLASS_DEFAULT, InitialContextFactory.class);
-
-    this.numAttempts = conf.getInt(LDAP_NUM_ATTEMPTS_KEY,
-        LDAP_NUM_ATTEMPTS_DEFAULT);
-    this.numAttemptsBeforeFailover = conf.getInt(
-        LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY,
-        LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_DEFAULT);
-
     this.conf = conf;
   }
 
-  /**
-   * Get URLs of configured LDAP servers.
-   * @return URLs of LDAP servers being used.
-   */
-  public Iterator<String> getLdapUrls() {
-    return ldapUrls;
-  }
-
   private void loadSslConf(Configuration sslConf) {
     keystore = sslConf.get(LDAP_KEYSTORE_KEY, LDAP_KEYSTORE_DEFAULT);
     keystorePass = getPassword(sslConf, LDAP_KEYSTORE_PASSWORD_KEY,
@@ -784,8 +721,8 @@ public class LdapGroupsMapping
         password = new String(passchars);
       }
     } catch (IOException ioe) {
-      LOG.warn("Exception while trying to get password for alias {}:",
-          alias, ioe);
+      LOG.warn("Exception while trying to get password for alias " + alias
+              + ": ", ioe);
     }
     return password;
   }

+ 2 - 26
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -233,36 +233,12 @@
   </description>
 </property>
 
-<property>
-  <name>hadoop.security.group.mapping.ldap.num.attempts</name>
-  <value>3</value>
-  <description>
-    This property is the number of attempts to be made for LDAP operations.
-    If this limit is exceeded, LdapGroupsMapping will return an empty
-    group list.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.num.attempts.before.failover</name>
-  <value>3</value>
-  <description>
-    This property is the number of attempts to be made for LDAP operations
-    using a single LDAP instance. If multiple LDAP servers are configured
-    and this number of failed operations is reached, we will switch to the
-    next LDAP server. The configuration for the overall number of attempts
-    will still be respected, failover will thus be performed only if this
-    property is less than hadoop.security.group.mapping.ldap.num.attempts.
-  </description>
-</property>
-
 <property>
   <name>hadoop.security.group.mapping.ldap.url</name>
   <value></value>
   <description>
-    The URL of the LDAP server(s) to use for resolving user groups when using
-    the LdapGroupsMapping user to group mapping. Supports configuring multiple
-    LDAP servers via a comma-separated list.
+    The URL of the LDAP server to use for resolving user groups when using
+    the LdapGroupsMapping user to group mapping.
   </description>
 </property>
 

+ 1 - 53
hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md

@@ -73,7 +73,7 @@ The cache invalidation is configurable via `hadoop.security.groups.negative-cach
 LDAP Groups Mapping
 --------
 This provider supports LDAP with simple password authentication using JNDI API.
-`hadoop.security.group.mapping.ldap.url` must be set. This refers to the URL of the LDAP server(s) for resolving user groups. It supports configuring multiple LDAP servers via a comma-separated list.
+`hadoop.security.group.mapping.ldap.url` must be set. This refers to the URL of the LDAP server for resolving user groups.
 
 `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.
@@ -115,58 +115,6 @@ To enable this feature, set `hadoop.security.group.mapping.ldap.search.attr.memb
 If the LDAP server's certificate is not signed by a well known certificate authority, specify the path to the truststore in `hadoop.security.group.mapping.ldap.ssl.truststore`.
 Similar to keystore, specify the truststore password file in `hadoop.security.group.mapping.ldap.ssl.truststore.password.file`.
 
-### Configuring retries and multiple LDAP servers with failover ###
-If there are issues encountered when retrieving information from LDAP servers, the request will be retried. To configure the number of retries, use the following configuration:
-
-```<property>
-     <name>hadoop.security.group.mapping.ldap.num.attempts</name>
-     <value>3</value>
-     <description>
-       This property is the number of attempts to be made for LDAP operations.
-       If this limit is exceeded, LdapGroupsMapping will return an empty
-       group list.
-     </description>
-    </property>
-```
-
-LDAP Groups Mapping also supports configuring multiple LDAP servers and failover if a particular instance is not available or is misbehaving.
-The following configuration shows configuring 3 LDAP servers. Additionally, 2 attempts will be made for each server before failing over to the next one, with 6 attempts overall before failing.
-
-```
-<property>
-  <name>hadoop.security.group.mapping.ldap.url</name>
-  <value>ldap://server1,ldap://server2,ldap://server3</value>
-  <description>
-    The URL of the LDAP server(s) to use for resolving user groups when using
-    the LdapGroupsMapping user to group mapping. Supports configuring multiple
-    LDAP servers via a comma-separated list.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.num.attempts</name>
-  <value>6</value>
-  <description>
-    This property is the number of attempts to be made for LDAP operations.
-    If this limit is exceeded, LdapGroupsMapping will return an empty
-    group list.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.num.attempts.before.failover</name>
-  <value>2</value>
-  <description>
-    This property is the number of attempts to be made for LDAP operations
-    using a single LDAP instance. If multiple LDAP servers are configured
-    and this number of failed operations is reached, we will switch to the
-    next LDAP server. The configuration for the overall number of attempts
-    will still be respected, failover will thus be performed only if this
-    property is less than hadoop.security.group.mapping.ldap.num.attempts.
-  </description>
-</property>
-```
-
 Composite Groups Mapping
 --------
 `CompositeGroupsMapping` works by enumerating a list of service providers in `hadoop.security.group.mapping.providers`.

+ 48 - 32
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java

@@ -18,12 +18,11 @@
 package org.apache.hadoop.security;
 
 import static org.apache.hadoop.security.LdapGroupsMapping.CONNECTION_TIMEOUT;
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_KEY;
 import static org.apache.hadoop.security.LdapGroupsMapping.READ_TIMEOUT;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
@@ -83,16 +82,14 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
 
   private final String userDN = "CN=some_user,DC=test,DC=com";
 
-  private static final String TEST_LDAP_URL = "ldap://test";
-
   @Before
-  public void setupMocks() {
+  public void setupMocks() throws NamingException {
     when(getUserSearchResult().getNameInNamespace()).
         thenReturn(userDN);
   }
   
   @Test
-  public void testGetGroups() throws NamingException {
+  public void testGetGroups() throws IOException, NamingException {
     // The search functionality of the mock context is reused, so we will
     // return the user NamingEnumeration first, and then the group
     when(getContext().search(anyString(), anyString(), any(Object[].class),
@@ -103,7 +100,9 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
 
   @Test
   public void testGetGroupsWithDifferentBaseDNs() throws Exception {
-    Configuration conf = getBaseConf(TEST_LDAP_URL);
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
     String userBaseDN = "ou=Users,dc=xxx,dc=com ";
     String groupBaseDN = " ou=Groups,dc=xxx,dc=com";
     conf.set(LdapGroupsMapping.USER_BASE_DN_KEY, userBaseDN);
@@ -114,7 +113,9 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
 
   @Test
   public void testGetGroupsWithDefaultBaseDN() throws Exception {
-    Configuration conf = getBaseConf(TEST_LDAP_URL);
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
     String baseDN = " dc=xxx,dc=com ";
     conf.set(LdapGroupsMapping.BASE_DN_KEY, baseDN);
     doTestGetGroupsWithBaseDN(conf, baseDN.trim(), baseDN.trim());
@@ -158,7 +159,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
   }
 
   @Test
-  public void testGetGroupsWithHierarchy() throws NamingException {
+  public void testGetGroupsWithHierarchy() throws IOException, NamingException {
     // The search functionality of the mock context is reused, so we will
     // return the user NamingEnumeration first, and then the group
     // The parent search is run once for each level, and is a different search
@@ -174,7 +175,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
   }
 
   @Test
-  public void testGetGroupsWithConnectionClosed() throws NamingException {
+  public void testGetGroupsWithConnectionClosed() throws IOException, NamingException {
     // The case mocks connection is closed/gc-ed, so the first search call throws CommunicationException,
     // then after reconnected return the user NamingEnumeration first, and then the group
     when(getContext().search(anyString(), anyString(), any(Object[].class),
@@ -189,20 +190,21 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
   }
 
   @Test
-  public void testGetGroupsWithLdapDown() throws NamingException {
+  public void testGetGroupsWithLdapDown() throws IOException, NamingException {
     // This mocks the case where Ldap server is down, and always throws CommunicationException 
     when(getContext().search(anyString(), anyString(), any(Object[].class),
         any(SearchControls.class)))
         .thenThrow(new CommunicationException("Connection is closed"));
     
     // Ldap server is down, no groups should be retrieved
-    doTestGetGroups(Arrays.asList(new String[] {}), 4);
+    doTestGetGroups(Arrays.asList(new String[] {}), 
+        LdapGroupsMapping.RECONNECT_RETRY_COUNT);
   }
   
-  private void doTestGetGroups(List<String> expectedGroups, int searchTimes)
-      throws NamingException {
-    Configuration conf = getBaseConf(TEST_LDAP_URL);
-    conf.setInt(LDAP_NUM_ATTEMPTS_KEY, searchTimes);
+  private void doTestGetGroups(List<String> expectedGroups, int searchTimes) throws IOException, NamingException {  
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
 
     LdapGroupsMapping groupsMapping = getGroupsMapping();
     groupsMapping.setConf(conf);
@@ -214,12 +216,17 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
     
     // We should have searched for a user, and then two groups
     verify(getContext(), times(searchTimes)).search(anyString(),
-        anyString(), any(Object[].class), any(SearchControls.class));
+                                         anyString(),
+                                         any(Object[].class),
+                                         any(SearchControls.class));
   }
 
   private void doTestGetGroupsWithParent(List<String> expectedGroups,
-      int searchTimesGroup, int searchTimesParentGroup) throws NamingException {
-    Configuration conf = getBaseConf(TEST_LDAP_URL);
+      int searchTimesGroup, int searchTimesParentGroup)
+          throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
     // Set the config to get parents 1 level up
     conf.setInt(LdapGroupsMapping.GROUP_HIERARCHY_LEVELS_KEY, 1);
 
@@ -230,14 +237,18 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
     List<String> groups = groupsMapping.getGroups("some_user");
 
     // compare lists, ignoring the order
-    Assert.assertEquals(new HashSet<>(expectedGroups), new HashSet<>(groups));
+    Assert.assertEquals(new HashSet<String>(expectedGroups),
+        new HashSet<String>(groups));
 
     // We should have searched for a user, and group
     verify(getContext(), times(searchTimesGroup)).search(anyString(),
-        anyString(), any(Object[].class), any(SearchControls.class));
+                                         anyString(),
+                                         any(Object[].class),
+                                         any(SearchControls.class));
     // One groups search for the parent group should have been done
-    verify(getContext(), times(searchTimesParentGroup)).search(
-        anyString(), anyString(), any(SearchControls.class));
+    verify(getContext(), times(searchTimesParentGroup)).search(anyString(),
+                                         anyString(),
+                                         any(SearchControls.class));
   }
 
   @Test
@@ -257,7 +268,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
   @Test
   public void testConfGetPassword() throws Exception {
     File testDir = GenericTestUtils.getTestDir();
-    Configuration conf = getBaseConf();
+    Configuration conf = new Configuration();
     final Path jksPath = new Path(testDir.toString(), "test.jks");
     final String ourUrl =
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
@@ -272,10 +283,10 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
     char[] storepass = {'s', 't', 'o', 'r', 'e', 'p', 'a', 's', 's'};
 
     // ensure that we get nulls when the key isn't there
-    assertNull(provider.getCredentialEntry(
+    assertEquals(null, provider.getCredentialEntry(
         LdapGroupsMapping.BIND_PASSWORD_KEY));
-    assertNull(provider.getCredentialEntry(
-        LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY));
+    assertEquals(null, provider.getCredentialEntry
+        (LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY));
 
     // create new aliases
     try {
@@ -339,8 +350,9 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
       ldapServer.start();
 
       final LdapGroupsMapping mapping = new LdapGroupsMapping();
-      String ldapUrl = "ldap://localhost:" + serverSock.getLocalPort();
-      final Configuration conf = getBaseConf(ldapUrl, null);
+      final Configuration conf = new Configuration();
+      conf.set(LdapGroupsMapping.LDAP_URL_KEY,
+          "ldap://localhost:" + serverSock.getLocalPort());
       conf.setInt(CONNECTION_TIMEOUT, connectionTimeoutMs);
       mapping.setConf(conf);
 
@@ -394,8 +406,9 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
       ldapServer.start();
 
       final LdapGroupsMapping mapping = new LdapGroupsMapping();
-      String ldapUrl = "ldap://localhost:" + serverSock.getLocalPort();
-      final Configuration conf = getBaseConf(ldapUrl, null);
+      final Configuration conf = new Configuration();
+      conf.set(LdapGroupsMapping.LDAP_URL_KEY,
+          "ldap://localhost:" + serverSock.getLocalPort());
       conf.setInt(READ_TIMEOUT, readTimeoutMs);
       mapping.setConf(conf);
 
@@ -423,12 +436,15 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
    */
   @Test(timeout = 10000)
   public void testSetConf() throws Exception {
-    Configuration conf = getBaseConf(TEST_LDAP_URL);
+    Configuration conf = new Configuration();
     Configuration mockConf = Mockito.spy(conf);
     when(mockConf.getPassword(anyString()))
         .thenThrow(new IOException("injected IOException"));
+    // Set a dummy LDAP server URL.
+    mockConf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
 
     LdapGroupsMapping groupsMapping = getGroupsMapping();
     groupsMapping.setConf(mockConf);
   }
+
 }

+ 2 - 74
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java

@@ -18,15 +18,10 @@
 
 package org.apache.hadoop.security;
 
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_CTX_FACTORY_CLASS_DEFAULT;
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_CTX_FACTORY_CLASS_KEY;
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_URL_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import javax.naming.Context;
 import javax.naming.NamingEnumeration;
 import javax.naming.NamingException;
 import javax.naming.directory.Attribute;
@@ -36,17 +31,13 @@ import javax.naming.directory.BasicAttributes;
 import javax.naming.directory.DirContext;
 import javax.naming.directory.SearchControls;
 import javax.naming.directory.SearchResult;
-import javax.naming.spi.InitialContextFactory;
 
-import org.apache.hadoop.conf.Configuration;
 import org.junit.Before;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.MockitoAnnotations;
 import org.mockito.Spy;
 
-import java.util.Hashtable;
-
 public class TestLdapGroupsMappingBase {
   @Mock
   private DirContext context;
@@ -71,6 +62,7 @@ public class TestLdapGroupsMappingBase {
   public void setupMocksBase() throws NamingException {
     MockitoAnnotations.initMocks(this);
     DirContext ctx = getContext();
+    doReturn(ctx).when(groupsMapping).getDirContext();
 
     when(ctx.search(Mockito.anyString(), Mockito.anyString(),
         Mockito.any(Object[].class), Mockito.any(SearchControls.class))).
@@ -125,30 +117,9 @@ public class TestLdapGroupsMappingBase {
         thenReturn("CN=some_group,DC=test,DC=com");
   }
 
-  protected Configuration getBaseConf() {
-    return getBaseConf("ldap://test");
-  }
-
-  protected Configuration getBaseConf(String ldapUrl) {
-    return getBaseConf(ldapUrl, getContext());
-  }
-
-  protected Configuration getBaseConf(
-      String ldapUrl, DirContext contextToReturn) {
-    DummyLdapCtxFactory.setContextToReturn(contextToReturn);
-    DummyLdapCtxFactory.setExpectedLdapUrl(ldapUrl);
-
-    Configuration conf = new Configuration();
-    conf.set(LDAP_URL_KEY, ldapUrl);
-    conf.setClass(LDAP_CTX_FACTORY_CLASS_KEY, DummyLdapCtxFactory.class,
-        InitialContextFactory.class);
-    return conf;
-  }
-
   protected DirContext getContext() {
     return context;
   }
-
   protected NamingEnumeration<SearchResult> getUserNames() {
     return userNames;
   }
@@ -168,56 +139,13 @@ public class TestLdapGroupsMappingBase {
   protected LdapGroupsMapping getGroupsMapping() {
     return groupsMapping;
   }
-
   protected String[] getTestGroups() {
     return testGroups;
   }
-
   protected NamingEnumeration getParentGroupNames() {
     return parentGroupNames;
   }
-
   protected String[] getTestParentGroups() {
     return testParentGroups;
   }
-
-  /**
-   * Ldap Context Factory implementation to be used for testing to check
-   * contexts are requested for the expected LDAP server URLs etc.
-   */
-  public static class DummyLdapCtxFactory implements InitialContextFactory {
-
-    private static DirContext contextToReturn;
-    private static String expectedLdapUrl;
-
-    public DummyLdapCtxFactory() {
-    }
-
-    protected static void setContextToReturn(DirContext ctx) {
-      contextToReturn = ctx;
-    }
-
-    protected static void setExpectedLdapUrl(String url) {
-      expectedLdapUrl = url;
-    }
-
-    @Override
-    public Context getInitialContext(Hashtable<?, ?> env)
-        throws NamingException {
-      if (expectedLdapUrl != null) {
-        String actualLdapUrl = (String) env.get(Context.PROVIDER_URL);
-        assertEquals(expectedLdapUrl, actualLdapUrl);
-      }
-      if (contextToReturn == null) {
-        InitialContextFactory defaultFactory = null;
-        try {
-          defaultFactory = LDAP_CTX_FACTORY_CLASS_DEFAULT.newInstance();
-        } catch (ReflectiveOperationException e) {
-          fail("Could not initialize the default factory");
-        }
-        return defaultFactory.getInitialContext(env);
-      }
-      return contextToReturn;
-    }
-  }
 }

+ 0 - 142
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithFailover.java

@@ -1,142 +0,0 @@
-/**
- * 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 org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import javax.naming.CommunicationException;
-import javax.naming.directory.SearchControls;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY;
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_NUM_ATTEMPTS_KEY;
-import static org.apache.hadoop.security.LdapGroupsMapping.LDAP_URL_KEY;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Test failover functionality for switching to different
- * LDAP server URLs upon failures.
- */
-public class TestLdapGroupsMappingWithFailover
-    extends TestLdapGroupsMappingBase {
-
-  private static final String TEST_USER_NAME = "some_user";
-
-  /**
-   * Test that when disabled, we will retry the configured number
-   * of times using the same LDAP server.
-   */
-  @Test
-  public void testDoesNotFailoverWhenDisabled() throws Exception {
-    final int numAttempts = 3;
-    Configuration conf = getBaseConf();
-    conf.setStrings(LDAP_URL_KEY, "ldap://test", "ldap://test1",
-        "ldap://test2");
-    DummyLdapCtxFactory.setExpectedLdapUrl("ldap://test");
-    conf.setInt(LDAP_NUM_ATTEMPTS_KEY, numAttempts);
-    conf.setInt(LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY, numAttempts);
-
-    when(getContext().search(anyString(), anyString(), any(Object[].class),
-        any(SearchControls.class))).thenThrow(new CommunicationException());
-
-    LdapGroupsMapping groupsMapping = getGroupsMapping();
-    groupsMapping.setConf(conf);
-
-    List<String> groups = groupsMapping.getGroups(TEST_USER_NAME);
-    assertTrue(groups.isEmpty());
-
-    // Test that we made 3 attempts using the same server
-    verify(getContext(), times(numAttempts)).search(anyString(), anyString(),
-        any(Object[].class), any(SearchControls.class));
-  }
-
-  /**
-   * Test that when configured, we will make the specified amount of
-   * attempts using one ldap url before failing over to the next one.
-   *
-   * This also tests that we wrap back to the first server
-   * if we've tried them all.
-   */
-  @Test
-  public void testFailover() throws Exception {
-    Queue<String> ldapUrls = new LinkedList<>();
-    ldapUrls.add("ldap://test");
-    ldapUrls.add("ldap://test1");
-    ldapUrls.add("ldap://test2");
-
-    final int numAttempts = 12;
-    final int numAttemptsBeforeFailover = 2;
-
-    Configuration conf = getBaseConf();
-    conf.setStrings(LDAP_URL_KEY, "ldap://test", "ldap://test1",
-        "ldap://test2");
-    conf.setInt(LDAP_NUM_ATTEMPTS_KEY, numAttempts);
-    conf.setInt(LDAP_NUM_ATTEMPTS_BEFORE_FAILOVER_KEY,
-        numAttemptsBeforeFailover);
-
-    // Set the first expected url and add it back to the queue
-    String nextLdapUrl = ldapUrls.remove();
-    DummyLdapCtxFactory.setExpectedLdapUrl(nextLdapUrl);
-    ldapUrls.add(nextLdapUrl);
-
-    // Number of attempts using a single ldap server url
-    final AtomicInteger serverAttempts = new AtomicInteger(
-        numAttemptsBeforeFailover);
-
-    when(getContext().search(anyString(), anyString(), any(Object[].class),
-        any(SearchControls.class))).thenAnswer(new Answer<Object>() {
-          @Override
-          public Object answer(InvocationOnMock invocationOnMock)
-              throws Throwable {
-            if (serverAttempts.get() == 1) {
-              // Take the head of the queue and re-queue it to the back
-              String nextLdapUrl = ldapUrls.remove();
-              ldapUrls.add(nextLdapUrl);
-
-              DummyLdapCtxFactory.setExpectedLdapUrl(nextLdapUrl);
-              serverAttempts.set(numAttemptsBeforeFailover);
-            } else {
-              serverAttempts.decrementAndGet();
-            }
-            throw new CommunicationException();
-          }
-        });
-
-    LdapGroupsMapping groupsMapping = getGroupsMapping();
-    groupsMapping.setConf(conf);
-
-    List<String> groups = groupsMapping.getGroups(TEST_USER_NAME);
-    assertTrue(groups.isEmpty());
-
-    // Test that we made 6 attempts overall
-    verify(getContext(), times(numAttempts)).search(anyString(),
-        anyString(), any(Object[].class), any(SearchControls.class));
-  }
-}

+ 10 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithOneQuery.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.security;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
@@ -67,7 +68,7 @@ public class TestLdapGroupsMappingWithOneQuery
   }
 
   @Test
-  public void testGetGroups() throws NamingException {
+  public void testGetGroups() throws IOException, NamingException {
     // given a user whose ldap query returns a user object with three "memberOf"
     // properties, return an array of strings representing its groups.
     String[] testGroups = new String[] {"abc", "xyz", "sss"};
@@ -75,9 +76,10 @@ public class TestLdapGroupsMappingWithOneQuery
   }
 
   private void doTestGetGroups(List<String> expectedGroups)
-      throws NamingException {
-    String ldapUrl = "ldap://test";
-    Configuration conf = getBaseConf(ldapUrl);
+      throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
     // enable single-query lookup
     conf.set(LdapGroupsMapping.MEMBEROF_ATTR_KEY, "memberOf");
 
@@ -90,7 +92,9 @@ public class TestLdapGroupsMappingWithOneQuery
     Assert.assertEquals(expectedGroups, groups);
 
     // We should have only made one query because single-query lookup is enabled
-    verify(getContext(), times(1)).search(anyString(), anyString(),
-        any(Object[].class), any(SearchControls.class));
+    verify(getContext(), times(1)).search(anyString(),
+        anyString(),
+        any(Object[].class),
+        any(SearchControls.class));
   }
 }

+ 6 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java

@@ -28,6 +28,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
@@ -61,7 +62,7 @@ public class TestLdapGroupsMappingWithPosixGroup
   }
 
   @Test
-  public void testGetGroups() throws NamingException {
+  public void testGetGroups() throws IOException, NamingException {
     // The search functionality of the mock context is reused, so we will
     // return the user NamingEnumeration first, and then the group
     when(getContext().search(anyString(), contains("posix"),
@@ -72,9 +73,10 @@ public class TestLdapGroupsMappingWithPosixGroup
   }
 
   private void doTestGetGroups(List<String> expectedGroups, int searchTimes)
-      throws NamingException {
-    String ldapUrl = "ldap://test";
-    Configuration conf = getBaseConf(ldapUrl);
+      throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
     conf.set(LdapGroupsMapping.GROUP_SEARCH_FILTER_KEY,
         "(objectClass=posixGroup)(cn={0})");
     conf.set(LdapGroupsMapping.USER_SEARCH_FILTER_KEY,