HADOOP-15950. Failover for LdapGroupsMapping. Contributed by Lukas Majercak.
This commit is contained in:
parent
6b01e4d2a8
commit
7cfcabc473
|
@ -25,6 +25,7 @@ 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;
|
||||
|
@ -40,7 +41,10 @@ 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;
|
||||
|
@ -83,7 +87,7 @@ public class LdapGroupsMapping
|
|||
public static final String LDAP_CONFIG_PREFIX = "hadoop.security.group.mapping.ldap";
|
||||
|
||||
/*
|
||||
* URL of the LDAP server
|
||||
* URL of the LDAP server(s)
|
||||
*/
|
||||
public static final String LDAP_URL_KEY = LDAP_CONFIG_PREFIX + ".url";
|
||||
public static final String LDAP_URL_DEFAULT = "";
|
||||
|
@ -232,6 +236,20 @@ 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);
|
||||
|
||||
|
@ -243,7 +261,9 @@ public class LdapGroupsMapping
|
|||
private DirContext ctx;
|
||||
private Configuration conf;
|
||||
|
||||
private String ldapUrl;
|
||||
private Iterator<String> ldapUrls;
|
||||
private String currentLdapUrl;
|
||||
|
||||
private boolean useSsl;
|
||||
private String keystore;
|
||||
private String keystorePass;
|
||||
|
@ -258,13 +278,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;
|
||||
|
||||
public static final int RECONNECT_RETRY_COUNT = 3;
|
||||
private int numAttempts;
|
||||
private int numAttemptsBeforeFailover;
|
||||
private Class<? extends InitialContextFactory> ldapCxtFactoryClass;
|
||||
|
||||
/**
|
||||
* Returns list of groups for a user.
|
||||
|
@ -279,20 +300,31 @@ 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.
|
||||
*/
|
||||
for(int retry = 0; retry < RECONNECT_RETRY_COUNT; retry++) {
|
||||
|
||||
// Tracks the number of attempts made using the same LDAP server
|
||||
int atemptsBeforeFailover = 1;
|
||||
|
||||
for (int attempt = 1; attempt <= numAttempts; attempt++,
|
||||
atemptsBeforeFailover++) {
|
||||
try {
|
||||
return doGetGroups(user, groupHierarchyLevels);
|
||||
} catch (NamingException e) {
|
||||
LOG.warn("Failed to get groups for user " + user + " (retry=" + retry
|
||||
+ ") by " + e);
|
||||
LOG.warn("Failed to get groups for user {} (attempt={}/{}) using {}. " +
|
||||
"Exception: ", user, attempt, numAttempts, currentLdapUrl, 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;
|
||||
}
|
||||
|
||||
|
@ -378,10 +410,10 @@ public class LdapGroupsMapping
|
|||
private List<String> lookupGroup(SearchResult result, DirContext c,
|
||||
int goUpHierarchy)
|
||||
throws NamingException {
|
||||
List<String> groups = new ArrayList<String>();
|
||||
Set<String> groupDNs = new HashSet<String>();
|
||||
List<String> groups = new ArrayList<>();
|
||||
Set<String> groupDNs = new HashSet<>();
|
||||
|
||||
NamingEnumeration<SearchResult> groupResults = null;
|
||||
NamingEnumeration<SearchResult> groupResults;
|
||||
// perform the second LDAP query
|
||||
if (isPosix) {
|
||||
groupResults = lookupPosixGroup(result, c);
|
||||
|
@ -402,10 +434,10 @@ public class LdapGroupsMapping
|
|||
}
|
||||
if (goUpHierarchy > 0 && !isPosix) {
|
||||
// convert groups to a set to ensure uniqueness
|
||||
Set<String> groupset = new HashSet<String>(groups);
|
||||
Set<String> groupset = new HashSet<>(groups);
|
||||
goUpGroupHierarchy(groupDNs, goUpHierarchy, groupset);
|
||||
// convert set back to list for compatibility
|
||||
groups = new ArrayList<String>(groupset);
|
||||
groups = new ArrayList<>(groupset);
|
||||
}
|
||||
}
|
||||
return groups;
|
||||
|
@ -433,11 +465,9 @@ public class LdapGroupsMapping
|
|||
userSearchFilter, new Object[]{user}, SEARCH_CONTROLS);
|
||||
// return empty list if the user can not be found.
|
||||
if (!results.hasMoreElements()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("doGetGroups(" + user + ") returned no groups because the " +
|
||||
"user is not found.");
|
||||
}
|
||||
return new ArrayList<String>();
|
||||
LOG.debug("doGetGroups({}) returned no groups because the " +
|
||||
"user is not found.", user);
|
||||
return new ArrayList<>();
|
||||
}
|
||||
SearchResult result = results.nextElement();
|
||||
|
||||
|
@ -455,7 +485,7 @@ public class LdapGroupsMapping
|
|||
memberOfAttr + "' attribute." +
|
||||
"Returned user object: " + result.toString());
|
||||
}
|
||||
groups = new ArrayList<String>();
|
||||
groups = new ArrayList<>();
|
||||
NamingEnumeration groupEnumeration = groupDNAttr.getAll();
|
||||
while (groupEnumeration.hasMore()) {
|
||||
String groupDN = groupEnumeration.next().toString();
|
||||
|
@ -470,9 +500,7 @@ public class LdapGroupsMapping
|
|||
if (groups == null || groups.isEmpty() || goUpHierarchy > 0) {
|
||||
groups = lookupGroup(result, c, goUpHierarchy);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("doGetGroups(" + user + ") returned " + groups);
|
||||
}
|
||||
LOG.debug("doGetGroups({}) returned {}", user, groups);
|
||||
return groups;
|
||||
}
|
||||
|
||||
|
@ -480,7 +508,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 " +
|
||||
|
@ -517,7 +545,7 @@ public class LdapGroupsMapping
|
|||
return;
|
||||
}
|
||||
DirContext context = getDirContext();
|
||||
Set<String> nextLevelGroups = new HashSet<String>();
|
||||
Set<String> nextLevelGroups = new HashSet<>();
|
||||
StringBuilder filter = new StringBuilder();
|
||||
filter.append("(&").append(groupSearchFilter).append("(|");
|
||||
for (String dn : groupDNs) {
|
||||
|
@ -537,13 +565,32 @@ public class LdapGroupsMapping
|
|||
goUpGroupHierarchy(nextLevelGroups, goUpHierarchy - 1, groups);
|
||||
}
|
||||
|
||||
DirContext getDirContext() throws NamingException {
|
||||
/**
|
||||
* 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 {
|
||||
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);
|
||||
Hashtable<String, String> env = new Hashtable<>();
|
||||
env.put(Context.INITIAL_CONTEXT_FACTORY, ldapCxtFactoryClass.getName());
|
||||
env.put(Context.PROVIDER_URL, currentLdapUrl);
|
||||
env.put(Context.SECURITY_AUTHENTICATION, "simple");
|
||||
|
||||
// Set up SSL security, if necessary
|
||||
|
@ -581,7 +628,7 @@ public class LdapGroupsMapping
|
|||
* Caches groups, no need to do that for this provider
|
||||
*/
|
||||
@Override
|
||||
public void cacheGroupsRefresh() throws IOException {
|
||||
public void cacheGroupsRefresh() {
|
||||
// does nothing in this provider of user to groups mapping
|
||||
}
|
||||
|
||||
|
@ -591,7 +638,7 @@ public class LdapGroupsMapping
|
|||
* @param groups unused
|
||||
*/
|
||||
@Override
|
||||
public void cacheGroupsAdd(List<String> groups) throws IOException {
|
||||
public void cacheGroupsAdd(List<String> groups) {
|
||||
// does nothing in this provider of user to groups mapping
|
||||
}
|
||||
|
||||
|
@ -602,10 +649,12 @@ public class LdapGroupsMapping
|
|||
|
||||
@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");
|
||||
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");
|
||||
}
|
||||
ldapUrls = Iterators.cycle(urls);
|
||||
currentLdapUrl = ldapUrls.next();
|
||||
|
||||
useSsl = conf.getBoolean(LDAP_USE_SSL_KEY, LDAP_USE_SSL_DEFAULT);
|
||||
if (useSsl) {
|
||||
|
@ -621,17 +670,13 @@ 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);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Usersearch baseDN: " + userbaseDN);
|
||||
}
|
||||
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);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Groupsearch baseDN: " + userbaseDN);
|
||||
}
|
||||
LOG.debug("Groupsearch baseDN: {}", groupbaseDN);
|
||||
|
||||
groupSearchFilter =
|
||||
conf.get(GROUP_SEARCH_FILTER_KEY, GROUP_SEARCH_FILTER_DEFAULT);
|
||||
|
@ -655,7 +700,8 @@ 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.
|
||||
|
@ -669,9 +715,26 @@ 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,
|
||||
|
@ -721,8 +784,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;
|
||||
}
|
||||
|
|
|
@ -224,12 +224,36 @@
|
|||
</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 to use for resolving user groups when using
|
||||
the LdapGroupsMapping user to group mapping.
|
||||
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>
|
||||
|
||||
|
|
|
@ -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 for resolving user groups.
|
||||
`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.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,6 +115,58 @@ 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`.
|
||||
|
|
|
@ -18,11 +18,12 @@
|
|||
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;
|
||||
|
@ -82,14 +83,16 @@ 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() throws NamingException {
|
||||
public void setupMocks() {
|
||||
when(getUserSearchResult().getNameInNamespace()).
|
||||
thenReturn(userDN);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetGroups() throws IOException, NamingException {
|
||||
public void testGetGroups() throws 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),
|
||||
|
@ -100,9 +103,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
|
||||
@Test
|
||||
public void testGetGroupsWithDifferentBaseDNs() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
// Set this, so we don't throw an exception
|
||||
conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
|
||||
Configuration conf = getBaseConf(TEST_LDAP_URL);
|
||||
String userBaseDN = "ou=Users,dc=xxx,dc=com ";
|
||||
String groupBaseDN = " ou=Groups,dc=xxx,dc=com";
|
||||
conf.set(LdapGroupsMapping.USER_BASE_DN_KEY, userBaseDN);
|
||||
|
@ -113,9 +114,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
|
||||
@Test
|
||||
public void testGetGroupsWithDefaultBaseDN() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
// Set this, so we don't throw an exception
|
||||
conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
|
||||
Configuration conf = getBaseConf(TEST_LDAP_URL);
|
||||
String baseDN = " dc=xxx,dc=com ";
|
||||
conf.set(LdapGroupsMapping.BASE_DN_KEY, baseDN);
|
||||
doTestGetGroupsWithBaseDN(conf, baseDN.trim(), baseDN.trim());
|
||||
|
@ -159,7 +158,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetGroupsWithHierarchy() throws IOException, NamingException {
|
||||
public void testGetGroupsWithHierarchy() throws 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
|
||||
|
@ -175,7 +174,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetGroupsWithConnectionClosed() throws IOException, NamingException {
|
||||
public void testGetGroupsWithConnectionClosed() throws 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),
|
||||
|
@ -190,21 +189,20 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetGroupsWithLdapDown() throws IOException, NamingException {
|
||||
public void testGetGroupsWithLdapDown() throws 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[] {}),
|
||||
LdapGroupsMapping.RECONNECT_RETRY_COUNT);
|
||||
doTestGetGroups(Arrays.asList(new String[] {}), 4);
|
||||
}
|
||||
|
||||
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");
|
||||
private void doTestGetGroups(List<String> expectedGroups, int searchTimes)
|
||||
throws NamingException {
|
||||
Configuration conf = getBaseConf(TEST_LDAP_URL);
|
||||
conf.setInt(LDAP_NUM_ATTEMPTS_KEY, searchTimes);
|
||||
|
||||
LdapGroupsMapping groupsMapping = getGroupsMapping();
|
||||
groupsMapping.setConf(conf);
|
||||
|
@ -216,17 +214,12 @@ 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 IOException, NamingException {
|
||||
Configuration conf = new Configuration();
|
||||
// Set this, so we don't throw an exception
|
||||
conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
|
||||
int searchTimesGroup, int searchTimesParentGroup) throws NamingException {
|
||||
Configuration conf = getBaseConf(TEST_LDAP_URL);
|
||||
// Set the config to get parents 1 level up
|
||||
conf.setInt(LdapGroupsMapping.GROUP_HIERARCHY_LEVELS_KEY, 1);
|
||||
|
||||
|
@ -237,18 +230,14 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
List<String> groups = groupsMapping.getGroups("some_user");
|
||||
|
||||
// compare lists, ignoring the order
|
||||
Assert.assertEquals(new HashSet<String>(expectedGroups),
|
||||
new HashSet<String>(groups));
|
||||
Assert.assertEquals(new HashSet<>(expectedGroups), new HashSet<>(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
|
||||
|
@ -268,7 +257,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
@Test
|
||||
public void testConfGetPassword() throws Exception {
|
||||
File testDir = GenericTestUtils.getTestDir();
|
||||
Configuration conf = new Configuration();
|
||||
Configuration conf = getBaseConf();
|
||||
final Path jksPath = new Path(testDir.toString(), "test.jks");
|
||||
final String ourUrl =
|
||||
JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
|
||||
|
@ -283,10 +272,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
|
||||
assertEquals(null, provider.getCredentialEntry(
|
||||
assertNull(provider.getCredentialEntry(
|
||||
LdapGroupsMapping.BIND_PASSWORD_KEY));
|
||||
assertEquals(null, provider.getCredentialEntry
|
||||
(LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY));
|
||||
assertNull(provider.getCredentialEntry(
|
||||
LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY));
|
||||
|
||||
// create new aliases
|
||||
try {
|
||||
|
@ -350,9 +339,8 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
ldapServer.start();
|
||||
|
||||
final LdapGroupsMapping mapping = new LdapGroupsMapping();
|
||||
final Configuration conf = new Configuration();
|
||||
conf.set(LdapGroupsMapping.LDAP_URL_KEY,
|
||||
"ldap://localhost:" + serverSock.getLocalPort());
|
||||
String ldapUrl = "ldap://localhost:" + serverSock.getLocalPort();
|
||||
final Configuration conf = getBaseConf(ldapUrl, null);
|
||||
conf.setInt(CONNECTION_TIMEOUT, connectionTimeoutMs);
|
||||
mapping.setConf(conf);
|
||||
|
||||
|
@ -406,9 +394,8 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
ldapServer.start();
|
||||
|
||||
final LdapGroupsMapping mapping = new LdapGroupsMapping();
|
||||
final Configuration conf = new Configuration();
|
||||
conf.set(LdapGroupsMapping.LDAP_URL_KEY,
|
||||
"ldap://localhost:" + serverSock.getLocalPort());
|
||||
String ldapUrl = "ldap://localhost:" + serverSock.getLocalPort();
|
||||
final Configuration conf = getBaseConf(ldapUrl, null);
|
||||
conf.setInt(READ_TIMEOUT, readTimeoutMs);
|
||||
mapping.setConf(conf);
|
||||
|
||||
|
@ -436,15 +423,12 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
|
|||
*/
|
||||
@Test(timeout = 10000)
|
||||
public void testSetConf() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
Configuration conf = getBaseConf(TEST_LDAP_URL);
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -18,10 +18,15 @@
|
|||
|
||||
package org.apache.hadoop.security;
|
||||
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
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.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import javax.naming.Context;
|
||||
import javax.naming.NamingEnumeration;
|
||||
import javax.naming.NamingException;
|
||||
import javax.naming.directory.Attribute;
|
||||
|
@ -31,13 +36,17 @@ 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;
|
||||
|
@ -62,7 +71,6 @@ 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))).
|
||||
|
@ -117,9 +125,30 @@ 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;
|
||||
}
|
||||
|
@ -139,13 +168,56 @@ 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,0 +1,142 @@
|
|||
/**
|
||||
* 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));
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.security;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -68,7 +67,7 @@ public class TestLdapGroupsMappingWithOneQuery
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetGroups() throws IOException, NamingException {
|
||||
public void testGetGroups() throws 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"};
|
||||
|
@ -76,10 +75,9 @@ public class TestLdapGroupsMappingWithOneQuery
|
|||
}
|
||||
|
||||
private void doTestGetGroups(List<String> expectedGroups)
|
||||
throws IOException, NamingException {
|
||||
Configuration conf = new Configuration();
|
||||
// Set this, so we don't throw an exception
|
||||
conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
|
||||
throws NamingException {
|
||||
String ldapUrl = "ldap://test";
|
||||
Configuration conf = getBaseConf(ldapUrl);
|
||||
// enable single-query lookup
|
||||
conf.set(LdapGroupsMapping.MEMBEROF_ATTR_KEY, "memberOf");
|
||||
|
||||
|
@ -92,9 +90,7 @@ 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));
|
||||
}
|
||||
}
|
|
@ -28,7 +28,6 @@ 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;
|
||||
|
||||
|
@ -62,7 +61,7 @@ public class TestLdapGroupsMappingWithPosixGroup
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetGroups() throws IOException, NamingException {
|
||||
public void testGetGroups() throws 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"),
|
||||
|
@ -73,10 +72,9 @@ public class TestLdapGroupsMappingWithPosixGroup
|
|||
}
|
||||
|
||||
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");
|
||||
throws NamingException {
|
||||
String ldapUrl = "ldap://test";
|
||||
Configuration conf = getBaseConf(ldapUrl);
|
||||
conf.set(LdapGroupsMapping.GROUP_SEARCH_FILTER_KEY,
|
||||
"(objectClass=posixGroup)(cn={0})");
|
||||
conf.set(LdapGroupsMapping.USER_SEARCH_FILTER_KEY,
|
||||
|
|
Loading…
Reference in New Issue