Migrate authentication calls to be asynchronous

This commit migrates the actual authentication calls that are performed by internal realms
to be asynchronous so that we do not execute blocking calls on network threads. The realms
that used LDAP were especially bad as they can issue many different requests and/or open
multiple connections per authentication attempt.

The LdapRealm now uses the ThreadPool to fork a thread for authentication. This is necessary
because a LDAP bind operation is not allowed to be performed asynchronously and must be done
in a blocking manner. After the bind request has completed, all other requests can be done
asynchrnously. The LdapUtils class now provides asynchronous methods for searching and also
includes the handling of referral following as automated following is not supported by
UnboundID when using the asynchronous calls.

Original commit: elastic/x-pack-elasticsearch@dfb259e498
This commit is contained in:
Jay Modi 2016-12-05 10:07:31 -05:00 committed by GitHub
parent 549629b1aa
commit ffeaea0663
53 changed files with 1555 additions and 1236 deletions

View File

@ -5,6 +5,7 @@
*/
package org.elasticsearch.xpack;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse;
@ -231,8 +232,12 @@ public class XPackPlugin extends Plugin implements ScriptPlugin, ActionPlugin, I
components.add(licenseService);
components.add(licenseState);
components.addAll(security.createComponents(internalClient, threadPool, clusterService, resourceWatcherService,
extensionsService.getExtensions()));
try {
components.addAll(security.createComponents(internalClient, threadPool, clusterService, resourceWatcherService,
extensionsService.getExtensions()));
} catch (Exception e) {
throw new Error("security initialization failed", e);
}
components.addAll(monitoring.createComponents(internalClient, threadPool, clusterService, licenseService, sslService));
// watcher http stuff

View File

@ -81,7 +81,6 @@ import org.elasticsearch.xpack.security.authc.AuthenticationService;
import org.elasticsearch.xpack.security.authc.DefaultAuthenticationFailureHandler;
import org.elasticsearch.xpack.security.authc.Realm;
import org.elasticsearch.xpack.security.authc.Realms;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectoryRealm;
import org.elasticsearch.xpack.security.authc.esnative.NativeRealm;
import org.elasticsearch.xpack.security.authc.esnative.NativeUsersStore;
import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm;
@ -232,7 +231,8 @@ public class Security implements ActionPlugin, IngestPlugin, NetworkPlugin {
}
public Collection<Object> createComponents(InternalClient client, ThreadPool threadPool, ClusterService clusterService,
ResourceWatcherService resourceWatcherService, List<XPackExtension> extensions) {
ResourceWatcherService resourceWatcherService,
List<XPackExtension> extensions) throws Exception {
if (enabled == false) {
return Collections.emptyList();
}
@ -248,9 +248,10 @@ public class Security implements ActionPlugin, IngestPlugin, NetworkPlugin {
Map<String, Realm.Factory> realmFactories = new HashMap<>();
realmFactories.put(FileRealm.TYPE, config -> new FileRealm(config, resourceWatcherService));
realmFactories.put(NativeRealm.TYPE, config -> new NativeRealm(config, nativeUsersStore));
realmFactories.put(ActiveDirectoryRealm.TYPE,
config -> new ActiveDirectoryRealm(config, resourceWatcherService, sslService));
realmFactories.put(LdapRealm.TYPE, config -> new LdapRealm(config, resourceWatcherService, sslService));
realmFactories.put(LdapRealm.AD_TYPE,
config -> new LdapRealm(LdapRealm.AD_TYPE, config, resourceWatcherService, sslService, threadPool));
realmFactories.put(LdapRealm.LDAP_TYPE,
config -> new LdapRealm(LdapRealm.LDAP_TYPE, config, resourceWatcherService, sslService, threadPool));
realmFactories.put(PkiRealm.TYPE, config -> new PkiRealm(config, resourceWatcherService, sslService));
for (XPackExtension extension : extensions) {
Map<String, Realm.Factory> newRealms = extension.getRealms(resourceWatcherService);

View File

@ -138,7 +138,8 @@ public abstract class Realm implements Comparable<Realm> {
/**
* Constructs a realm which will be used for authentication.
* @param config The configuration for the realm
* @throws Exception an exception may be thrown if there was an error during realm creation
*/
Realm create(RealmConfig config);
Realm create(RealmConfig config) throws Exception;
}
}

View File

@ -16,16 +16,14 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.license.XPackLicenseState.AllowedRealmType;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectoryRealm;
import org.elasticsearch.xpack.security.authc.esnative.NativeRealm;
import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm;
import org.elasticsearch.xpack.security.authc.file.FileRealm;
@ -37,10 +35,10 @@ import static org.elasticsearch.xpack.security.Security.setting;
/**
* Serves as a realms registry (also responsible for ordering the realms appropriately)
*/
public class Realms extends AbstractLifecycleComponent implements Iterable<Realm> {
public class Realms extends AbstractComponent implements Iterable<Realm> {
static final List<String> INTERNAL_REALM_TYPES =
Arrays.asList(ReservedRealm.TYPE, NativeRealm.TYPE, FileRealm.TYPE, ActiveDirectoryRealm.TYPE, LdapRealm.TYPE, PkiRealm.TYPE);
Arrays.asList(ReservedRealm.TYPE, NativeRealm.TYPE, FileRealm.TYPE, LdapRealm.AD_TYPE, LdapRealm.LDAP_TYPE, PkiRealm.TYPE);
public static final Setting<Settings> REALMS_GROUPS_SETTINGS = Setting.groupSetting(setting("authc.realms."), Property.NodeScope);
@ -51,21 +49,17 @@ public class Realms extends AbstractLifecycleComponent implements Iterable<Realm
protected List<Realm> realms = Collections.emptyList();
// a list of realms that are considered default in that they are provided by x-pack and not a third party
protected List<Realm> internalRealmsOnly = Collections.emptyList();
List<Realm> internalRealmsOnly = Collections.emptyList();
// a list of realms that are considered native, that is they only interact with x-pack and no 3rd party auth sources
protected List<Realm> nativeRealmsOnly = Collections.emptyList();
List<Realm> nativeRealmsOnly = Collections.emptyList();
public Realms(Settings settings, Environment env, Map<String, Realm.Factory> factories, XPackLicenseState licenseState,
ReservedRealm reservedRealm) {
ReservedRealm reservedRealm) throws Exception {
super(settings);
this.env = env;
this.factories = factories;
this.licenseState = licenseState;
this.reservedRealm = reservedRealm;
}
@Override
protected void doStart() throws ElasticsearchException {
assert factories.get(ReservedRealm.TYPE) == null;
this.realms = initRealms();
// pre-computing a list of internal only realms allows us to have much cheaper iteration than a custom iterator
@ -97,14 +91,6 @@ public class Realms extends AbstractLifecycleComponent implements Iterable<Realm
this.nativeRealmsOnly = Collections.unmodifiableList(nativeRealms);
}
@Override
protected void doStop() throws ElasticsearchException {
}
@Override
protected void doClose() throws ElasticsearchException {
}
@Override
public Iterator<Realm> iterator() {
if (licenseState.isAuthAllowed() == false) {
@ -155,7 +141,7 @@ public class Realms extends AbstractLifecycleComponent implements Iterable<Realm
return factories.get(type);
}
protected List<Realm> initRealms() {
protected List<Realm> initRealms() throws Exception {
Settings realmsSettings = REALMS_GROUPS_SETTINGS.get(settings);
Set<String> internalTypes = new HashSet<>();
List<Realm> realms = new ArrayList<>();
@ -241,7 +227,7 @@ public class Realms extends AbstractLifecycleComponent implements Iterable<Realm
return realmMap;
}
private void addNativeRealms(List<Realm> realms) {
private void addNativeRealms(List<Realm> realms) throws Exception {
Realm.Factory fileRealm = factories.get(FileRealm.TYPE);
if (fileRealm != null) {

View File

@ -1,141 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.Filter;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResult;
import com.unboundid.ldap.sdk.SearchResultEntry;
import com.unboundid.ldap.sdk.SearchScope;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.OBJECT_CLASS_PRESENCE_FILTER;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.search;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.searchForEntry;
public class ActiveDirectoryGroupsResolver implements GroupsResolver {
private final String baseDn;
private final LdapSearchScope scope;
public ActiveDirectoryGroupsResolver(Settings settings, String baseDnDefault) {
this.baseDn = settings.get("base_dn", baseDnDefault);
this.scope = LdapSearchScope.resolve(settings.get("scope"), LdapSearchScope.SUB_TREE);
}
@Override
public List<String> resolve(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger,
Collection<Attribute> attributes) {
Filter groupSearchFilter = buildGroupQuery(connection, userDn, timeout, logger);
logger.debug("group SID to DN search filter: [{}]", groupSearchFilter);
if (groupSearchFilter == null) {
return Collections.emptyList();
}
SearchRequest searchRequest = new SearchRequest(baseDn, scope.scope(), groupSearchFilter, SearchRequest.NO_ATTRIBUTES);
searchRequest.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResult results;
try {
results = search(connection, searchRequest, logger);
} catch (LDAPException e) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("failed to fetch AD groups for DN [{}]", userDn), e);
return Collections.emptyList();
}
List<String> groupList = new ArrayList<>();
for (SearchResultEntry entry : results.getSearchEntries()) {
groupList.add(entry.getDN());
}
if (logger.isDebugEnabled()) {
logger.debug("found these groups [{}] for userDN [{}]", groupList, userDn);
}
return groupList;
}
@Override
public String[] attributes() {
// we have to return null since the tokenGroups attribute is computed and can only be retrieved using a BASE level search
return null;
}
static Filter buildGroupQuery(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger) {
try {
SearchRequest request = new SearchRequest(userDn, SearchScope.BASE, OBJECT_CLASS_PRESENCE_FILTER, "tokenGroups");
request.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResultEntry entry = searchForEntry(connection, request, logger);
if (entry == null) {
return null;
}
Attribute attribute = entry.getAttribute("tokenGroups");
byte[][] tokenGroupSIDBytes = attribute.getValueByteArrays();
List<Filter> orFilters = new ArrayList<>(tokenGroupSIDBytes.length);
for (byte[] SID : tokenGroupSIDBytes) {
orFilters.add(Filter.createEqualityFilter("objectSid", binarySidToStringSid(SID)));
}
return Filter.createORFilter(orFilters);
} catch (LDAPException e) {
logger.error((Supplier<?>) () -> new ParameterizedMessage("failed to fetch AD groups for DN [{}]", userDn), e);
return null;
}
}
/**
* To better understand what the sid is and how its string representation looks like, see
* http://blogs.msdn.com/b/alextch/archive/2007/06/18/sample-java-application-that-retrieves-group-membership-of-an-active-directory
* -user-account.aspx
*
* @param SID byte encoded security ID
*/
public static String binarySidToStringSid(byte[] SID) {
String strSID;
//convert the SID into string format
long version;
long authority;
long count;
long rid;
strSID = "S";
version = SID[0];
strSID = strSID + "-" + Long.toString(version);
authority = SID[4];
for (int i = 0; i < 4; i++) {
authority <<= 8;
authority += SID[4 + i] & 0xFF;
}
strSID = strSID + "-" + Long.toString(authority);
count = SID[2];
count <<= 8;
count += SID[1] & 0xFF;
for (int j = 0; j < count; j++) {
rid = SID[11 + (j * 4)] & 0xFF;
for (int k = 1; k < 4; k++) {
rid <<= 8;
rid += SID[11 - k + (j * 4)] & 0xFF;
}
strSID = strSID + "-" + Long.toString(rid);
}
return strSID;
}
}

View File

@ -1,28 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.AbstractLdapRealm;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.DnRoleMapper;
import org.elasticsearch.xpack.ssl.SSLService;
public class ActiveDirectoryRealm extends AbstractLdapRealm {
public static final String TYPE = "active_directory";
public ActiveDirectoryRealm(RealmConfig config, ResourceWatcherService watcherService, SSLService sslService) {
this(config, new ActiveDirectorySessionFactory(config, sslService),
new DnRoleMapper(TYPE, config, watcherService, null));
}
// pkg private for tests
ActiveDirectoryRealm(RealmConfig config, SessionFactory sessionFactory, DnRoleMapper roleMapper) {
super(TYPE, config, sessionFactory, roleMapper);
}
}

View File

@ -1,288 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.LDAPConnection;
import com.unboundid.ldap.sdk.LDAPConnectionOptions;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResult;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.cache.Cache;
import org.elasticsearch.common.cache.CacheBuilder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ssl.SSLService;
import java.util.concurrent.ExecutionException;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.attributesToSearchFor;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.createFilter;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.search;
/**
* This Class creates LdapSessions authenticating via the custom Active Directory protocol. (that being
* authenticating with a principal name, "username@domain", then searching through the directory to find the
* user entry in Active Directory that matches the user name). This eliminates the need for user templates, and simplifies
* the configuration for windows admins that may not be familiar with LDAP concepts.
*/
public class ActiveDirectorySessionFactory extends SessionFactory {
public static final String AD_DOMAIN_NAME_SETTING = "domain_name";
public static final String AD_GROUP_SEARCH_BASEDN_SETTING = "group_search.base_dn";
public static final String AD_GROUP_SEARCH_SCOPE_SETTING = "group_search.scope";
public static final String AD_USER_SEARCH_BASEDN_SETTING = "user_search.base_dn";
public static final String AD_USER_SEARCH_FILTER_SETTING = "user_search.filter";
public static final String AD_USER_SEARCH_SCOPE_SETTING = "user_search.scope";
private static final String NETBIOS_NAME_FILTER_TEMPLATE = "(netbiosname={0})";
private final String domainName;
private final GroupsResolver groupResolver;
private final DefaultADAuthenticator defaultADAuthenticator;
private final DownLevelADAuthenticator downLevelADAuthenticator;
private final UpnADAuthenticator upnADAuthenticator;
public ActiveDirectorySessionFactory(RealmConfig config, SSLService sslService) {
super(config, sslService);
Settings settings = config.settings();
domainName = settings.get(AD_DOMAIN_NAME_SETTING);
if (domainName == null) {
throw new IllegalArgumentException("missing [" + AD_DOMAIN_NAME_SETTING + "] setting for active directory");
}
String domainDN = buildDnFromDomain(domainName);
groupResolver = new ActiveDirectoryGroupsResolver(settings.getAsSettings("group_search"), domainDN);
defaultADAuthenticator = new DefaultADAuthenticator(settings, timeout, logger, groupResolver, domainDN);
downLevelADAuthenticator = new DownLevelADAuthenticator(settings, timeout, logger, groupResolver, domainDN);
upnADAuthenticator = new UpnADAuthenticator(settings, timeout, logger, groupResolver, domainDN);
}
@Override
protected String[] getDefaultLdapUrls(Settings settings) {
return new String[] {"ldap://" + settings.get(AD_DOMAIN_NAME_SETTING) + ":389"};
}
/**
* This is an active directory bind that looks up the user DN after binding with a windows principal.
*
* @param username name of the windows user without the domain
* @return An authenticated LdapSession
*/
@Override
protected LdapSession getSession(String username, SecuredString password) throws Exception {
LDAPConnection connection = serverSet.getConnection();
ADAuthenticator authenticator = getADAuthenticator(username);
return authenticator.authenticate(connection, username, password);
}
/**
* @param domain active directory domain name
* @return LDAP DN, distinguished name, of the root of the domain
*/
static String buildDnFromDomain(String domain) {
return "DC=" + domain.replace(".", ",DC=");
}
ADAuthenticator getADAuthenticator(String username) {
if (username.indexOf('\\') > 0) {
return downLevelADAuthenticator;
} else if (username.indexOf("@") > 0) {
return upnADAuthenticator;
}
return defaultADAuthenticator;
}
abstract static class ADAuthenticator {
final TimeValue timeout;
final Logger logger;
final GroupsResolver groupsResolver;
final String userSearchDN;
final LdapSearchScope userSearchScope;
ADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
this.timeout = timeout;
this.logger = logger;
this.groupsResolver = groupsResolver;
userSearchDN = settings.get(AD_USER_SEARCH_BASEDN_SETTING, domainDN);
userSearchScope = LdapSearchScope.resolve(settings.get(AD_USER_SEARCH_SCOPE_SETTING), LdapSearchScope.SUB_TREE);
}
LdapSession authenticate(LDAPConnection connection, String username, SecuredString password) throws LDAPException {
boolean success = false;
try {
connection.bind(bindUsername(username), new String(password.internalChars()));
SearchRequest searchRequest = getSearchRequest(connection, username, password);
searchRequest.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResult results = search(connection, searchRequest, logger);
int numResults = results.getEntryCount();
if (numResults > 1) {
throw new IllegalStateException("search for user [" + username + "] by principle name yielded multiple results");
} else if (numResults < 1) {
throw new IllegalStateException("search for user [" + username + "] by principle name yielded no results");
}
String dn = results.getSearchEntries().get(0).getDN();
LdapSession session = new LdapSession(logger, connection, dn, groupsResolver, timeout, null);
success = true;
return session;
} finally {
if (success == false) {
connection.close();
}
}
}
String bindUsername(String username) {
return username;
}
abstract SearchRequest getSearchRequest(LDAPConnection connection, String username, SecuredString password) throws LDAPException;
}
/**
* This authenticator is used for usernames that do not contain an `@` or `/`. It attempts a bind with the provided username combined
* with the domain name specified in settings. On AD DS this will work for both upn@domain and samaccountname@domain; AD LDS will only
* support the upn format
*/
static class DefaultADAuthenticator extends ADAuthenticator {
final String userSearchFilter;
final String domainName;
DefaultADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
super(settings, timeout, logger, groupsResolver, domainDN);
domainName = settings.get(AD_DOMAIN_NAME_SETTING);
userSearchFilter = settings.get(AD_USER_SEARCH_FILTER_SETTING, "(&(objectClass=user)(|(sAMAccountName={0})" +
"(userPrincipalName={0}@" + domainName + ")))");
}
@Override
SearchRequest getSearchRequest(LDAPConnection connection, String username, SecuredString password) throws LDAPException {
return new SearchRequest(userSearchDN, userSearchScope.scope(),
createFilter(userSearchFilter, username), attributesToSearchFor(groupsResolver.attributes()));
}
@Override
String bindUsername(String username) {
return username + "@" + domainName;
}
}
/**
* Active Directory calls the format <code>DOMAIN\\username</code> down-level credentials and this class contains the logic necessary
* to authenticate this form of a username
*/
static class DownLevelADAuthenticator extends ADAuthenticator {
Cache<String, String> domainNameCache = CacheBuilder.<String, String>builder().setMaximumWeight(100).build();
final String domainDN;
final Settings settings;
DownLevelADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
super(settings, timeout, logger, groupsResolver, domainDN);
this.domainDN = domainDN;
this.settings = settings;
}
SearchRequest getSearchRequest(LDAPConnection connection, String username, SecuredString password) throws LDAPException {
String[] parts = username.split("\\\\");
assert parts.length == 2;
final String netBiosDomainName = parts[0];
final String accountName = parts[1];
final String domainDn = netBiosDomainNameToDn(connection, netBiosDomainName, username, password);
return new SearchRequest(domainDn, LdapSearchScope.SUB_TREE.scope(),
createFilter("(&(objectClass=user)(sAMAccountName={0}))", accountName),
attributesToSearchFor(groupsResolver.attributes()));
}
String netBiosDomainNameToDn(LDAPConnection connection, String netBiosDomainName, String username, SecuredString password)
throws LDAPException {
try {
return domainNameCache.computeIfAbsent(netBiosDomainName, (key) -> {
LDAPConnection searchConnection = connection;
boolean openedConnection = false;
try {
// global catalog does not replicate the necessary information by default
// TODO add settings for ports and maybe cache connectionOptions
if (usingGlobalCatalog(settings, connection)) {
LDAPConnectionOptions options = connectionOptions(settings);
if (connection.getSSLSession() != null) {
searchConnection = new LDAPConnection(connection.getSocketFactory(), options,
connection.getConnectedAddress(), 636);
} else {
searchConnection = new LDAPConnection(options, connection.getConnectedAddress(), 389);
}
openedConnection = true;
searchConnection.bind(username, new String(password.internalChars()));
}
SearchRequest searchRequest = new SearchRequest(domainDN, LdapSearchScope.SUB_TREE.scope(),
createFilter(NETBIOS_NAME_FILTER_TEMPLATE, netBiosDomainName), "ncname");
SearchResult results = search(searchConnection, searchRequest, logger);
if (results.getEntryCount() > 0) {
Attribute attribute = results.getSearchEntries().get(0).getAttribute("ncname");
if (attribute != null) {
return attribute.getValue();
}
}
logger.debug("failed to find domain name DN from netbios name [{}]", netBiosDomainName);
return null;
} finally {
if (openedConnection) {
searchConnection.close();
}
}
});
} catch (ExecutionException e) {
Throwable cause = e.getCause();
if (cause instanceof LDAPException) {
throw (LDAPException) cause;
} else {
connection.close();
throw new ElasticsearchException("error occurred while mapping [{}] to domain DN", cause, netBiosDomainName);
}
}
}
static boolean usingGlobalCatalog(Settings settings, LDAPConnection ldapConnection) {
Boolean usingGlobalCatalog = settings.getAsBoolean("global_catalog", null);
if (usingGlobalCatalog != null) {
return usingGlobalCatalog;
}
return ldapConnection.getConnectedPort() == 3268 || ldapConnection.getConnectedPort() == 3269;
}
}
static class UpnADAuthenticator extends ADAuthenticator {
private static final String UPN_USER_FILTER = "(&(objectClass=user)(|(sAMAccountName={0})(userPrincipalName={1})))";
UpnADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
super(settings, timeout, logger, groupsResolver, domainDN);
}
SearchRequest getSearchRequest(LDAPConnection connection, String username, SecuredString password) throws LDAPException {
String[] parts = username.split("@");
assert parts.length == 2;
final String accountName = parts[0];
final String domainName = parts[1];
final String domainDN = buildDnFromDomain(domainName);
return new SearchRequest(domainDN, LdapSearchScope.SUB_TREE.scope(),
createFilter(UPN_USER_FILTER, accountName, username), attributesToSearchFor(groupsResolver.attributes()));
}
}
}

View File

@ -12,7 +12,6 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.DocWriteResponse.Result;
import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetRequest;
@ -61,8 +60,6 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
@ -168,38 +165,6 @@ public class NativeUsersStore extends AbstractComponent implements ClusterStateL
}
}
/**
* Blocking method to get the user and their password hash
*/
private UserAndPassword getUserAndPassword(final String username) {
final AtomicReference<UserAndPassword> userRef = new AtomicReference<>(null);
final CountDownLatch latch = new CountDownLatch(1);
getUserAndPassword(username, new LatchedActionListener<>(new ActionListener<UserAndPassword>() {
@Override
public void onResponse(UserAndPassword user) {
userRef.set(user);
}
@Override
public void onFailure(Exception t) {
if (t instanceof IndexNotFoundException) {
logger.trace(
(Supplier<?>) () -> new ParameterizedMessage(
"failed to retrieve user [{}] since security index does not exist", username), t);
} else {
logger.error((Supplier<?>) () -> new ParameterizedMessage("failed to retrieve user [{}]", username), t);
}
}
}, latch));
try {
latch.await(30, TimeUnit.SECONDS);
} catch (InterruptedException e) {
logger.error("timed out retrieving user [{}]", username);
return null;
}
return userRef.get();
}
/**
* Async method to retrieve a user and their password
*/

View File

@ -11,7 +11,6 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.CachingUsernamePasswordRealm;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken;
import org.elasticsearch.xpack.security.user.User;
@ -29,11 +28,10 @@ public class FileRealm extends CachingUsernamePasswordRealm {
// pkg private for testing
FileRealm(RealmConfig config, FileUserPasswdStore userPasswdStore, FileUserRolesStore userRolesStore) {
super(TYPE, config);
Listener listener = new Listener();
this.userPasswdStore = userPasswdStore;
userPasswdStore.addListener(listener);
userPasswdStore.addListener(this::expireAll);
this.userRolesStore = userRolesStore;
userRolesStore.addListener(listener);
userRolesStore.addListener(this::expireAll);
}
@Override
@ -68,11 +66,4 @@ public class FileRealm extends CachingUsernamePasswordRealm {
public boolean userLookupSupported() {
return true;
}
class Listener implements RefreshListener {
@Override
public void onRefresh() {
expireAll();
}
}
}

View File

@ -19,7 +19,6 @@ import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.XPackSettings;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.Hasher;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.security.support.NoOpLogger;
import org.elasticsearch.xpack.security.support.Validation;
@ -30,6 +29,7 @@ import java.io.PrintWriter;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
@ -47,20 +47,20 @@ public class FileUserPasswdStore {
private final Path file;
private final Hasher hasher = Hasher.BCRYPT;
private final Settings settings;
private final CopyOnWriteArrayList<Runnable> listeners;
private volatile Map<String, char[]> users;
private CopyOnWriteArrayList<RefreshListener> listeners;
public FileUserPasswdStore(RealmConfig config, ResourceWatcherService watcherService) {
this(config, watcherService, null);
this(config, watcherService, () -> {});
}
FileUserPasswdStore(RealmConfig config, ResourceWatcherService watcherService, RefreshListener listener) {
FileUserPasswdStore(RealmConfig config, ResourceWatcherService watcherService, Runnable listener) {
logger = config.logger(FileUserPasswdStore.class);
file = resolveFile(config.env());
settings = config.globalSettings();
users = parseFileLenient(file, logger, settings);
listeners = new CopyOnWriteArrayList<>(Collections.singletonList(listener));
FileWatcher watcher = new FileWatcher(file.getParent());
watcher.addListener(new FileListener());
try {
@ -68,14 +68,9 @@ public class FileUserPasswdStore {
} catch (IOException e) {
throw new ElasticsearchException("failed to start watching users file [{}]", e, file.toAbsolutePath());
}
listeners = new CopyOnWriteArrayList<>();
if (listener != null) {
listeners.add(listener);
}
}
public void addListener(RefreshListener listener) {
public void addListener(Runnable listener) {
listeners.add(listener);
}
@ -175,10 +170,8 @@ public class FileUserPasswdStore {
}
}
public void notifyRefresh() {
for (RefreshListener listener : listeners) {
listener.onRefresh();
}
void notifyRefresh() {
listeners.forEach(Runnable::run);
}
private class FileListener implements FileChangesListener {

View File

@ -17,7 +17,6 @@ import org.elasticsearch.watcher.FileWatcher;
import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.support.NoOpLogger;
import org.elasticsearch.xpack.security.support.Validation;
@ -27,6 +26,7 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
@ -45,17 +45,18 @@ public class FileUserRolesStore {
private final Logger logger;
private final Path file;
private CopyOnWriteArrayList<RefreshListener> listeners;
private final CopyOnWriteArrayList<Runnable> listeners;
private volatile Map<String, String[]> userRoles;
public FileUserRolesStore(RealmConfig config, ResourceWatcherService watcherService) {
this(config, watcherService, null);
FileUserRolesStore(RealmConfig config, ResourceWatcherService watcherService) {
this(config, watcherService, () -> {});
}
FileUserRolesStore(RealmConfig config, ResourceWatcherService watcherService, RefreshListener listener) {
FileUserRolesStore(RealmConfig config, ResourceWatcherService watcherService, Runnable listener) {
logger = config.logger(FileUserRolesStore.class);
file = resolveFile(config.env());
userRoles = parseFileLenient(file, logger);
listeners = new CopyOnWriteArrayList<>(Collections.singletonList(listener));
FileWatcher watcher = new FileWatcher(file.getParent());
watcher.addListener(new FileListener());
try {
@ -63,14 +64,9 @@ public class FileUserRolesStore {
} catch (IOException e) {
throw new ElasticsearchException("failed to start watching the user roles file [" + file.toAbsolutePath() + "]", e);
}
listeners = new CopyOnWriteArrayList<>();
if (listener != null) {
listeners.add(listener);
}
}
public synchronized void addListener(RefreshListener listener) {
public void addListener(Runnable listener) {
listeners.add(listener);
}
@ -207,10 +203,8 @@ public class FileUserRolesStore {
}
}
public void notifyRefresh() {
for (RefreshListener listener : listeners) {
listener.onRefresh();
}
void notifyRefresh() {
listeners.forEach(Runnable::run);
}
private class FileListener implements FileChangesListener {

View File

@ -0,0 +1,128 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.Filter;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResultEntry;
import com.unboundid.ldap.sdk.SearchScope;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.OBJECT_CLASS_PRESENCE_FILTER;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.search;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.searchForEntry;
class ActiveDirectoryGroupsResolver implements GroupsResolver {
private static final String TOKEN_GROUPS = "tokenGroups";
private final String baseDn;
private final LdapSearchScope scope;
ActiveDirectoryGroupsResolver(Settings settings, String baseDnDefault) {
this.baseDn = settings.get("base_dn", baseDnDefault);
this.scope = LdapSearchScope.resolve(settings.get("scope"), LdapSearchScope.SUB_TREE);
}
@Override
public void resolve(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger, Collection<Attribute> attributes,
ActionListener<List<String>> listener) {
buildGroupQuery(connection, userDn, timeout,
ActionListener.wrap((filter) -> {
if (filter == null) {
listener.onResponse(Collections.emptyList());
} else {
logger.debug("group SID to DN [{}] search filter: [{}]", userDn, filter);
search(connection, baseDn, scope.scope(), filter, Math.toIntExact(timeout.seconds()),
ActionListener.wrap((results) -> {
List<String> groups = results.stream()
.map(SearchResultEntry::getDN)
.collect(Collectors.toList());
listener.onResponse(Collections.unmodifiableList(groups));
},
listener::onFailure),
SearchRequest.NO_ATTRIBUTES);
}
}, listener::onFailure));
}
@Override
public String[] attributes() {
// we have to return null since the tokenGroups attribute is computed and can only be retrieved using a BASE level search
return null;
}
static void buildGroupQuery(LDAPInterface connection, String userDn, TimeValue timeout, ActionListener<Filter> listener) {
searchForEntry(connection, userDn, SearchScope.BASE, OBJECT_CLASS_PRESENCE_FILTER, Math.toIntExact(timeout.seconds()),
ActionListener.wrap((entry) -> {
if (entry == null || entry.hasAttribute(TOKEN_GROUPS) == false) {
listener.onResponse(null);
} else {
final byte[][] tokenGroupSIDBytes = entry.getAttributeValueByteArrays(TOKEN_GROUPS);
List<Filter> orFilters = Arrays.stream(tokenGroupSIDBytes)
.map((sidBytes) -> Filter.createEqualityFilter("objectSid", binarySidToStringSid(sidBytes)))
.collect(Collectors.toList());
listener.onResponse(Filter.createORFilter(orFilters));
}
}, listener::onFailure),
TOKEN_GROUPS);
}
/**
* To better understand what the sid is and how its string representation looks like, see
* http://blogs.msdn.com/b/alextch/archive/2007/06/18/sample-java-application-that-retrieves-group-membership-of-an-active-directory
* -user-account.aspx
*
* @param SID byte encoded security ID
*/
private static String binarySidToStringSid(byte[] SID) {
String strSID;
//convert the SID into string format
long version;
long authority;
long count;
long rid;
strSID = "S";
version = SID[0];
strSID = strSID + "-" + Long.toString(version);
authority = SID[4];
for (int i = 0; i < 4; i++) {
authority <<= 8;
authority += SID[4 + i] & 0xFF;
}
strSID = strSID + "-" + Long.toString(authority);
count = SID[2];
count <<= 8;
count += SID[1] & 0xFF;
for (int j = 0; j < count; j++) {
rid = SID[11 + (j * 4)] & 0xFF;
for (int k = 1; k < 4; k++) {
rid <<= 8;
rid += SID[11 - k + (j * 4)] & 0xFF;
}
strSID = strSID + "-" + Long.toString(rid);
}
return strSID;
}
}

View File

@ -0,0 +1,354 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.Filter;
import com.unboundid.ldap.sdk.LDAPConnection;
import com.unboundid.ldap.sdk.LDAPConnectionOptions;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.SearchResultEntry;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchSecurityException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.cache.Cache;
import org.elasticsearch.common.cache.CacheBuilder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ssl.SSLService;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.attributesToSearchFor;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.createFilter;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.search;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.searchForEntry;
/**
* This Class creates LdapSessions authenticating via the custom Active Directory protocol. (that being
* authenticating with a principal name, "username@domain", then searching through the directory to find the
* user entry in Active Directory that matches the user name). This eliminates the need for user templates, and simplifies
* the configuration for windows admins that may not be familiar with LDAP concepts.
*/
class ActiveDirectorySessionFactory extends SessionFactory {
static final String AD_DOMAIN_NAME_SETTING = "domain_name";
static final String AD_GROUP_SEARCH_BASEDN_SETTING = "group_search.base_dn";
static final String AD_GROUP_SEARCH_SCOPE_SETTING = "group_search.scope";
static final String AD_USER_SEARCH_BASEDN_SETTING = "user_search.base_dn";
static final String AD_USER_SEARCH_FILTER_SETTING = "user_search.filter";
static final String AD_USER_SEARCH_SCOPE_SETTING = "user_search.scope";
private static final String NETBIOS_NAME_FILTER_TEMPLATE = "(netbiosname={0})";
private final DefaultADAuthenticator defaultADAuthenticator;
private final DownLevelADAuthenticator downLevelADAuthenticator;
private final UpnADAuthenticator upnADAuthenticator;
ActiveDirectorySessionFactory(RealmConfig config, SSLService sslService) {
super(config, sslService);
Settings settings = config.settings();
String domainName = settings.get(AD_DOMAIN_NAME_SETTING);
if (domainName == null) {
throw new IllegalArgumentException("missing [" + AD_DOMAIN_NAME_SETTING + "] setting for active directory");
}
String domainDN = buildDnFromDomain(domainName);
GroupsResolver groupResolver = new ActiveDirectoryGroupsResolver(settings.getAsSettings("group_search"), domainDN);
defaultADAuthenticator = new DefaultADAuthenticator(settings, timeout, logger, groupResolver, domainDN);
downLevelADAuthenticator = new DownLevelADAuthenticator(settings, timeout, logger, groupResolver, domainDN);
upnADAuthenticator = new UpnADAuthenticator(settings, timeout, logger, groupResolver, domainDN);
}
@Override
protected String[] getDefaultLdapUrls(Settings settings) {
return new String[] {"ldap://" + settings.get(AD_DOMAIN_NAME_SETTING) + ":389"};
}
/**
* This is an active directory bind that looks up the user DN after binding with a windows principal.
*
* @param username name of the windows user without the domain
*/
@Override
public void session(String username, SecuredString password, ActionListener<LdapSession> listener) {
// the runnable action here allows us make the control/flow logic simpler to understand. If we got a connection then lets
// authenticate. If there was a failure pass it back using the listener
Runnable runnable;
try {
final LDAPConnection connection = serverSet.getConnection();
runnable = () -> getADAuthenticator(username).authenticate(connection, username, password,
ActionListener.wrap(listener::onResponse,
(e) -> {
IOUtils.closeWhileHandlingException(connection);
listener.onFailure(e);
}));
} catch (LDAPException e) {
runnable = () -> listener.onFailure(e);
}
runnable.run();
}
/**
* @param domain active directory domain name
* @return LDAP DN, distinguished name, of the root of the domain
*/
static String buildDnFromDomain(String domain) {
return "DC=" + domain.replace(".", ",DC=");
}
ADAuthenticator getADAuthenticator(String username) {
if (username.indexOf('\\') > 0) {
return downLevelADAuthenticator;
} else if (username.indexOf("@") > 0) {
return upnADAuthenticator;
}
return defaultADAuthenticator;
}
abstract static class ADAuthenticator {
final TimeValue timeout;
final Logger logger;
final GroupsResolver groupsResolver;
final String userSearchDN;
final LdapSearchScope userSearchScope;
ADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
this.timeout = timeout;
this.logger = logger;
this.groupsResolver = groupsResolver;
userSearchDN = settings.get(AD_USER_SEARCH_BASEDN_SETTING, domainDN);
userSearchScope = LdapSearchScope.resolve(settings.get(AD_USER_SEARCH_SCOPE_SETTING), LdapSearchScope.SUB_TREE);
}
final void authenticate(LDAPConnection connection, String username, SecuredString password,
ActionListener<LdapSession> listener) {
boolean success = false;
try {
connection.bind(bindUsername(username), new String(password.internalChars()));
searchForDN(connection, username, password, Math.toIntExact(timeout.seconds()), ActionListener.wrap((entry) -> {
if (entry == null) {
IOUtils.close(connection);
// we did not find the user, cannot authenticate in this realm
listener.onFailure(new ElasticsearchSecurityException("search for user [" + username
+ "] by principle name yielded no results"));
} else {
final String dn = entry.getDN();
listener.onResponse(new LdapSession(logger, connection, dn, groupsResolver, timeout, null));
}
}, (e) -> {
IOUtils.closeWhileHandlingException(connection);
listener.onFailure(e);
}));
success = true;
} catch (LDAPException e) {
listener.onFailure(e);
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(connection);
}
}
}
String bindUsername(String username) {
return username;
}
abstract void searchForDN(LDAPConnection connection, String username, SecuredString password, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener);
}
/**
* This authenticator is used for usernames that do not contain an `@` or `/`. It attempts a bind with the provided username combined
* with the domain name specified in settings. On AD DS this will work for both upn@domain and samaccountname@domain; AD LDS will only
* support the upn format
*/
static class DefaultADAuthenticator extends ADAuthenticator {
final String userSearchFilter;
final String domainName;
DefaultADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
super(settings, timeout, logger, groupsResolver, domainDN);
domainName = settings.get(AD_DOMAIN_NAME_SETTING);
userSearchFilter = settings.get(AD_USER_SEARCH_FILTER_SETTING, "(&(objectClass=user)(|(sAMAccountName={0})" +
"(userPrincipalName={0}@" + domainName + ")))");
}
@Override
void searchForDN(LDAPConnection connection, String username, SecuredString password, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener) {
try {
searchForEntry(connection, userSearchDN, userSearchScope.scope(), createFilter(userSearchFilter, username),
timeLimitSeconds, listener, attributesToSearchFor(groupsResolver.attributes()));
} catch (LDAPException e) {
listener.onFailure(e);
}
}
@Override
String bindUsername(String username) {
return username + "@" + domainName;
}
}
/**
* Active Directory calls the format <code>DOMAIN\\username</code> down-level credentials and this class contains the logic necessary
* to authenticate this form of a username
*/
static class DownLevelADAuthenticator extends ADAuthenticator {
Cache<String, String> domainNameCache = CacheBuilder.<String, String>builder().setMaximumWeight(100).build();
final String domainDN;
final Settings settings;
DownLevelADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
super(settings, timeout, logger, groupsResolver, domainDN);
this.domainDN = domainDN;
this.settings = settings;
}
@Override
void searchForDN(LDAPConnection connection, String username, SecuredString password, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener) {
String[] parts = username.split("\\\\");
assert parts.length == 2;
final String netBiosDomainName = parts[0];
final String accountName = parts[1];
netBiosDomainNameToDn(connection, netBiosDomainName, username, password, timeLimitSeconds, ActionListener.wrap((domainDN) -> {
if (domainDN == null) {
IOUtils.close(connection);
listener.onResponse(null);
} else {
try {
searchForEntry(connection, domainDN, LdapSearchScope.SUB_TREE.scope(),
createFilter("(&(objectClass=user)(sAMAccountName={0}))", accountName), timeLimitSeconds, listener,
attributesToSearchFor(groupsResolver.attributes()));
} catch (LDAPException e) {
IOUtils.closeWhileHandlingException(connection);
listener.onFailure(e);
}
}
}, (e) -> {
IOUtils.closeWhileHandlingException(connection);
listener.onFailure(e);
}));
}
void netBiosDomainNameToDn(LDAPConnection connection, String netBiosDomainName, String username, SecuredString password,
int timeLimitSeconds, ActionListener<String> listener) {
final String cachedName = domainNameCache.get(netBiosDomainName);
if (cachedName != null) {
listener.onResponse(cachedName);
} else if (usingGlobalCatalog(settings, connection)) {
// the global catalog does not replicate the necessary information to map a netbios dns name to a DN so we need to instead
// connect to the normal ports. This code uses the standard ports to avoid adding even more settings and is probably ok as
// most AD users do not use non-standard ports
final LDAPConnectionOptions options = connectionOptions(settings);
boolean startedSearching = false;
LDAPConnection searchConnection = null;
try {
Filter filter = createFilter(NETBIOS_NAME_FILTER_TEMPLATE, netBiosDomainName);
if (connection.getSSLSession() != null) {
searchConnection = new LDAPConnection(connection.getSocketFactory(), options,
connection.getConnectedAddress(), 636);
} else {
searchConnection = new LDAPConnection(options, connection.getConnectedAddress(), 389);
}
searchConnection.bind(username, new String(password.internalChars()));
final LDAPConnection finalConnection = searchConnection;
search(finalConnection, domainDN, LdapSearchScope.SUB_TREE.scope(), filter, timeLimitSeconds,
ActionListener.wrap((results) -> {
IOUtils.close(finalConnection);
handleSearchResults(results, netBiosDomainName, domainNameCache, listener);
}, (e) -> {
IOUtils.closeWhileHandlingException(connection);
listener.onFailure(e);
}),
"ncname");
startedSearching = true;
} catch (LDAPException e) {
listener.onFailure(e);
} finally {
if (startedSearching == false) {
IOUtils.closeWhileHandlingException(searchConnection);
}
}
} else {
try {
Filter filter = createFilter(NETBIOS_NAME_FILTER_TEMPLATE, netBiosDomainName);
search(connection, domainDN, LdapSearchScope.SUB_TREE.scope(), filter, timeLimitSeconds,
ActionListener.wrap((results) -> handleSearchResults(results, netBiosDomainName, domainNameCache, listener),
(e) -> {
IOUtils.closeWhileHandlingException(connection);
listener.onFailure(e);
}),
"ncname");
} catch (LDAPException e) {
listener.onFailure(e);
}
}
}
static void handleSearchResults(List<SearchResultEntry> results, String netBiosDomainName, Cache<String, String> domainNameCache,
ActionListener<String> listener) {
Optional<SearchResultEntry> entry = results.stream().filter((r) -> r.hasAttribute("ncname")).findFirst();
if (entry.isPresent()) {
final String value = entry.get().getAttributeValue("ncname");
try {
domainNameCache.computeIfAbsent(netBiosDomainName, (s) -> value);
} catch (ExecutionException e) {
throw new AssertionError("failed to load constant non-null value", e);
}
listener.onResponse(value);
} else {
listener.onResponse(null);
}
}
static boolean usingGlobalCatalog(Settings settings, LDAPConnection ldapConnection) {
Boolean usingGlobalCatalog = settings.getAsBoolean("global_catalog", null);
if (usingGlobalCatalog != null) {
return usingGlobalCatalog;
}
return ldapConnection.getConnectedPort() == 3268 || ldapConnection.getConnectedPort() == 3269;
}
}
static class UpnADAuthenticator extends ADAuthenticator {
private static final String UPN_USER_FILTER = "(&(objectClass=user)(|(sAMAccountName={0})(userPrincipalName={1})))";
UpnADAuthenticator(Settings settings, TimeValue timeout, Logger logger, GroupsResolver groupsResolver, String domainDN) {
super(settings, timeout, logger, groupsResolver, domainDN);
}
void searchForDN(LDAPConnection connection, String username, SecuredString password, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener) {
String[] parts = username.split("@");
assert parts.length == 2;
final String accountName = parts[0];
final String domainName = parts[1];
final String domainDN = buildDnFromDomain(domainName);
try {
Filter filter = createFilter(UPN_USER_FILTER, accountName, username);
searchForEntry(connection, domainDN, LdapSearchScope.SUB_TREE.scope(), filter, timeLimitSeconds, listener,
attributesToSearchFor(groupsResolver.attributes()));
} catch (LDAPException e) {
listener.onFailure(e);
}
}
}
}

View File

@ -6,59 +6,176 @@
package org.elasticsearch.xpack.security.authc.ldap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import com.unboundid.ldap.sdk.LDAPException;
import org.elasticsearch.ElasticsearchException;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.AbstractLdapRealm;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapLoadBalancing;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.CachingUsernamePasswordRealm;
import org.elasticsearch.xpack.security.authc.support.DnRoleMapper;
import org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken;
import org.elasticsearch.xpack.security.user.User;
import org.elasticsearch.xpack.ssl.SSLService;
/**
* Authenticates username/password tokens against ldap, locates groups and maps them to roles.
*/
public class LdapRealm extends AbstractLdapRealm {
public final class LdapRealm extends CachingUsernamePasswordRealm {
public static final String TYPE = "ldap";
public static final String LDAP_TYPE = "ldap";
public static final String AD_TYPE = "active_directory";
public LdapRealm(RealmConfig config, ResourceWatcherService watcherService, SSLService sslService) {
this(config, sessionFactory(config, sslService), new DnRoleMapper(TYPE, config, watcherService, null));
private final SessionFactory sessionFactory;
private final DnRoleMapper roleMapper;
private final ThreadPool threadPool;
public LdapRealm(String type, RealmConfig config, ResourceWatcherService watcherService, SSLService sslService,
ThreadPool threadPool) throws LDAPException {
this(type, config, sessionFactory(config, sslService, type), new DnRoleMapper(type, config, watcherService, null), threadPool);
}
// pkg private for testing
LdapRealm(RealmConfig config, SessionFactory sessionFactory, DnRoleMapper roleMapper) {
super(TYPE, config, sessionFactory, roleMapper);
LdapRealm(String type, RealmConfig config, SessionFactory sessionFactory, DnRoleMapper roleMapper, ThreadPool threadPool) {
super(type, config);
this.sessionFactory = sessionFactory;
this.roleMapper = roleMapper;
this.threadPool = threadPool;
roleMapper.addListener(this::expireAll);
}
static SessionFactory sessionFactory(RealmConfig config, SSLService sslService) {
Settings searchSettings = userSearchSettings(config);
try {
if (!searchSettings.names().isEmpty()) {
if (config.settings().getAsArray(LdapSessionFactory.USER_DN_TEMPLATES_SETTING).length > 0) {
throw new IllegalArgumentException("settings were found for both user search and user template modes of operation. " +
"Please remove the settings for the mode you do not wish to use. For more details refer to the ldap " +
"authentication section of the X-Pack guide.");
}
return new LdapUserSearchSessionFactory(config, sslService);
static SessionFactory sessionFactory(RealmConfig config, SSLService sslService, String type) throws LDAPException {
final SessionFactory sessionFactory;
if (AD_TYPE.equals(type)) {
sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
} else {
assert LDAP_TYPE.equals(type) : "type [" + type + "] is unknown. expected one of [" + AD_TYPE + ", " + LDAP_TYPE + "]";
Settings searchSettings = userSearchSettings(config);
if (searchSettings.names().isEmpty()) {
sessionFactory = new LdapSessionFactory(config, sslService);
} else if (config.settings().getAsArray(LdapSessionFactory.USER_DN_TEMPLATES_SETTING).length > 0) {
throw new IllegalArgumentException("settings were found for both user search and user template modes of operation. " +
"Please remove the settings for the mode you do not wish to use. For more details refer to the ldap " +
"authentication section of the X-Pack guide.");
} else {
sessionFactory = new LdapUserSearchSessionFactory(config, sslService);
}
return new LdapSessionFactory(config, sslService);
} catch (LDAPException e) {
throw new ElasticsearchException("failed to create realm [{}/{}]", e, LdapRealm.TYPE, config.name());
}
return sessionFactory;
}
static Settings userSearchSettings(RealmConfig config) {
return config.settings().getAsSettings("user_search");
}
/**
* Given a username and password, open a connection to ldap, bind to authenticate, retrieve groups, map to roles and build the user.
* This user will then be passed to the listener
*/
@Override
protected void doAuthenticate(UsernamePasswordToken token, ActionListener<User> listener) {
// we submit to the threadpool because authentication using LDAP will execute blocking I/O for a bind request and we don't want
// network threads stuck waiting for a socket to connect. After the bind, then all interaction with LDAP should be async
threadPool.generic().execute(() -> sessionFactory.session(token.principal(), token.credentials(),
new LdapSessionActionListener(token.principal(), listener, roleMapper)));
}
@Override
protected void doLookupUser(String username, ActionListener<User> listener) {
if (sessionFactory.supportsUnauthenticatedSession()) {
// we submit to the threadpool because authentication using LDAP will execute blocking I/O for a bind request and we don't want
// network threads stuck waiting for a socket to connect. After the bind, then all interaction with LDAP should be async
threadPool.generic().execute(() ->
sessionFactory.unauthenticatedSession(username, new LdapSessionActionListener(username, listener, roleMapper)));
} else {
listener.onResponse(null);
}
}
@Override
public boolean userLookupSupported() {
return sessionFactory.supportsUnauthenticatedSession();
}
@Override
public Map<String, Object> usageStats() {
Map<String, Object> stats = super.usageStats();
stats.put("user_search", userSearchSettings(config).isEmpty() == false);
return stats;
Map<String, Object> usage = super.usageStats();
usage.put("load_balance_type", LdapLoadBalancing.resolve(config.settings()).toString());
usage.put("ssl", sessionFactory.isSslUsed());
usage.put("user_search", userSearchSettings(config).isEmpty() == false);
return usage;
}
private static void lookupGroups(LdapSession session, String username, ActionListener<User> listener, DnRoleMapper roleMapper) {
if (session == null) {
listener.onResponse(null);
} else {
boolean loadingGroups = false;
try {
session.groups(ActionListener.wrap((groups) -> {
Set<String> roles = roleMapper.resolveRoles(session.userDn(), groups);
IOUtils.close(session);
listener.onResponse(new User(username, roles.toArray(Strings.EMPTY_ARRAY)));
},
(e) -> {
IOUtils.closeWhileHandlingException(session);
listener.onFailure(e);
}));
loadingGroups = true;
} finally {
if (loadingGroups == false) {
session.close();
}
}
}
}
/**
* A special {@link ActionListener} that encapsulates the handling of a LdapSession, which is used to return a user. This class handles
* cases where the session is null or where an exception may be caught after a session has been established, which requires the
* closing of the session.
*/
private static class LdapSessionActionListener implements ActionListener<LdapSession> {
private final AtomicReference<LdapSession> ldapSessionAtomicReference = new AtomicReference<>();
private final String username;
private final ActionListener<User> userActionListener;
private final DnRoleMapper roleMapper;
LdapSessionActionListener(String username, ActionListener<User> userActionListener, DnRoleMapper roleMapper) {
this.username = username;
this.userActionListener = userActionListener;
this.roleMapper = roleMapper;
}
@Override
public void onResponse(LdapSession session) {
if (session == null) {
userActionListener.onResponse(null);
} else {
ldapSessionAtomicReference.set(session);
lookupGroups(session, username, userActionListener, roleMapper);
}
}
@Override
public void onFailure(Exception e) {
if (ldapSessionAtomicReference.get() != null) {
IOUtils.closeWhileHandlingException(ldapSessionAtomicReference.get());
userActionListener.onFailure(e);
} else {
userActionListener.onFailure(e);
}
}
}
}

View File

@ -7,17 +7,22 @@ package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.LDAPConnection;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.SimpleBindRequest;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.CharArrays;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ssl.SSLService;
import java.text.MessageFormat;
import java.util.Arrays;
import java.util.Locale;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.escapedRDNValue;
@ -50,35 +55,52 @@ public class LdapSessionFactory extends SessionFactory {
* is kept as the cause of the thrown exception
*
* @param username a relative name, Not a distinguished name, that will be inserted into the template.
* @return authenticated exception
*/
@Override
protected LdapSession getSession(String username, SecuredString password) throws Exception {
LDAPConnection connection = serverSet.getConnection();
public void session(String username, SecuredString password, ActionListener<LdapSession> listener) {
LDAPException lastException = null;
String passwordString = new String(password.internalChars());
for (String template : userDnTemplates) {
String dn = buildDnFromTemplate(username, template);
try {
connection.bind(dn, passwordString);
return new LdapSession(logger, connection, dn, groupResolver, timeout, null);
} catch (LDAPException e) {
// we catch the ldapException here since we expect it can happen and we shouldn't be logging this all the time otherwise
// it is just noise
logger.debug((Supplier<?>) () -> new ParameterizedMessage(
"failed LDAP authentication with user template [{}] and DN [{}]", template, dn), e);
if (lastException == null) {
lastException = e;
} else {
lastException.addSuppressed(e);
LDAPConnection connection = null;
LdapSession ldapSession = null;
final byte[] passwordBytes = CharArrays.toUtf8Bytes(password.internalChars());
boolean success = false;
try {
connection = serverSet.getConnection();
for (String template : userDnTemplates) {
String dn = buildDnFromTemplate(username, template);
try {
connection.bind(new SimpleBindRequest(dn, passwordBytes));
ldapSession = new LdapSession(logger, connection, dn, groupResolver, timeout, null);
success = true;
break;
} catch (LDAPException e) {
// we catch the ldapException here since we expect it can happen and we shouldn't be logging this all the time otherwise
// it is just noise
logger.trace((Supplier<?>) () -> new ParameterizedMessage(
"failed LDAP authentication with user template [{}] and DN [{}]", template, dn), e);
if (lastException == null) {
lastException = e;
} else {
lastException.addSuppressed(e);
}
}
}
} catch (LDAPException e) {
assert lastException == null : "if we catch a LDAPException here, we should have never seen another exception";
assert ldapSession == null : "LDAPSession should not have been established due to a connection failure";
lastException = e;
} finally {
Arrays.fill(passwordBytes, (byte) 0);
if (success == false) {
IOUtils.closeWhileHandlingException(connection);
}
}
connection.close();
assert lastException != null;
throw lastException;
if (ldapSession != null) {
listener.onResponse(ldapSession);
} else {
assert lastException != null : "if there is not LDAPSession, then we must have a exception";
listener.onFailure(lastException);
}
}
/**

View File

@ -11,11 +11,12 @@ import com.unboundid.ldap.sdk.LDAPConnectionPool;
import com.unboundid.ldap.sdk.LDAPConnectionPoolHealthCheck;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResultEntry;
import com.unboundid.ldap.sdk.ServerSet;
import com.unboundid.ldap.sdk.SimpleBindRequest;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.RealmConfig;
@ -23,11 +24,11 @@ import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.CharArrays;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ssl.SSLService;
import org.elasticsearch.xpack.security.support.Exceptions;
import java.util.Locale;
import java.util.Arrays;
import static com.unboundid.ldap.sdk.Filter.createEqualityFilter;
import static com.unboundid.ldap.sdk.Filter.encodeValue;
@ -116,38 +117,91 @@ class LdapUserSearchSessionFactory extends SessionFactory {
}
@Override
protected LdapSession getSession(String user, SecuredString password) throws Exception {
public void session(String user, SecuredString password, ActionListener<LdapSession> listener) {
if (useConnectionPool) {
return getSessionWithPool(user, password);
getSessionWithPool(user, password, listener);
} else {
return getSessionWithoutPool(user, password);
getSessionWithoutPool(user, password, listener);
}
}
private LdapSession getSessionWithPool(String user, SecuredString password) throws Exception {
SearchResultEntry searchResult = findUser(user, connectionPool);
assert searchResult != null;
final String dn = searchResult.getDN();
connectionPool.bindAndRevertAuthentication(dn, new String(password.internalChars()));
return new LdapSession(logger, connectionPool, dn, groupResolver, timeout, searchResult.getAttributes());
/**
* Sets up a LDAPSession using the connection pool that potentially holds existing connections to the server
*/
private void getSessionWithPool(String user, SecuredString password, ActionListener<LdapSession> listener) {
findUser(user, connectionPool, ActionListener.wrap((entry) -> {
if (entry == null) {
listener.onResponse(null);
} else {
final String dn = entry.getDN();
try {
connectionPool.bindAndRevertAuthentication(dn, new String(password.internalChars()));
listener.onResponse(new LdapSession(logger, connectionPool, dn, groupResolver, timeout, entry.getAttributes()));
} catch (LDAPException e) {
listener.onFailure(e);
}
}
}, listener::onFailure));
}
private LdapSession getSessionWithoutPool(String user, SecuredString password) throws Exception {
/**
* Sets up a LDAPSession using the following process:
* <ol>
* <li>Opening a new connection to the LDAP server</li>
* <li>Executes a bind request using the bind user</li>
* <li>Executes a search to find the DN of the user</li>
* <li>Closes the opened connection</li>
* <li>Opens a new connection to the LDAP server</li>
* <li>Executes a bind request using the found DN and provided password</li>
* <li>Creates a new LDAPSession with the bound connection</li>
* </ol>
*/
private void getSessionWithoutPool(String user, SecuredString password, ActionListener<LdapSession> listener) {
boolean success = false;
LDAPConnection connection = null;
try {
connection = serverSet.getConnection();
connection.bind(bindRequest(config.settings()));
SearchResultEntry searchResult = findUser(user, connection);
assert searchResult != null;
final String dn = searchResult.getDN();
connection.bind(dn, new String(password.internalChars()));
LdapSession session = new LdapSession(logger, connection, dn, groupResolver, timeout, searchResult.getAttributes());
final LDAPConnection finalConnection = connection;
findUser(user, connection, ActionListener.wrap((entry) -> {
// close the existing connection since we are executing in this handler of the previous request and cannot bind here
// so we need to open a new connection to bind on and use for the session
IOUtils.close(finalConnection);
if (entry == null) {
listener.onResponse(null);
} else {
final String dn = entry.getDN();
boolean sessionCreated = false;
LDAPConnection userConnection = null;
final byte[] passwordBytes = CharArrays.toUtf8Bytes(password.internalChars());
try {
userConnection = serverSet.getConnection();
userConnection.bind(new SimpleBindRequest(dn, passwordBytes));
LdapSession session = new LdapSession(logger, userConnection, dn, groupResolver, timeout,
entry.getAttributes());
sessionCreated = true;
listener.onResponse(session);
} catch (Exception e) {
listener.onFailure(e);
} finally {
Arrays.fill(passwordBytes, (byte) 0);
if (sessionCreated == false) {
IOUtils.close(userConnection);
}
}
}
},
(e) -> {
IOUtils.closeWhileHandlingException(finalConnection);
listener.onFailure(e);
}));
success = true;
return session;
} catch (LDAPException e) {
listener.onFailure(e);
} finally {
if (success == false && connection != null) {
connection.close();
// need the success flag since the search is async and we don't want to close it if it is in progress
if (success == false) {
IOUtils.closeWhileHandlingException(connection);
}
}
}
@ -158,7 +212,7 @@ class LdapUserSearchSessionFactory extends SessionFactory {
}
@Override
public LdapSession unauthenticatedSession(String user) throws Exception {
public void unauthenticatedSession(String user, ActionListener<LdapSession> listener) {
LDAPConnection connection = null;
boolean success = false;
try {
@ -171,29 +225,37 @@ class LdapUserSearchSessionFactory extends SessionFactory {
ldapInterface = connection;
}
SearchResultEntry searchResult = findUser(user, ldapInterface);
assert searchResult != null;
final String dn = searchResult.getDN();
LdapSession session = new LdapSession(logger, ldapInterface, dn, groupResolver, timeout, searchResult.getAttributes());
findUser(user, ldapInterface, ActionListener.wrap((entry) -> {
if (entry == null) {
listener.onResponse(null);
} else {
boolean sessionCreated = false;
try {
final String dn = entry.getDN();
LdapSession session = new LdapSession(logger, ldapInterface, dn, groupResolver, timeout, entry.getAttributes());
sessionCreated = true;
listener.onResponse(session);
} finally {
if (sessionCreated == false && useConnectionPool == false) {
IOUtils.close((LDAPConnection) ldapInterface);
}
}
}
}, listener::onFailure));
success = true;
return session;
} catch (LDAPException e) {
listener.onFailure(e);
} finally {
if (success == false && connection != null) {
connection.close();
if (success == false) {
IOUtils.closeWhileHandlingException(connection);
}
}
}
private SearchResultEntry findUser(String user, LDAPInterface ldapInterface) throws Exception {
SearchRequest request = new SearchRequest(userSearchBaseDn, scope.scope(), createEqualityFilter(userAttribute, encodeValue(user)),
private void findUser(String user, LDAPInterface ldapInterface, ActionListener<SearchResultEntry> listener) {
searchForEntry(ldapInterface, userSearchBaseDn, scope.scope(),
createEqualityFilter(userAttribute, encodeValue(user)), Math.toIntExact(timeout.seconds()), listener,
attributesToSearchFor(groupResolver.attributes()));
request.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResultEntry entry = searchForEntry(ldapInterface, request, logger);
if (entry == null) {
throw Exceptions.authenticationError("failed to find user [{}] with search base [{}] scope [{}]", user, userSearchBaseDn,
scope.toString().toLowerCase(Locale.ENGLISH));
}
return entry;
}
/*

View File

@ -6,22 +6,22 @@
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.Filter;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResult;
import com.unboundid.ldap.sdk.SearchResultEntry;
import com.unboundid.ldap.sdk.SearchScope;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.OBJECT_CLASS_PRESENCE_FILTER;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.createFilter;
@ -54,23 +54,24 @@ class SearchGroupsResolver implements GroupsResolver {
}
@Override
public List<String> resolve(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger,
Collection<Attribute> attributes) throws LDAPException {
String userId = getUserId(userDn, attributes, connection, timeout, logger);
if (userId == null) {
// attributes were queried but the requested wasn't found
return Collections.emptyList();
}
SearchRequest searchRequest = new SearchRequest(baseDn, scope.scope(), createFilter(filter, userId),
SearchRequest.NO_ATTRIBUTES);
searchRequest.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResult results = search(connection, searchRequest, logger);
List<String> groups = new ArrayList<>(results.getSearchEntries().size());
for (SearchResultEntry entry : results.getSearchEntries()) {
groups.add(entry.getDN());
}
return groups;
public void resolve(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger,
Collection<Attribute> attributes, ActionListener<List<String>> listener) {
getUserId(userDn, attributes, connection, timeout, ActionListener.wrap((userId) -> {
if (userId == null) {
listener.onResponse(Collections.emptyList());
} else {
try {
Filter userFilter = createFilter(filter, userId);
search(connection, baseDn, scope.scope(), userFilter, Math.toIntExact(timeout.seconds()),
ActionListener.wrap(
(results) -> listener.onResponse(results.stream().map((r) -> r.getDN()).collect(Collectors.toList())),
listener::onFailure),
SearchRequest.NO_ATTRIBUTES);
} catch (LDAPException e) {
listener.onFailure(e);
}
}
}, listener::onFailure));
}
public String[] attributes() {
@ -80,34 +81,30 @@ class SearchGroupsResolver implements GroupsResolver {
return null;
}
private String getUserId(String dn, Collection<Attribute> attributes, LDAPInterface connection, TimeValue
timeout, Logger logger) throws LDAPException {
private void getUserId(String dn, Collection<Attribute> attributes, LDAPInterface connection, TimeValue timeout,
ActionListener<String> listener) {
if (userAttribute == null) {
return dn;
listener.onResponse(dn);
} else if (attributes != null) {
final String value = attributes.stream().filter((attribute) -> attribute.getName().equals(userAttribute))
.map(Attribute::getValue)
.findFirst()
.orElse(null);
listener.onResponse(value);
} else {
readUserAttribute(connection, dn, timeout, listener);
}
if (attributes != null) {
for (Attribute attribute : attributes) {
if (attribute.getName().equals(userAttribute)) {
return attribute.getValue();
}
}
}
return readUserAttribute(connection, dn, timeout, logger);
}
String readUserAttribute(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger) throws LDAPException {
SearchRequest request = new SearchRequest(userDn, SearchScope.BASE, OBJECT_CLASS_PRESENCE_FILTER, userAttribute);
request.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResultEntry results = searchForEntry(connection, request, logger);
if (results == null) {
return null;
}
Attribute attribute = results.getAttribute(userAttribute);
if (attribute == null) {
return null;
}
return attribute.getValue();
void readUserAttribute(LDAPInterface connection, String userDn, TimeValue timeout, ActionListener<String> listener) {
searchForEntry(connection, userDn, SearchScope.BASE, OBJECT_CLASS_PRESENCE_FILTER, Math.toIntExact(timeout.seconds()),
ActionListener.wrap((entry) -> {
if (entry == null || entry.hasAttribute(userAttribute) == false) {
listener.onResponse(null);
} else {
listener.onResponse(entry.getAttributeValue(userAttribute));
}
}, listener::onFailure),
userAttribute);
}
}

View File

@ -6,12 +6,10 @@
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResultEntry;
import com.unboundid.ldap.sdk.SearchScope;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
@ -21,6 +19,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.OBJECT_CLASS_PRESENCE_FILTER;
import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.searchForEntry;
@ -41,30 +40,22 @@ class UserAttributeGroupsResolver implements GroupsResolver {
}
@Override
public List<String> resolve(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger,
Collection<Attribute> attributes) throws LDAPException {
public void resolve(LDAPInterface connection, String userDn, TimeValue timeout, Logger logger, Collection<Attribute> attributes,
ActionListener<List<String>> listener) {
if (attributes != null) {
for (Attribute attribute : attributes) {
if (attribute.getName().equals(attribute)) {
String[] values = attribute.getValues();
return Arrays.asList(values);
}
}
return Collections.emptyList();
List<String> list = attributes.stream().filter((attr) -> attr.getName().equals(attribute))
.flatMap(attr -> Arrays.stream(attr.getValues())).collect(Collectors.toList());
listener.onResponse(Collections.unmodifiableList(list));
} else {
searchForEntry(connection, userDn, SearchScope.BASE, OBJECT_CLASS_PRESENCE_FILTER, Math.toIntExact(timeout.seconds()),
ActionListener.wrap((entry) -> {
if (entry == null || entry.hasAttribute(attribute) == false) {
listener.onResponse(Collections.emptyList());
} else {
listener.onResponse(Collections.unmodifiableList(Arrays.asList(entry.getAttributeValues(attribute))));
}
}, listener::onFailure), attribute);
}
SearchRequest request = new SearchRequest(userDn, SearchScope.BASE, OBJECT_CLASS_PRESENCE_FILTER, attribute);
request.setTimeLimitSeconds(Math.toIntExact(timeout.seconds()));
SearchResultEntry result = searchForEntry(connection, request, logger);
if (result == null) {
return Collections.emptyList();
}
Attribute attributeReturned = result.getAttribute(attribute);
if (attributeReturned == null) {
return Collections.emptyList();
}
String[] values = attributeReturned.getValues();
return Arrays.asList(values);
}
@Override

View File

@ -1,118 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.ldap.support;
import com.unboundid.ldap.sdk.LDAPException;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.CachingUsernamePasswordRealm;
import org.elasticsearch.xpack.security.authc.support.DnRoleMapper;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken;
import org.elasticsearch.xpack.security.user.User;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Supporting class for LDAP realms
*/
public abstract class AbstractLdapRealm extends CachingUsernamePasswordRealm {
protected final SessionFactory sessionFactory;
protected final DnRoleMapper roleMapper;
protected AbstractLdapRealm(String type, RealmConfig config,
SessionFactory sessionFactory, DnRoleMapper roleMapper) {
super(type, config);
this.sessionFactory = sessionFactory;
this.roleMapper = roleMapper;
roleMapper.addListener(new Listener());
}
/**
* Given a username and password, open a connection to ldap, bind to authenticate, retrieve groups, map to roles and build the user.
* This user will then be passed to the listener
*/
@Override
protected final void doAuthenticate(UsernamePasswordToken token, ActionListener<User> listener) {
// we use a runnable so that we call the listener outside of the try catch block. If we call within the try catch block and the
// listener throws an exception then we mistakenly could continue realm authentication when we already authenticated the user and
// there was some other issue
Runnable action;
try (LdapSession session = sessionFactory.session(token.principal(), token.credentials())) {
final User user = createUser(token.principal(), session);
action = () -> listener.onResponse(user);
} catch (Exception e) {
logException("authentication", e, token.principal());
action = () -> listener.onResponse(null);
}
action.run();
}
@Override
protected final void doLookupUser(String username, ActionListener<User> listener) {
// we use a runnable so that we call the listener outside of the try catch block. If we call within the try catch block and the
// listener throws an exception then we mistakenly could continue realm lookup when we already found a matching user and
// there was some other issue
Runnable action;
if (sessionFactory.supportsUnauthenticatedSession()) {
try (LdapSession session = sessionFactory.unauthenticatedSession(username)) {
final User user = createUser(username, session);
action = () -> listener.onResponse(user);
} catch (Exception e) {
logException("lookup", e, username);
action = () -> listener.onResponse(null);
}
} else {
action = () -> listener.onResponse(null);
}
action.run();
}
@Override
public boolean userLookupSupported() {
return sessionFactory.supportsUnauthenticatedSession();
}
@Override
public Map<String, Object> usageStats() {
Map<String, Object> usage = super.usageStats();
usage.put("load_balance_type", LdapLoadBalancing.resolve(config.settings()).toString());
usage.put("ssl", sessionFactory.sslUsed);
return usage;
}
private void logException(String action, Exception e, String principal) {
if (logger.isDebugEnabled()) {
logger.debug((Supplier<?>) () -> new ParameterizedMessage("{} failed for user [{}]", action, principal), e);
} else {
String causeMessage = (e.getCause() == null) ? null : e.getCause().getMessage();
if (causeMessage == null) {
logger.warn("{} failed for user [{}]: {}", action, principal, e.getMessage());
} else {
logger.warn("{} failed for user [{}]: {}\ncause: {}: {}", action, principal, e.getMessage(),
e.getCause().getClass().getName(), causeMessage);
}
}
}
private User createUser(String principal, LdapSession session) throws LDAPException {
List<String> groupDNs = session.groups();
Set<String> roles = roleMapper.resolveRoles(session.userDn(), groupDNs);
return new User(principal, roles.toArray(new String[roles.size()]));
}
class Listener implements RefreshListener {
@Override
public void onRefresh() {
expireAll();
}
}
}

View File

@ -7,19 +7,19 @@ package org.elasticsearch.xpack.security.authc.ldap.support;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.LDAPConnection;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPInterface;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.unit.TimeValue;
import java.io.Closeable;
import java.util.Collection;
import java.util.List;
/**
* Represents a LDAP connection with an authenticated/bound user that needs closing.
*/
public class LdapSession implements Closeable {
public class LdapSession implements Releasable {
protected final Logger logger;
protected final LDAPInterface ldap;
@ -51,7 +51,8 @@ public class LdapSession implements Closeable {
*/
@Override
public void close() {
// Only if it is an LDAPConnection do we need to close it
// Only if it is an LDAPConnection do we need to close it, otherwise it is a connection pool and we will close all of the
// connections in the pool
if (ldap instanceof LDAPConnection) {
((LDAPConnection) ldap).close();
}
@ -65,17 +66,33 @@ public class LdapSession implements Closeable {
}
/**
* @return List of fully distinguished group names
* Asynchronously retrieves a list of group distinguished names
*/
public List<String> groups() throws LDAPException {
return groupsResolver.resolve(ldap, userDn, timeout, logger, attributes);
public void groups(ActionListener<List<String>> listener) {
groupsResolver.resolve(ldap, userDn, timeout, logger, attributes, listener);
}
/**
* A GroupsResolver is used to resolve the group names of a given LDAP user
*/
public interface GroupsResolver {
List<String> resolve(LDAPInterface ldapConnection, String userDn, TimeValue timeout, Logger logger,
Collection<Attribute> attributes) throws LDAPException;
/**
* Asynchronously resolve the group name for the given ldap user
* @param ldapConnection an authenticated {@link LDAPConnection} to be used for LDAP queries
* @param userDn the distinguished name of the ldap user
* @param timeout the timeout for any ldap operation
* @param logger the logger to use if necessary
* @param attributes a collection of attributes that were previously retrieved for the user such as during a user search.
* {@code null} indicates that the attributes have not been attempted to be retrieved
* @param listener the listener to call on a result or on failure
*/
void resolve(LDAPInterface ldapConnection, String userDn, TimeValue timeout, Logger logger, Collection<Attribute> attributes,
ActionListener<List<String>> listener);
/**
* Returns the attributes that this resolvers uses. If no attributes are required, return {@code null}.
*/
String[] attributes();
}
}

View File

@ -5,22 +5,42 @@
*/
package org.elasticsearch.xpack.security.authc.ldap.support;
import com.unboundid.ldap.sdk.AsyncRequestID;
import com.unboundid.ldap.sdk.AsyncSearchResultListener;
import com.unboundid.ldap.sdk.DN;
import com.unboundid.ldap.sdk.DereferencePolicy;
import com.unboundid.ldap.sdk.Filter;
import com.unboundid.ldap.sdk.LDAPConnection;
import com.unboundid.ldap.sdk.LDAPConnectionPool;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.LDAPSearchException;
import com.unboundid.ldap.sdk.LDAPURL;
import com.unboundid.ldap.sdk.ResultCode;
import com.unboundid.ldap.sdk.SearchRequest;
import com.unboundid.ldap.sdk.SearchResult;
import com.unboundid.ldap.sdk.SearchResultEntry;
import com.unboundid.ldap.sdk.SearchResultReference;
import com.unboundid.ldap.sdk.SearchScope;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.util.concurrent.CountDown;
import org.elasticsearch.xpack.security.support.Exceptions;
import javax.naming.ldap.Rdn;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
public final class LdapUtils {
@ -47,53 +67,127 @@ public final class LdapUtils {
}
/**
* This method performs a LDAPConnection.search(...) operation while handling referral exceptions. This is necessary
* to maintain backwards compatibility with the original JNDI implementation
* This method performs an asynchronous ldap search operation that could have multiple results
*/
public static SearchResult search(LDAPInterface ldap, SearchRequest searchRequest, Logger logger) throws LDAPException {
SearchResult results;
try {
results = ldap.search(searchRequest);
} catch (LDAPSearchException e) {
if (e.getResultCode().equals(ResultCode.REFERRAL) && e.getSearchResult() != null) {
if (logger.isDebugEnabled()) {
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"a referral could not be followed for request [{}] so some results may not have been retrieved",
searchRequest),
e);
}
results = e.getSearchResult();
} else {
throw e;
}
public static void searchForEntry(LDAPInterface ldap, String baseDN, SearchScope scope, Filter filter, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener, String... attributes) {
if (ldap instanceof LDAPConnection) {
searchForEntry((LDAPConnection) ldap, baseDN, scope, filter, timeLimitSeconds, listener, attributes);
} else if (ldap instanceof LDAPConnectionPool) {
searchForEntry((LDAPConnectionPool) ldap, baseDN, scope, filter, timeLimitSeconds, listener, attributes);
} else {
throw new IllegalArgumentException("unsupported LDAPInterface implementation: " + ldap);
}
return results;
}
/**
* This method performs a LDAPConnection.searchForEntry(...) operation while handling referral exceptions. This is necessary
* to maintain backwards compatibility with the original JNDI implementation
* This method performs an asynchronous ldap search operation that only expects at most one result. If more than one result is found
* then this is an error. If no results are found, then {@code null} will be returned.
*/
public static SearchResultEntry searchForEntry(LDAPInterface ldap, SearchRequest searchRequest, Logger logger) throws LDAPException {
SearchResultEntry entry;
public static void searchForEntry(LDAPConnection ldap, String baseDN, SearchScope scope, Filter filter, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener, String... attributes) {
LdapSearchResultListener searchResultListener = new SingleEntryListener(ldap, listener, filter);
try {
entry = ldap.searchForEntry(searchRequest);
} catch (LDAPSearchException e) {
if (e.getResultCode().equals(ResultCode.REFERRAL) && e.getSearchResult() != null && e.getSearchResult().getEntryCount() > 0) {
if (logger.isDebugEnabled()) {
logger.debug(
(Supplier<?>) () -> new ParameterizedMessage(
"a referral could not be followed for request [{}] so some results may not have been retrieved",
searchRequest),
e);
}
entry = e.getSearchResult().getSearchEntries().get(0);
} else {
throw e;
SearchRequest request = new SearchRequest(searchResultListener, baseDN, scope, DereferencePolicy.NEVER, 0, timeLimitSeconds,
false, filter, attributes);
searchResultListener.setSearchRequest(request);
ldap.asyncSearch(request);
} catch (LDAPException e) {
listener.onFailure(e);
}
}
/**
* This method performs an asynchronous ldap search operation that only expects at most one result. If more than one result is found
* then this is an error. If no results are found, then {@code null} will be returned.
*/
public static void searchForEntry(LDAPConnectionPool ldap, String baseDN, SearchScope scope, Filter filter, int timeLimitSeconds,
ActionListener<SearchResultEntry> listener, String... attributes) {
boolean searching = false;
LDAPConnection ldapConnection = null;
try {
ldapConnection = ldap.getConnection();
final LDAPConnection finalConnection = ldapConnection;
searchForEntry(finalConnection, baseDN, scope, filter, timeLimitSeconds, ActionListener.wrap(
(entry) -> {
IOUtils.close(() -> ldap.releaseConnection(finalConnection));
listener.onResponse(entry);
},
(e) -> {
IOUtils.closeWhileHandlingException(() -> ldap.releaseConnection(finalConnection));
listener.onFailure(e);
}), attributes);
searching = true;
} catch (LDAPException e) {
listener.onFailure(e);
} finally {
if (searching == false) {
final LDAPConnection finalConnection = ldapConnection;
IOUtils.closeWhileHandlingException(() -> ldap.releaseConnection(finalConnection));
}
}
}
/**
* This method performs an asynchronous ldap search operation that could have multiple results
*/
public static void search(LDAPInterface ldap, String baseDN, SearchScope scope, Filter filter, int timeLimitSeconds,
ActionListener<List<SearchResultEntry>> listener, String... attributes) {
if (ldap instanceof LDAPConnection) {
search((LDAPConnection) ldap, baseDN, scope, filter, timeLimitSeconds, listener, attributes);
} else if (ldap instanceof LDAPConnectionPool) {
search((LDAPConnectionPool) ldap, baseDN, scope, filter, timeLimitSeconds, listener, attributes);
} else {
throw new IllegalArgumentException("unsupported LDAPInterface implementation: " + ldap);
}
}
/**
* This method performs an asynchronous ldap search operation that could have multiple results
*/
public static void search(LDAPConnection ldap, String baseDN, SearchScope scope, Filter filter, int timeLimitSeconds,
ActionListener<List<SearchResultEntry>> listener, String... attributes) {
LdapSearchResultListener searchResultListener = new LdapSearchResultListener(ldap,
(asyncRequestID, searchResult) -> listener.onResponse(Collections.unmodifiableList(searchResult.getSearchEntries())), 1);
try {
SearchRequest request = new SearchRequest(searchResultListener, baseDN, scope, DereferencePolicy.NEVER, 0, timeLimitSeconds,
false, filter, attributes);
searchResultListener.setSearchRequest(request);
ldap.asyncSearch(request);
} catch (LDAPException e) {
listener.onFailure(e);
}
}
/**
* This method performs an asynchronous ldap search operation that could have multiple results
*/
public static void search(LDAPConnectionPool ldap, String baseDN, SearchScope scope, Filter filter, int timeLimitSeconds,
ActionListener<List<SearchResultEntry>> listener, String... attributes) {
boolean searching = false;
LDAPConnection ldapConnection = null;
try {
ldapConnection = ldap.getConnection();
final LDAPConnection finalConnection = ldapConnection;
LdapSearchResultListener ldapSearchResultListener = new LdapSearchResultListener(ldapConnection,
(asyncRequestID, searchResult) -> {
IOUtils.closeWhileHandlingException(() -> ldap.releaseConnection(finalConnection));
listener.onResponse(Collections.unmodifiableList(searchResult.getSearchEntries()));
}, 1);
SearchRequest request = new SearchRequest(ldapSearchResultListener, baseDN, scope, DereferencePolicy.NEVER, 0, timeLimitSeconds,
false, filter, attributes);
ldapSearchResultListener.setSearchRequest(request);
finalConnection.asyncSearch(request);
searching = true;
} catch (LDAPException e) {
listener.onFailure(e);
} finally {
if (searching == false && ldapConnection != null) {
final LDAPConnection finalConnection = ldapConnection;
IOUtils.closeWhileHandlingException(() -> ldap.releaseConnection(finalConnection));
}
}
return entry;
}
public static Filter createFilter(String filterTemplate, String... arguments) throws LDAPException {
@ -111,4 +205,186 @@ public final class LdapUtils {
}
return arguments;
}
private static class SingleEntryListener extends LdapSearchResultListener {
SingleEntryListener(LDAPConnection ldapConnection, ActionListener<SearchResultEntry> listener, Filter filter) {
super(ldapConnection, ((asyncRequestID, searchResult) -> {
final List<SearchResultEntry> entryList = searchResult.getSearchEntries();
if (entryList.size() > 1) {
listener.onFailure(Exceptions.authenticationError("multiple search results found for [{}]", filter));
} else if (entryList.size() == 1) {
listener.onResponse(entryList.get(0));
} else {
listener.onResponse(null);
}
}), 1);
}
}
private static class LdapSearchResultListener implements AsyncSearchResultListener {
private static final Logger LOGGER = ESLoggerFactory.getLogger(LdapUtils.class);
private final List<SearchResultEntry> entryList = new ArrayList<>();
private final List<SearchResultReference> referenceList = new ArrayList<>();
protected final SetOnce<SearchRequest> searchRequestRef = new SetOnce<>();
private final BiConsumer<AsyncRequestID, SearchResult> consumer;
private final LDAPConnection ldapConnection;
private final int depth;
LdapSearchResultListener(LDAPConnection ldapConnection, BiConsumer<AsyncRequestID, SearchResult> consumer, int depth) {
this.ldapConnection = ldapConnection;
this.consumer = consumer;
this.depth = depth;
}
@Override
public void searchEntryReturned(SearchResultEntry searchEntry) {
entryList.add(searchEntry);
}
@Override
public void searchReferenceReturned(SearchResultReference searchReference) {
referenceList.add(searchReference);
}
@Override
public void searchResultReceived(AsyncRequestID requestID, SearchResult searchResult) {
// whenever we get a search result we need to check for a referral. A referral is a mechanism for an LDAP server to reference
// an object stored in a different LDAP server/partition. There are cases where we need to follow a referral in order to get
// the actual object we are searching for
final String[] referralUrls = referenceList.stream()
.flatMap((ref) -> Arrays.stream(ref.getReferralURLs()))
.collect(Collectors.toList())
.toArray(Strings.EMPTY_ARRAY);
final SearchRequest searchRequest = searchRequestRef.get();
if (referralUrls.length == 0 || searchRequest.followReferrals(ldapConnection) == false) {
// either no referrals to follow or we have explicitly disabled referral following on the connection so we just create
// a new search result that has the values we've collected. The search result passed to this method will not have of the
// entries as we are using a result listener and the results are not being collected by the LDAP library
SearchResult resultWithValues = new SearchResult(searchResult.getMessageID(), searchResult.getResultCode(), searchResult
.getDiagnosticMessage(), searchResult.getMatchedDN(), referralUrls, entryList, referenceList, entryList.size(),
referenceList.size(), searchResult.getResponseControls());
consumer.accept(requestID, resultWithValues);
} else if (depth >= ldapConnection.getConnectionOptions().getReferralHopLimit()) {
// we've gone through too many levels of referrals so we terminate with the values collected so far and the proper result
// code to indicate the search was terminated early
SearchResult resultWithValues = new SearchResult(searchResult.getMessageID(), ResultCode.REFERRAL_LIMIT_EXCEEDED,
searchResult.getDiagnosticMessage(), searchResult.getMatchedDN(), referralUrls, entryList, referenceList,
entryList.size(), referenceList.size(), searchResult.getResponseControls());
consumer.accept(requestID, resultWithValues);
} else {
// there are referrals to follow, so we start the process to follow the referrals
final CountDown countDown = new CountDown(referralUrls.length);
final List<String> referralUrlsList = new ArrayList<>(Arrays.asList(referralUrls));
BiConsumer<AsyncRequestID, SearchResult> referralConsumer = (reqID, innerResult) -> {
// synchronize here since we are possibly sending out a lot of requests and the result lists are not thread safe and
// this also provides us with a consistent view
synchronized (this) {
if (innerResult.getSearchEntries() != null) {
entryList.addAll(innerResult.getSearchEntries());
}
if (innerResult.getSearchReferences() != null) {
referenceList.addAll(innerResult.getSearchReferences());
}
}
// count down and once all referrals have been traversed then we can create the results
if (countDown.countDown()) {
SearchResult resultWithValues = new SearchResult(searchResult.getMessageID(), searchResult.getResultCode(),
searchResult.getDiagnosticMessage(), searchResult.getMatchedDN(),
referralUrlsList.toArray(Strings.EMPTY_ARRAY), entryList, referenceList,
entryList.size(), referenceList.size(), searchResult.getResponseControls());
consumer.accept(requestID, resultWithValues);
}
};
for (String referralUrl : referralUrls) {
try {
// for each referral follow it and any other referrals returned until we get to a depth that is greater than or
// equal to the referral hop limit or all referrals have been followed. Each time referrals are followed from a
// search result, the depth increases by 1
followReferral(ldapConnection, referralUrl, searchRequest, referralConsumer, depth + 1, searchResult, requestID);
} catch (LDAPException e) {
LOGGER.warn((Supplier<?>)
() -> new ParameterizedMessage("caught exception while trying to follow referral [{}]", referralUrl), e);
referralConsumer.accept(requestID, new SearchResult(searchResult.getMessageID(), e.getResultCode(),
e.getDiagnosticMessage(), e.getMatchedDN(), e.getReferralURLs(), 0, 0, e.getResponseControls()));
}
}
}
}
void setSearchRequest(SearchRequest searchRequest) {
this.searchRequestRef.set(searchRequest);
}
}
/**
* Performs the actual connection and following of a referral given a URL string. This referral is being followed as it may contain a
* result that is relevant to our search
*/
private static void followReferral(LDAPConnection ldapConnection, String urlString, SearchRequest searchRequest,
BiConsumer<AsyncRequestID, SearchResult> consumer, int depth,
SearchResult originatingResult, AsyncRequestID asyncRequestID) throws LDAPException {
final LDAPURL referralURL = new LDAPURL(urlString);
final String host = referralURL.getHost();
// the host must be present in order to follow a referral
if (host != null) {
// the referral URL often contains information necessary about the LDAP request such as the base DN, scope, and filter. If it
// does not, then we reuse the values from the originating search request
final String requestBaseDN;
if (referralURL.baseDNProvided()) {
requestBaseDN = referralURL.getBaseDN().toString();
} else {
requestBaseDN = searchRequest.getBaseDN();
}
final SearchScope requestScope;
if (referralURL.scopeProvided()) {
requestScope = referralURL.getScope();
} else {
requestScope = searchRequest.getScope();
}
final Filter requestFilter;
if (referralURL.filterProvided()) {
requestFilter = referralURL.getFilter();
} else {
requestFilter = searchRequest.getFilter();
}
// in order to follow the referral we need to open a new connection and we do so using the referral connector on the ldap
// connection
final LDAPConnection referralConn =
ldapConnection.getReferralConnector().getReferralConnection(referralURL, ldapConnection);
final LdapSearchResultListener listener = new LdapSearchResultListener(referralConn,
(reqId, searchResult) -> {
IOUtils.closeWhileHandlingException(referralConn);
consumer.accept(reqId, searchResult);
}, depth);
boolean success = false;
try {
final SearchRequest referralSearchRequest =
new SearchRequest(listener, searchRequest.getControls(),
requestBaseDN, requestScope, searchRequest.getDereferencePolicy(),
searchRequest.getSizeLimit(), searchRequest.getTimeLimitSeconds(), searchRequest.typesOnly(),
requestFilter, searchRequest.getAttributes());
listener.setSearchRequest(searchRequest);
referralConn.asyncSearch(referralSearchRequest);
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(referralConn);
}
}
} else {
// nothing to really do since a null host cannot really be handled, so we just return with a response that is empty...
consumer.accept(asyncRequestID, new SearchResult(originatingResult.getMessageID(), ResultCode.UNAVAILABLE,
null, null, null, Collections.emptyList(), Collections.emptyList(), 0, 0, null));
}
}
}

View File

@ -11,6 +11,7 @@ import com.unboundid.ldap.sdk.LDAPURL;
import com.unboundid.ldap.sdk.ServerSet;
import com.unboundid.util.ssl.HostNameSSLSocketVerifier;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
@ -19,10 +20,9 @@ import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.ssl.SSLService;
import javax.net.SocketFactory;
import java.util.Arrays;
import java.util.regex.Pattern;
import static java.util.Arrays.asList;
/**
* This factory holds settings needed for authenticating to LDAP and creating LdapConnections.
* Each created LdapConnection needs to be closed or else connections will pill up consuming resources.
@ -77,28 +77,13 @@ public abstract class SessionFactory {
*
* @param user The name of the user to authenticate the connection with.
* @param password The password of the user
* @return LdapSession representing a connection to LDAP as the provided user
* @throws Exception if an error occurred when creating the session
* @param listener the listener to call on a failure or result
*/
public final LdapSession session(String user, SecuredString password) throws Exception {
return getSession(user, password);
}
/**
* Implementors should create a {@link LdapSession} that will be used to Authenticates the given user. This connection
* should be bound to the user (meaning, all operations under the returned connection will be executed on behalf of the authenticated
* user.
*
* @param user The name of the user to authenticate the connection with.
* @param password The password of the user
* @return LdapSession representing a connection to LDAP as the provided user
* @throws Exception if an error occurred when creating the session
*/
protected abstract LdapSession getSession(String user, SecuredString password) throws Exception;
public abstract void session(String user, SecuredString password, ActionListener<LdapSession> listener);
/**
* Returns a flag to indicate if this session factory supports unauthenticated sessions. This means that a session can
* be established without providing any credentials in a call to {@link SessionFactory#unauthenticatedSession(String)}
* be established without providing any credentials in a call to {@link #unauthenticatedSession(String, ActionListener)}
*
* @return true if the factory supports unauthenticated sessions
*/
@ -110,10 +95,9 @@ public abstract class SessionFactory {
* Returns an {@link LdapSession} for the user identified by the String parameter
*
* @param username the identifier for the user
* @return LdapSession representing a connection to LDAP for the provided user.
* @throws Exception if an error occurs when creating the session or unauthenticated sessions are not supported
* @param listener the listener to call on a failure or result
*/
public LdapSession unauthenticatedSession(String username) throws Exception {
public void unauthenticatedSession(String username, ActionListener<LdapSession> listener) {
throw new UnsupportedOperationException("unauthenticated sessions are not supported");
}
@ -129,7 +113,7 @@ public abstract class SessionFactory {
return options;
}
protected final LDAPServers ldapServers(Settings settings) {
private LDAPServers ldapServers(Settings settings) {
// Parse LDAP urls
String[] ldapUrls = settings.getAsArray(URLS_SETTING, getDefaultLdapUrls(settings));
if (ldapUrls == null || ldapUrls.length == 0) {
@ -142,7 +126,7 @@ public abstract class SessionFactory {
return null;
}
protected ServerSet serverSet(Settings settings, SSLService clientSSLService, LDAPServers ldapServers) {
private ServerSet serverSet(Settings settings, SSLService clientSSLService, LDAPServers ldapServers) {
SocketFactory socketFactory = null;
if (ldapServers.ssl()) {
socketFactory = clientSSLService.sslSocketFactory(settings.getByPrefix("ssl."));
@ -161,6 +145,10 @@ public abstract class SessionFactory {
return serverSet;
}
public boolean isSslUsed() {
return sslUsed;
}
public static class LDAPServers {
private final String[] addresses;
@ -202,8 +190,8 @@ public abstract class SessionFactory {
return true;
}
boolean allSecure = asList(ldapUrls).stream().allMatch(s -> STARTS_WITH_LDAPS.matcher(s).find());
boolean allClear = asList(ldapUrls).stream().allMatch(s -> STARTS_WITH_LDAP.matcher(s).find());
final boolean allSecure = Arrays.stream(ldapUrls).allMatch(s -> STARTS_WITH_LDAPS.matcher(s).find());
final boolean allClear = Arrays.stream(ldapUrls).allMatch(s -> STARTS_WITH_LDAP.matcher(s).find());
if (!allSecure && !allClear) {
//No mixing is allowed because we use the same socketfactory

View File

@ -11,7 +11,6 @@ import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.watcher.FileChangesListener;
@ -54,12 +53,13 @@ public class DnRoleMapper {
private final boolean useUnmappedGroupsAsRoles;
private volatile Map<DN, Set<String>> dnRoles;
private CopyOnWriteArrayList<RefreshListener> listeners;
private CopyOnWriteArrayList<Runnable> listeners;
public DnRoleMapper(String realmType, RealmConfig config, ResourceWatcherService watcherService, @Nullable RefreshListener listener) {
public DnRoleMapper(String realmType, RealmConfig config, ResourceWatcherService watcherService, Runnable listener) {
this.realmType = realmType;
this.config = config;
this.logger = config.logger(getClass());
this.listeners = new CopyOnWriteArrayList<>(Collections.singleton(listener));
useUnmappedGroupsAsRoles = config.settings().getAsBoolean(USE_UNMAPPED_GROUPS_AS_ROLES_SETTING, false);
file = resolveFile(config.settings(), config.env());
@ -71,13 +71,9 @@ public class DnRoleMapper {
} catch (IOException e) {
throw new ElasticsearchException("failed to start file watcher for role mapping file [" + file.toAbsolutePath() + "]", e);
}
listeners = new CopyOnWriteArrayList<>();
if (listener != null) {
listeners.add(listener);
}
}
public synchronized void addListener(RefreshListener listener) {
public synchronized void addListener(Runnable listener) {
listeners.add(listener);
}
@ -186,9 +182,7 @@ public class DnRoleMapper {
}
public void notifyRefresh() {
for (RefreshListener listener : listeners) {
listener.onRefresh();
}
listeners.forEach(Runnable::run);
}
private class FileListener implements FileChangesListener {

View File

@ -1,13 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.support;
public interface RefreshListener {
RefreshListener NOOP = () -> {};
void onRefresh();
}

View File

@ -21,7 +21,6 @@ import org.elasticsearch.watcher.FileWatcher;
import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.XPackSettings;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.authz.RoleDescriptor;
import org.elasticsearch.xpack.security.authz.permission.IndicesPermission.Group;
import org.elasticsearch.xpack.security.authz.permission.Role;
@ -49,16 +48,16 @@ public class FileRolesStore extends AbstractLifecycleComponent {
private static final Pattern SKIP_LINE = Pattern.compile("(^#.*|^\\s*)");
private final Path file;
private final RefreshListener listener;
private final Runnable listener;
private final ResourceWatcherService watcherService;
private volatile Map<String, Role> permissions;
public FileRolesStore(Settings settings, Environment env, ResourceWatcherService watcherService) {
this(settings, env, watcherService, RefreshListener.NOOP);
this(settings, env, watcherService, () -> {});
}
public FileRolesStore(Settings settings, Environment env, ResourceWatcherService watcherService, RefreshListener listener) {
public FileRolesStore(Settings settings, Environment env, ResourceWatcherService watcherService, Runnable listener) {
super(settings);
this.file = resolveFile(env);
this.listener = listener;
@ -328,7 +327,7 @@ public class FileRolesStore extends AbstractLifecycleComponent {
"could not reload roles file [{}]. Current roles remain unmodified", file.toAbsolutePath()), e);
return;
}
listener.onRefresh();
listener.run();
}
}
}

View File

@ -212,6 +212,7 @@ public class SecurityServerTransportInterceptor implements TransportInterceptor
}
assert filter != null;
final Thread executingThread = Thread.currentThread();
ActionListener.CheckedConsumer<Void> consumer = (x) -> {
final Executor executor;
if (executingThread == Thread.currentThread()) {

View File

@ -13,7 +13,6 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.SecurityIntegTestCase;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectoryRealm;
import org.elasticsearch.xpack.security.authc.ldap.LdapRealm;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken;
@ -62,8 +61,8 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase
realmConfig = randomFrom(RealmConfig.values());
useGlobalSSL = randomBoolean();
sslEnabled = randomBoolean();
ESLoggerFactory.getLogger("test").info("running test with realm configuration [{}], with direct group to role mapping [{}]",
realmConfig, realmConfig.mapGroupsAsRoles);
ESLoggerFactory.getLogger("test").info("running test with realm configuration [{}], with direct group to role mapping [{}]. " +
"Settings [{}]", realmConfig, realmConfig.mapGroupsAsRoles, realmConfig.settings.getAsMap());
}
@AfterClass
@ -177,7 +176,7 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase
AD(false, AD_ROLE_MAPPING,
Settings.builder()
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", ActiveDirectoryRealm.TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.AD_TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".domain_name", "ad.test.elasticsearch.com")
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL
+ ".group_search.base_dn", "CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com")
@ -186,7 +185,7 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase
AD_SSL(false, AD_ROLE_MAPPING,
Settings.builder()
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", ActiveDirectoryRealm.TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.AD_TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".domain_name", "ad.test.elasticsearch.com")
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL
+ ".group_search.base_dn", "CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com")
@ -196,7 +195,7 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase
AD_LDAP_GROUPS_FROM_SEARCH(true, AD_ROLE_MAPPING,
Settings.builder()
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.LDAP_TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".url", "ldaps://ad.test.elasticsearch.com:636")
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL
+ ".group_search.base_dn", "CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com")
@ -207,7 +206,7 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase
AD_LDAP_GROUPS_FROM_ATTRIBUTE(true, AD_ROLE_MAPPING,
Settings.builder()
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.LDAP_TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".url", "ldaps://ad.test.elasticsearch.com:636")
.putArray(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".user_dn_templates",
"cn={0},CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com")
@ -215,7 +214,7 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase
OLDAP(false, OLDAP_ROLE_MAPPING,
Settings.builder()
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".type", LdapRealm.LDAP_TYPE)
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".url", "ldaps://54.200.235.244:636")
.put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".group_search.base_dn",
"ou=people, dc=oldap, dc=test, dc=elasticsearch, dc=com")

View File

@ -14,6 +14,7 @@ import java.io.IOException;
*/
@Network
public class MultiGroupMappingTests extends AbstractAdLdapRealmTestCase {
@Override
protected String configRoles() {
return super.configRoles() +

View File

@ -5,12 +5,10 @@
*/
package org.elasticsearch.xpack.security;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -59,7 +57,7 @@ public class SecurityTests extends ESTestCase {
}
}
private Collection<Object> createComponents(Settings testSettings, XPackExtension... extensions) throws IOException {
private Collection<Object> createComponents(Settings testSettings, XPackExtension... extensions) throws Exception {
Settings settings = Settings.builder().put(testSettings)
.put("path.home", createTempDir()).build();
Environment env = new Environment(settings);
@ -71,7 +69,7 @@ public class SecurityTests extends ESTestCase {
allowedSettings.addAll(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
ClusterSettings clusterSettings = new ClusterSettings(settings, allowedSettings);
when(clusterService.getClusterSettings()).thenReturn(clusterSettings);
return security.createComponents(null, threadPool, clusterService, null, Arrays.asList(extensions));
return security.createComponents(null, threadPool, clusterService, mock(ResourceWatcherService.class), Arrays.asList(extensions));
}
private <T> T findComponent(Class<T> type, Collection<Object> components) {

View File

@ -8,6 +8,8 @@ package org.elasticsearch.xpack.security.authc;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchSecurityException;
@ -27,6 +29,7 @@ import org.elasticsearch.transport.TransportMessage;
import org.elasticsearch.xpack.security.audit.AuditTrailService;
import org.elasticsearch.xpack.security.authc.Authentication.RealmRef;
import org.elasticsearch.xpack.security.authc.AuthenticationService.Authenticator;
import org.elasticsearch.xpack.security.authc.Realm.Factory;
import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken;
@ -93,17 +96,8 @@ public class AuthenticationServiceTests extends ESTestCase {
XPackLicenseState licenseState = mock(XPackLicenseState.class);
when(licenseState.allowedRealmType()).thenReturn(XPackLicenseState.AllowedRealmType.ALL);
when(licenseState.isAuthAllowed()).thenReturn(true);
realms = new Realms(Settings.EMPTY, new Environment(settings), Collections.<String, Realm.Factory>emptyMap(),
licenseState, mock(ReservedRealm.class)) {
@Override
protected void doStart() {
this.realms = Arrays.asList(firstRealm, secondRealm);
this.internalRealmsOnly = Collections.singletonList(firstRealm);
}
};
realms.start();
realms = new TestRealms(Settings.EMPTY, new Environment(settings), Collections.<String, Realm.Factory>emptyMap(),
licenseState, mock(ReservedRealm.class), Arrays.asList(firstRealm, secondRealm), Collections.singletonList(firstRealm));
cryptoService = mock(CryptoService.class);
auditTrail = mock(AuditTrailService.class);
@ -821,4 +815,14 @@ public class AuthenticationServiceTests extends ESTestCase {
service.authenticate(action, message, fallbackUser, future);
return future.actionGet();
}
static class TestRealms extends Realms {
TestRealms(Settings settings, Environment env, Map<String, Factory> factories, XPackLicenseState licenseState,
ReservedRealm reservedRealm, List<Realm> realms, List<Realm> internalRealms) throws Exception {
super(settings, env, factories, licenseState, reservedRealm);
this.realms = realms;
this.internalRealmsOnly = internalRealms;
}
}
}

View File

@ -74,7 +74,6 @@ public class RealmsTests extends ESTestCase {
Settings settings = builder.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
Iterator<Realm> iterator = realms.iterator();
assertThat(iterator.hasNext(), is(true));
@ -102,7 +101,7 @@ public class RealmsTests extends ESTestCase {
.build();
Environment env = new Environment(settings);
try {
new Realms(settings, env, factories, licenseState, reservedRealm).start();
new Realms(settings, env, factories, licenseState, reservedRealm);
fail("Expected IllegalArgumentException");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("multiple [file] realms are configured"));
@ -112,7 +111,6 @@ public class RealmsTests extends ESTestCase {
public void testWithEmptySettings() throws Exception {
Realms realms = new Realms(Settings.EMPTY, new Environment(Settings.builder().put("path.home", createTempDir()).build()),
factories, licenseState, reservedRealm);
realms.start();
Iterator<Realm> iter = realms.iterator();
assertThat(iter.hasNext(), is(true));
Realm realm = iter.next();
@ -145,7 +143,6 @@ public class RealmsTests extends ESTestCase {
Settings settings = builder.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
// this is the iterator when licensed
Iterator<Realm> iter = realms.iterator();
@ -196,7 +193,7 @@ public class RealmsTests extends ESTestCase {
}
public void testUnlicensedWithInternalRealms() throws Exception {
factories.put(LdapRealm.TYPE, config -> new DummyRealm(LdapRealm.TYPE, config));
factories.put(LdapRealm.LDAP_TYPE, config -> new DummyRealm(LdapRealm.LDAP_TYPE, config));
assertThat(factories.get("type_0"), notNullValue());
Settings.Builder builder = Settings.builder()
.put("path.home", createTempDir())
@ -207,7 +204,6 @@ public class RealmsTests extends ESTestCase {
Settings settings = builder.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
Iterator<Realm> iter = realms.iterator();
assertThat(iter.hasNext(), is(true));
Realm realm = iter.next();
@ -253,7 +249,7 @@ public class RealmsTests extends ESTestCase {
}
public void testUnlicensedWithNativeRealms() throws Exception {
factories.put(LdapRealm.TYPE, config -> new DummyRealm(LdapRealm.TYPE, config));
factories.put(LdapRealm.LDAP_TYPE, config -> new DummyRealm(LdapRealm.LDAP_TYPE, config));
final String type = randomFrom(FileRealm.TYPE, NativeRealm.TYPE);
Settings.Builder builder = Settings.builder()
.put("path.home", createTempDir())
@ -264,7 +260,6 @@ public class RealmsTests extends ESTestCase {
Settings settings = builder.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
Iterator<Realm> iter = realms.iterator();
assertThat(iter.hasNext(), is(true));
Realm realm = iter.next();
@ -310,7 +305,6 @@ public class RealmsTests extends ESTestCase {
Settings settings = builder.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
Iterator<Realm> iterator = realms.iterator();
Realm realm = iterator.next();
assertThat(realm, is(reservedRealm));
@ -340,7 +334,7 @@ public class RealmsTests extends ESTestCase {
assertThat(count, equalTo(orderToIndex.size()));
}
public void testAuthcAuthzDisabled() {
public void testAuthcAuthzDisabled() throws Exception {
Settings settings = Settings.builder()
.put("path.home", createTempDir())
.put("xpack.security.authc.realms.realm_1.type", FileRealm.TYPE)
@ -348,7 +342,6 @@ public class RealmsTests extends ESTestCase {
.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
assertThat(realms.iterator().hasNext(), is(true));
@ -356,7 +349,7 @@ public class RealmsTests extends ESTestCase {
assertThat(realms.iterator().hasNext(), is(false));
}
public void testUsageStats() {
public void testUsageStats() throws Exception {
// test realms with duplicate values
Settings.Builder builder = Settings.builder()
.put("path.home", createTempDir())
@ -367,7 +360,6 @@ public class RealmsTests extends ESTestCase {
Settings settings = builder.build();
Environment env = new Environment(settings);
Realms realms = new Realms(settings, env, factories, licenseState, reservedRealm);
realms.start();
Map<String, Object> usageStats = realms.usageStats();
assertThat(usageStats.size(), is(factories.size()));

View File

@ -16,7 +16,6 @@ import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.security.audit.logfile.CapturingLogger;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.Hasher;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.authc.support.SecuredStringTests;
import org.elasticsearch.xpack.XPackPlugin;
import org.junit.After;
@ -90,12 +89,7 @@ public class FileUserPasswdStoreTests extends ESTestCase {
ResourceWatcherService watcherService = new ResourceWatcherService(settings, threadPool);
final CountDownLatch latch = new CountDownLatch(1);
FileUserPasswdStore store = new FileUserPasswdStore(config, watcherService, new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
FileUserPasswdStore store = new FileUserPasswdStore(config, watcherService, latch::countDown);
assertThat(store.userExists("bcrypt"), is(true));
assertThat(store.verifyPassword("bcrypt", SecuredStringTests.build("test123")), is(true));
@ -130,12 +124,7 @@ public class FileUserPasswdStoreTests extends ESTestCase {
ResourceWatcherService watcherService = new ResourceWatcherService(settings, threadPool);
final CountDownLatch latch = new CountDownLatch(1);
FileUserPasswdStore store = new FileUserPasswdStore(config, watcherService, new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
FileUserPasswdStore store = new FileUserPasswdStore(config, watcherService, latch::countDown);
assertTrue(store.verifyPassword("bcrypt", SecuredStringTests.build("test123")));

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.security.authc.file;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.core.LogEvent;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
@ -19,7 +18,6 @@ import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.XPackSettings;
import org.elasticsearch.xpack.security.audit.logfile.CapturingLogger;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.junit.After;
import org.junit.Before;
@ -97,12 +95,7 @@ public class FileUserRolesStoreTests extends ESTestCase {
ResourceWatcherService watcherService = new ResourceWatcherService(settings, threadPool);
final CountDownLatch latch = new CountDownLatch(1);
FileUserRolesStore store = new FileUserRolesStore(config, watcherService, new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
FileUserRolesStore store = new FileUserRolesStore(config, watcherService, latch::countDown);
String[] roles = store.roles("user1");
assertThat(roles, notNullValue());
@ -140,12 +133,7 @@ public class FileUserRolesStoreTests extends ESTestCase {
ResourceWatcherService watcherService = new ResourceWatcherService(settings, threadPool);
final CountDownLatch latch = new CountDownLatch(1);
FileUserRolesStore store = new FileUserRolesStore(config, watcherService, new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
FileUserRolesStore store = new FileUserRolesStore(config, watcherService, latch::countDown);
String[] roles = store.roles("user1");
assertThat(roles, notNullValue());

View File

@ -3,7 +3,7 @@
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
package org.elasticsearch.xpack.security.authc.ldap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;

View File

@ -3,12 +3,12 @@
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.Filter;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.ldap.GroupsResolverTestCase;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.support.NoOpLogger;
import org.elasticsearch.test.junit.annotations.Network;
@ -31,8 +31,8 @@ public class ActiveDirectoryGroupsResolverTests extends GroupsResolverTestCase {
.put("scope", LdapSearchScope.SUB_TREE)
.build();
ActiveDirectoryGroupsResolver resolver = new ActiveDirectoryGroupsResolver(settings, "DC=ad,DC=test,DC=elasticsearch,DC=com");
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE,
null);
List<String> groups = resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10),
NoOpLogger.INSTANCE, null);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
containsString("SHIELD"),
@ -49,8 +49,8 @@ public class ActiveDirectoryGroupsResolverTests extends GroupsResolverTestCase {
.put("base_dn", "CN=Builtin, DC=ad, DC=test, DC=elasticsearch,DC=com")
.build();
ActiveDirectoryGroupsResolver resolver = new ActiveDirectoryGroupsResolver(settings, "DC=ad,DC=test,DC=elasticsearch,DC=com");
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE,
null);
List<String> groups = resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10),
NoOpLogger.INSTANCE, null);
assertThat(groups, hasItem(containsString("Users")));
}
@ -60,8 +60,8 @@ public class ActiveDirectoryGroupsResolverTests extends GroupsResolverTestCase {
.put("base_dn", "CN=Users, CN=Builtin, DC=ad, DC=test, DC=elasticsearch, DC=com")
.build();
ActiveDirectoryGroupsResolver resolver = new ActiveDirectoryGroupsResolver(settings, "DC=ad,DC=test,DC=elasticsearch,DC=com");
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE,
null);
List<String> groups = resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10),
NoOpLogger.INSTANCE, null);
assertThat(groups, hasItem(containsString("CN=Users,CN=Builtin")));
}
@ -73,8 +73,9 @@ public class ActiveDirectoryGroupsResolverTests extends GroupsResolverTestCase {
"S-1-5-21-3510024162-210737641-214529065-513" //Default Domain Users group
};
final String dn = "CN=Jarvis, CN=Users, DC=ad, DC=test, DC=elasticsearch, DC=com";
Filter query =
ActiveDirectoryGroupsResolver.buildGroupQuery(ldapConnection, dn, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE);
PlainActionFuture<Filter> future = new PlainActionFuture<>();
ActiveDirectoryGroupsResolver.buildGroupQuery(ldapConnection, dn, TimeValue.timeValueSeconds(10), future);
Filter query = future.actionGet();
assertValidSidQuery(query, expectedSids);
}
@ -85,8 +86,9 @@ public class ActiveDirectoryGroupsResolverTests extends GroupsResolverTestCase {
"S-1-5-21-3510024162-210737641-214529065-513", //Default Domain Users group
"S-1-5-21-3510024162-210737641-214529065-1117"}; //Gods group
final String dn = "CN=Odin, CN=Users, DC=ad, DC=test, DC=elasticsearch, DC=com";
Filter query =
ActiveDirectoryGroupsResolver.buildGroupQuery(ldapConnection, dn, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE);
PlainActionFuture<Filter> future = new PlainActionFuture<>();
ActiveDirectoryGroupsResolver.buildGroupQuery(ldapConnection, dn, TimeValue.timeValueSeconds(10), future);
Filter query = future.actionGet();
assertValidSidQuery(query, expectedSids);
}
@ -102,8 +104,9 @@ public class ActiveDirectoryGroupsResolverTests extends GroupsResolverTestCase {
"S-1-5-21-3510024162-210737641-214529065-1105"};//Avengers
final String dn = BRUCE_BANNER_DN;
Filter query =
ActiveDirectoryGroupsResolver.buildGroupQuery(ldapConnection, dn, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE);
PlainActionFuture<Filter> future = new PlainActionFuture<>();
ActiveDirectoryGroupsResolver.buildGroupQuery(ldapConnection, dn, TimeValue.timeValueSeconds(10), future);
Filter query = future.actionGet();
assertValidSidQuery(query, expectedSids);
}
}

View File

@ -3,7 +3,7 @@
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.listener.InMemoryDirectoryServer;
import com.unboundid.ldap.listener.InMemoryDirectoryServerConfig;
@ -12,6 +12,7 @@ import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPURL;
import com.unboundid.ldap.sdk.schema.Schema;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
@ -64,8 +65,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
private static final String PASSWORD = "password";
protected static int numberOfLdapServers;
protected InMemoryDirectoryServer[] directoryServers;
static int numberOfLdapServers;
InMemoryDirectoryServer[] directoryServers;
private ResourceWatcherService resourceWatcherService;
private ThreadPool threadPool;
@ -114,8 +115,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
Settings settings = settings();
RealmConfig config = new RealmConfig("testAuthenticateUserPrincipleName", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, null);
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
realm.authenticate(new UsernamePasswordToken("CN=ironman", SecuredStringTests.build(PASSWORD)), future);
@ -128,8 +129,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
Settings settings = settings();
RealmConfig config = new RealmConfig("testAuthenticateSAMAccountName", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, null);
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
// Thor does not have a UPN of form CN=Thor@ad.test.elasticsearch.com
PlainActionFuture<User> future = new PlainActionFuture<>();
@ -152,8 +153,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
Settings settings = settings();
RealmConfig config = new RealmConfig("testAuthenticateCachesSuccesfulAuthentications", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = spy(new ActiveDirectorySessionFactory(config, null));
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
int count = randomIntBetween(2, 10);
for (int i = 0; i < count; i++) {
@ -163,15 +164,15 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
}
// verify one and only one session as further attempts should be returned from cache
verify(sessionFactory, times(1)).session(eq("CN=ironman"), any(SecuredString.class));
verify(sessionFactory, times(1)).session(eq("CN=ironman"), any(SecuredString.class), any(ActionListener.class));
}
public void testAuthenticateCachingCanBeDisabled() throws Exception {
Settings settings = settings(Settings.builder().put(CachingUsernamePasswordRealm.CACHE_TTL_SETTING, -1).build());
RealmConfig config = new RealmConfig("testAuthenticateCachingCanBeDisabled", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = spy(new ActiveDirectorySessionFactory(config, null));
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
int count = randomIntBetween(2, 10);
for (int i = 0; i < count; i++) {
@ -181,15 +182,15 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
}
// verify one and only one session as second attempt should be returned from cache
verify(sessionFactory, times(count)).session(eq("CN=ironman"), any(SecuredString.class));
verify(sessionFactory, times(count)).session(eq("CN=ironman"), any(SecuredString.class), any(ActionListener.class));
}
public void testAuthenticateCachingClearsCacheOnRoleMapperRefresh() throws Exception {
Settings settings = settings();
RealmConfig config = new RealmConfig("testAuthenticateCachingClearsCacheOnRoleMapperRefresh", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = spy(new ActiveDirectorySessionFactory(config, null));
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
int count = randomIntBetween(2, 10);
for (int i = 0; i < count; i++) {
@ -199,7 +200,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
}
// verify one and only one session as further attempts should be returned from cache
verify(sessionFactory, times(1)).session(eq("CN=ironman"), any(SecuredString.class));
verify(sessionFactory, times(1)).session(eq("CN=ironman"), any(SecuredString.class), any(ActionListener.class));
// Refresh the role mappings
roleMapper.notifyRefresh();
@ -210,7 +211,7 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
future.actionGet();
}
verify(sessionFactory, times(2)).session(eq("CN=ironman"), any(SecuredString.class));
verify(sessionFactory, times(2)).session(eq("CN=ironman"), any(SecuredString.class), any(ActionListener.class));
}
public void testRealmMapsGroupsToRoles() throws Exception {
@ -219,8 +220,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
.build());
RealmConfig config = new RealmConfig("testRealmMapsGroupsToRoles", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, null);
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
realm.authenticate(new UsernamePasswordToken("CN=ironman", SecuredStringTests.build(PASSWORD)), future);
@ -235,8 +236,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
.build());
RealmConfig config = new RealmConfig("testRealmMapsGroupsToRoles", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, null);
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
realm.authenticate(new UsernamePasswordToken("CN=Thor", SecuredStringTests.build(PASSWORD)), future);
@ -253,8 +254,8 @@ public class ActiveDirectoryRealmTests extends ESTestCase {
.build());
RealmConfig config = new RealmConfig("testRealmUsageStats", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, null);
DnRoleMapper roleMapper = new DnRoleMapper(ActiveDirectoryRealm.TYPE, config, resourceWatcherService, null);
ActiveDirectoryRealm realm = new ActiveDirectoryRealm(config, sessionFactory, roleMapper);
DnRoleMapper roleMapper = new DnRoleMapper(LdapRealm.AD_TYPE, config, resourceWatcherService, () -> {});
LdapRealm realm = new LdapRealm(LdapRealm.AD_TYPE, config, sessionFactory, roleMapper, threadPool);
Map<String, Object> stats = realm.usageStats();
assertThat(stats, is(notNullValue()));

View File

@ -3,25 +3,29 @@
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.security.authc.activedirectory;
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.LDAPException;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.UncategorizedExecutionException;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.LdapSessionFactory;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapTestCase;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.security.authc.support.SecuredStringTests;
import org.elasticsearch.test.junit.annotations.Network;
import java.util.List;
import java.util.concurrent.ExecutionException;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
@Network
@ -33,8 +37,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
String userName = "ironman";
try (LdapSession ldap = sessionFactory.session(userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Geniuses"),
containsString("Billionaire"),
@ -54,8 +58,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
String userName = "ades\\ironman";
try (LdapSession ldap = sessionFactory.session(userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Geniuses"),
containsString("Billionaire"),
@ -80,8 +84,9 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
RealmConfig config = new RealmConfig("ad-test", settings, globalSettings);
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
LDAPException expected = expectThrows(LDAPException.class,
() -> sessionFactory.session("ironman", SecuredStringTests.build(PASSWORD)).groups());
PlainActionFuture<List<String>> groups = new PlainActionFuture<>();
session(sessionFactory, "ironman", SecuredStringTests.build(PASSWORD)).groups(groups);
LDAPException expected = expectThrows(LDAPException.class, groups::actionGet);
assertThat(expected.getMessage(), containsString("A client-side timeout was encountered while waiting"));
}
@ -91,8 +96,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
String[] users = new String[]{"cap", "hawkeye", "hulk", "ironman", "thor", "blackwidow", };
for(String user: users) {
try (LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(PASSWORD))) {
assertThat("group avenger test for user "+user, ldap.groups(), hasItem(containsString("Avengers")));
try (LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(PASSWORD))) {
assertThat("group avenger test for user "+user, groups(ldap), hasItem(containsString("Avengers")));
}
}
}
@ -105,8 +110,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
String userName = "hulk";
try (LdapSession ldap = sessionFactory.session(userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
@ -127,8 +132,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
String userName = "hulk";
try (LdapSession ldap = sessionFactory.session(userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
@ -153,8 +158,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
String userName = "hulk";
try (LdapSession ldap = sessionFactory.session(userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, userName, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, hasItem(containsString("Avengers")));
}
@ -169,8 +174,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
//Login with the UserPrincipalName
String userDN = "CN=Erik Selvig,CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com";
try (LdapSession ldap = sessionFactory.session("erik.selvig", SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, "erik.selvig", SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(ldap.userDn(), is(userDN));
assertThat(groups, containsInAnyOrder(
containsString("Geniuses"),
@ -187,10 +192,10 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
//login with sAMAccountName
String userDN = "CN=Erik Selvig,CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com";
try (LdapSession ldap = sessionFactory.session("selvig", SecuredStringTests.build(PASSWORD))) {
try (LdapSession ldap = session(sessionFactory, "selvig", SecuredStringTests.build(PASSWORD))) {
assertThat(ldap.userDn(), is(userDN));
List<String> groups = ldap.groups();
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Geniuses"),
containsString("CN=Users,CN=Builtin"),
@ -210,8 +215,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
//Login with the UserPrincipalName
try (LdapSession ldap = sessionFactory.session("erik.selvig", SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, "erik.selvig", SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("CN=Geniuses"),
containsString("CN=Domain Users"),
@ -236,8 +241,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService);
String user = "Bruce Banner";
try (LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
@ -262,8 +267,8 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService);
String user = "Bruce Banner";
try (LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
@ -278,8 +283,11 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
ActiveDirectorySessionFactory sessionFactory = new ActiveDirectorySessionFactory(config, sslService);
String userName = "ironman";
LDAPException expected =
expectThrows(LDAPException.class, () -> sessionFactory.session(userName, SecuredStringTests.build(PASSWORD)));
UncategorizedExecutionException e = expectThrows(UncategorizedExecutionException.class,
() -> session(sessionFactory, userName, SecuredStringTests.build(PASSWORD)));
assertThat(e.getCause(), instanceOf(ExecutionException.class));
assertThat(e.getCause().getCause(), instanceOf(LDAPException.class));
final LDAPException expected = (LDAPException) e.getCause().getCause();
assertThat(expected.getMessage(), anyOf(containsString("Hostname verification failed"), containsString("peer not authenticated")));
}
@ -294,7 +302,11 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService);
String user = "Bruce Banner";
LDAPException expected = expectThrows(LDAPException.class, () -> sessionFactory.session(user, SecuredStringTests.build(PASSWORD)));
UncategorizedExecutionException e = expectThrows(UncategorizedExecutionException.class,
() -> session(sessionFactory, user, SecuredStringTests.build(PASSWORD)));
assertThat(e.getCause(), instanceOf(ExecutionException.class));
assertThat(e.getCause().getCause(), instanceOf(LDAPException.class));
final LDAPException expected = (LDAPException) e.getCause().getCause();
assertThat(expected.getMessage(), anyOf(containsString("Hostname verification failed"), containsString("peer not authenticated")));
}
@ -309,4 +321,16 @@ public class ActiveDirectorySessionFactoryTests extends AbstractActiveDirectoryI
}
return builder.build();
}
private LdapSession session(SessionFactory factory, String username, SecuredString password) {
PlainActionFuture<LdapSession> future = new PlainActionFuture<>();
factory.session(username, password, future);
return future.actionGet();
}
private List<String> groups(LdapSession ldapSession) {
PlainActionFuture<List<String>> future = new PlainActionFuture<>();
ldapSession.groups(future);
return future.actionGet();
}
}

View File

@ -5,11 +5,17 @@
*/
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.LDAPConnection;
import com.unboundid.ldap.sdk.LDAPConnectionOptions;
import com.unboundid.ldap.sdk.LDAPInterface;
import com.unboundid.ldap.sdk.LDAPURL;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.Environment;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession.GroupsResolver;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.ssl.SSLService;
@ -17,10 +23,12 @@ import org.junit.After;
import org.junit.Before;
import java.nio.file.Path;
import java.util.Collection;
import java.util.List;
public abstract class GroupsResolverTestCase extends ESTestCase {
protected LDAPConnection ldapConnection;
LDAPConnection ldapConnection;
protected abstract String ldapUrl();
@ -66,6 +74,15 @@ public abstract class GroupsResolverTestCase extends ESTestCase {
@After
public void tearDownLdapConnection() throws Exception {
ldapConnection.close();
if (ldapConnection != null) {
ldapConnection.close();
}
}
protected static List<String> resolveBlocking(GroupsResolver resolver, LDAPInterface ldapConnection, String dn, TimeValue timeLimit,
Logger logger, Collection<Attribute> attributes) {
PlainActionFuture<List<String>> future = new PlainActionFuture<>();
resolver.resolve(ldapConnection, dn, timeLimit, logger, attributes, future);
return future.actionGet();
}
}

View File

@ -5,6 +5,7 @@
*/
package org.elasticsearch.xpack.security.authc.ldap;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.security.authc.RealmConfig;
@ -68,7 +69,8 @@ public class LdapRealmTests extends LdapTestCase {
Settings settings = buildLdapSettings(ldapUrls(), userTemplate, groupSearchBase, LdapSearchScope.SUB_TREE);
RealmConfig config = new RealmConfig("test-ldap-realm", settings, globalSettings);
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
LdapRealm ldap = new LdapRealm(config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService));
LdapRealm ldap = new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService),
threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, SecuredStringTests.build(PASSWORD)), future);
@ -86,7 +88,8 @@ public class LdapRealmTests extends LdapTestCase {
RealmConfig config = new RealmConfig("test-ldap-realm", settings, globalSettings);
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
LdapRealm ldap = new LdapRealm(config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService));
LdapRealm ldap =
new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, SecuredStringTests.build(PASSWORD)), future);
@ -105,7 +108,8 @@ public class LdapRealmTests extends LdapTestCase {
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
ldapFactory = spy(ldapFactory);
LdapRealm ldap = new LdapRealm(config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService));
LdapRealm ldap =
new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, SecuredStringTests.build(PASSWORD)), future);
future.actionGet();
@ -114,7 +118,7 @@ public class LdapRealmTests extends LdapTestCase {
future.actionGet();
//verify one and only one session -> caching is working
verify(ldapFactory, times(1)).session(anyString(), any(SecuredString.class));
verify(ldapFactory, times(1)).session(anyString(), any(SecuredString.class), any(ActionListener.class));
}
public void testAuthenticateCachingRefresh() throws Exception {
@ -128,7 +132,7 @@ public class LdapRealmTests extends LdapTestCase {
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
DnRoleMapper roleMapper = buildGroupAsRoleMapper(resourceWatcherService);
ldapFactory = spy(ldapFactory);
LdapRealm ldap = new LdapRealm(config, ldapFactory, roleMapper);
LdapRealm ldap = new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory, roleMapper, threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, SecuredStringTests.build(PASSWORD)), future);
future.actionGet();
@ -137,7 +141,7 @@ public class LdapRealmTests extends LdapTestCase {
future.actionGet();
//verify one and only one session -> caching is working
verify(ldapFactory, times(1)).session(anyString(), any(SecuredString.class));
verify(ldapFactory, times(1)).session(anyString(), any(SecuredString.class), any(ActionListener.class));
roleMapper.notifyRefresh();
@ -146,7 +150,7 @@ public class LdapRealmTests extends LdapTestCase {
future.actionGet();
//we need to session again
verify(ldapFactory, times(2)).session(anyString(), any(SecuredString.class));
verify(ldapFactory, times(2)).session(anyString(), any(SecuredString.class), any(ActionListener.class));
}
public void testAuthenticateNoncaching() throws Exception {
@ -160,7 +164,8 @@ public class LdapRealmTests extends LdapTestCase {
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
ldapFactory = spy(ldapFactory);
LdapRealm ldap = new LdapRealm(config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService));
LdapRealm ldap =
new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory, buildGroupAsRoleMapper(resourceWatcherService), threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
ldap.authenticate(new UsernamePasswordToken(VALID_USERNAME, SecuredStringTests.build(PASSWORD)), future);
future.actionGet();
@ -169,7 +174,7 @@ public class LdapRealmTests extends LdapTestCase {
future.actionGet();
//verify two and only two binds -> caching is disabled
verify(ldapFactory, times(2)).session(anyString(), any(SecuredString.class));
verify(ldapFactory, times(2)).session(anyString(), any(SecuredString.class), any(ActionListener.class));
}
public void testLdapRealmSelectsLdapSessionFactory() throws Exception {
@ -183,7 +188,7 @@ public class LdapRealmTests extends LdapTestCase {
.put(HOSTNAME_VERIFICATION_SETTING, false)
.build();
RealmConfig config = new RealmConfig("test-ldap-realm", settings, globalSettings);
SessionFactory sessionFactory = LdapRealm.sessionFactory(config, null);
SessionFactory sessionFactory = LdapRealm.sessionFactory(config, null, LdapRealm.LDAP_TYPE);
assertThat(sessionFactory, is(instanceOf(LdapSessionFactory.class)));
}
@ -199,7 +204,7 @@ public class LdapRealmTests extends LdapTestCase {
.put(HOSTNAME_VERIFICATION_SETTING, false)
.build();
RealmConfig config = new RealmConfig("test-ldap-realm-user-search", settings, globalSettings);
SessionFactory sessionFactory = LdapRealm.sessionFactory(config, null);
SessionFactory sessionFactory = LdapRealm.sessionFactory(config, null, LdapRealm.LDAP_TYPE);
try {
assertThat(sessionFactory, is(instanceOf(LdapUserSearchSessionFactory.class)));
} finally {
@ -217,12 +222,9 @@ public class LdapRealmTests extends LdapTestCase {
.put(HOSTNAME_VERIFICATION_SETTING, false)
.build();
RealmConfig config = new RealmConfig("test-ldap-realm-user-search", settings, globalSettings);
try {
LdapRealm.sessionFactory(config, null);
fail("an exception should have been thrown because both user template and user search settings were specified");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("settings were found for both user search and user template"));
}
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> LdapRealm.sessionFactory(config, null, LdapRealm.LDAP_TYPE));
assertThat(e.getMessage(), containsString("settings were found for both user search and user template"));
}
public void testLdapRealmMapsUserDNToRole() throws Exception {
@ -236,7 +238,8 @@ public class LdapRealmTests extends LdapTestCase {
RealmConfig config = new RealmConfig("test-ldap-realm-userdn", settings, globalSettings);
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
LdapRealm ldap = new LdapRealm(config, ldapFactory, new DnRoleMapper(LdapRealm.TYPE, config, resourceWatcherService, null));
LdapRealm ldap = new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory,
new DnRoleMapper(LdapRealm.LDAP_TYPE, config, resourceWatcherService, null), threadPool);
PlainActionFuture<User> future = new PlainActionFuture<>();
ldap.authenticate(new UsernamePasswordToken("Horatio Hornblower", SecuredStringTests.build(PASSWORD)), future);
@ -266,7 +269,8 @@ public class LdapRealmTests extends LdapTestCase {
RealmConfig config = new RealmConfig("ldap-realm", settings.build(), globalSettings);
LdapSessionFactory ldapFactory = new LdapSessionFactory(config, null);
LdapRealm realm = new LdapRealm(config, ldapFactory, new DnRoleMapper(LdapRealm.TYPE, config, resourceWatcherService, null));
LdapRealm realm = new LdapRealm(LdapRealm.LDAP_TYPE, config, ldapFactory,
new DnRoleMapper(LdapRealm.LDAP_TYPE, config, resourceWatcherService, null), threadPool);
Map<String, Object> stats = realm.usageStats();
assertThat(stats, is(notNullValue()));

View File

@ -9,6 +9,7 @@ import com.unboundid.ldap.listener.InMemoryDirectoryServer;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPURL;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.UncategorizedExecutionException;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
@ -20,6 +21,7 @@ import org.elasticsearch.test.junit.annotations.Network;
import org.junit.Before;
import java.util.List;
import java.util.concurrent.ExecutionException;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.contains;
@ -54,11 +56,12 @@ public class LdapSessionFactoryTests extends LdapTestCase {
SecuredString userPass = SecuredStringTests.build("pass");
ldapServer.setProcessingDelayMillis(500L);
try (LdapSession session = sessionFactory.session(user, userPass)) {
fail("expected connection timeout error here");
} catch (Exception e) {
assertThat(e, instanceOf(LDAPException.class));
assertThat(e.getMessage(), containsString("A client-side timeout was encountered while waiting "));
try {
UncategorizedExecutionException e =
expectThrows(UncategorizedExecutionException.class, () -> session(sessionFactory, user, userPass));
assertThat(e.getCause(), instanceOf(ExecutionException.class));
assertThat(e.getCause().getCause(), instanceOf(LDAPException.class));
assertThat(e.getCause().getCause().getMessage(), containsString("A client-side timeout was encountered while waiting "));
} finally {
ldapServer.setProcessingDelayMillis(0L);
}
@ -83,7 +86,7 @@ public class LdapSessionFactoryTests extends LdapTestCase {
SecuredString userPass = SecuredStringTests.build("pass");
long start = System.currentTimeMillis();
LDAPException expected = expectThrows(LDAPException.class, () -> sessionFactory.session(user, userPass));
LDAPException expected = expectThrows(LDAPException.class, () -> session(sessionFactory, user, userPass));
long time = System.currentTimeMillis() - start;
assertThat(time, lessThan(10000L));
assertThat(expected, instanceOf(LDAPException.class));
@ -106,7 +109,7 @@ public class LdapSessionFactoryTests extends LdapTestCase {
String user = "Horatio Hornblower";
SecuredString userPass = SecuredStringTests.build("pass");
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
try (LdapSession ldap = session(sessionFactory, user, userPass)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
@ -126,9 +129,11 @@ public class LdapSessionFactoryTests extends LdapTestCase {
String user = "Horatio Hornblower";
SecuredString userPass = SecuredStringTests.build("pass");
LDAPException expected = expectThrows(LDAPException.class, () -> ldapFac.session(user, userPass));
assertThat(expected.getMessage(), containsString("Unable to bind as user"));
Throwable[] suppressed = expected.getSuppressed();
UncategorizedExecutionException e = expectThrows(UncategorizedExecutionException.class, () -> session(ldapFac, user, userPass));
assertThat(e.getCause(), instanceOf(ExecutionException.class));
assertThat(e.getCause().getCause(), instanceOf(LDAPException.class));
assertThat(e.getCause().getCause().getMessage(), containsString("Unable to bind as user"));
Throwable[] suppressed = e.getCause().getCause().getSuppressed();
assertThat(suppressed.length, is(2));
}
@ -143,8 +148,8 @@ public class LdapSessionFactoryTests extends LdapTestCase {
String user = "Horatio Hornblower";
SecuredString userPass = SecuredStringTests.build("pass");
try (LdapSession ldap = ldapFac.session(user, userPass)) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(ldapFac, user, userPass)) {
List<String> groups = groups(ldap);
assertThat(groups, contains("cn=HMS Lydia,ou=crews,ou=groups,o=sevenSeas"));
}
}
@ -158,8 +163,8 @@ public class LdapSessionFactoryTests extends LdapTestCase {
LdapSessionFactory ldapFac = new LdapSessionFactory(config, null);
String user = "Horatio Hornblower";
try (LdapSession ldap = ldapFac.session(user, SecuredStringTests.build("pass"))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(ldapFac, user, SecuredStringTests.build("pass"))) {
List<String> groups = groups(ldap);
assertThat(groups, contains("cn=HMS Lydia,ou=crews,ou=groups,o=sevenSeas"));
}
}
@ -175,8 +180,8 @@ public class LdapSessionFactoryTests extends LdapTestCase {
String user = "Horatio Hornblower";
SecuredString userPass = SecuredStringTests.build("pass");
try (LdapSession ldap = ldapFac.session(user, userPass)) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(ldapFac, user, userPass)) {
List<String> groups = groups(ldap);
assertThat(groups.size(), is(1));
assertThat(groups, contains("cn=HMS Lydia,ou=crews,ou=groups,o=sevenSeas"));
}

View File

@ -13,13 +13,11 @@ import com.unboundid.ldap.sdk.LDAPConnectionPoolHealthCheck;
import com.unboundid.ldap.sdk.LDAPURL;
import com.unboundid.ldap.sdk.SimpleBindRequest;
import com.unboundid.ldap.sdk.SingleServerSet;
import org.elasticsearch.ElasticsearchSecurityException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.Environment;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectorySessionFactoryTests;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapTestCase;
@ -35,7 +33,6 @@ import java.text.MessageFormat;
import java.util.List;
import java.util.Locale;
import static org.elasticsearch.test.SecurityTestsUtils.assertAuthenticationException;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
@ -103,13 +100,13 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
try {
// auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
try (LdapSession ldap = session(sessionFactory, user, userPass)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
try (LdapSession ldap = unauthenticatedSession(sessionFactory, user)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
@ -138,21 +135,8 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
SecuredString userPass = SecuredStringTests.build("pass");
try {
//auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
fail("the user should not have been found");
} catch (ElasticsearchSecurityException e) {
assertAuthenticationException(e, containsString("failed to find user [William Bush] with search base [o=sevenSeas] scope " +
"[base]"));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
fail("the user should not have been found");
} catch (ElasticsearchSecurityException e) {
assertAuthenticationException(e, containsString("failed to find user [William Bush] with search base [o=sevenSeas] scope " +
"[base]"));
}
assertNull(session(sessionFactory, user, userPass));
assertNull(unauthenticatedSession(sessionFactory, user));
} finally {
sessionFactory.shutdown();
}
@ -179,13 +163,13 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
try {
// auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
try (LdapSession ldap = session(sessionFactory, user, userPass)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
try (LdapSession ldap = unauthenticatedSession(sessionFactory, user)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
@ -214,21 +198,8 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
SecuredString userPass = SecuredStringTests.build("pass");
try {
// auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
fail("the user should not have been found");
} catch (ElasticsearchSecurityException e) {
assertAuthenticationException(e, containsString("failed to find user [William Bush] with search base [o=sevenSeas] scope " +
"[one_level]"));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
fail("the user should not have been found");
} catch (ElasticsearchSecurityException e) {
assertAuthenticationException(e, containsString("failed to find user [William Bush] with search base [o=sevenSeas] scope " +
"[one_level]"));
}
assertNull(session(sessionFactory, user, userPass));
assertNull(unauthenticatedSession(sessionFactory, user));
} finally {
sessionFactory.shutdown();
}
@ -255,13 +226,13 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
try {
//auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
try (LdapSession ldap = session(sessionFactory, user, userPass)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
try (LdapSession ldap = unauthenticatedSession(sessionFactory, user)) {
String dn = ldap.userDn();
assertThat(dn, containsString(user));
}
@ -289,22 +260,9 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
SecuredString userPass = SecuredStringTests.build("pass");
try {
//auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
fail("the user should not have been found");
} catch (ElasticsearchSecurityException e) {
assertAuthenticationException(e, containsString("failed to find user [William Bush] with search base [o=sevenSeas] scope " +
"[sub_tree]"));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
fail("the user should not have been found");
} catch (ElasticsearchSecurityException e) {
assertAuthenticationException(e, containsString("failed to find user [William Bush] with search base [o=sevenSeas] scope " +
"[sub_tree]"));
}
}finally {
assertNull(session(sessionFactory, user, userPass));
assertNull(unauthenticatedSession(sessionFactory, user));
} finally {
sessionFactory.shutdown();
}
}
@ -328,13 +286,13 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
try {
//auth
try (LdapSession ldap = sessionFactory.session(user, userPass)) {
try (LdapSession ldap = session(sessionFactory, user, userPass)) {
String dn = ldap.userDn();
assertThat(dn, containsString("William Bush"));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
try (LdapSession ldap = unauthenticatedSession(sessionFactory, user)) {
String dn = ldap.userDn();
assertThat(dn, containsString("William Bush"));
}
@ -362,8 +320,8 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
String user = "Bruce Banner";
try {
//auth
try (LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(ActiveDirectorySessionFactoryTests.PASSWORD))) {
List<String> groups = ldap.groups();
try (LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(ActiveDirectorySessionFactoryTests.PASSWORD))) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
@ -373,8 +331,8 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
List<String> groups = ldap.groups();
try (LdapSession ldap = unauthenticatedSession(sessionFactory, user)) {
List<String> groups = groups(ldap);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
@ -405,17 +363,17 @@ public class LdapUserSearchSessionFactoryTests extends LdapTestCase {
try {
for (String user : users) {
//auth
try (LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(OpenLdapTests.PASSWORD))) {
try (LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(OpenLdapTests.PASSWORD))) {
assertThat(ldap.userDn(), is(equalTo(new MessageFormat("uid={0},ou=people,dc=oldap,dc=test,dc=elasticsearch,dc=com",
Locale.ROOT).format(new Object[]{user}, new StringBuffer(), null).toString())));
assertThat(ldap.groups(), hasItem(containsString("Avengers")));
assertThat(groups(ldap), hasItem(containsString("Avengers")));
}
//lookup
try (LdapSession ldap = sessionFactory.unauthenticatedSession(user)) {
try (LdapSession ldap = unauthenticatedSession(sessionFactory, user)) {
assertThat(ldap.userDn(), is(equalTo(new MessageFormat("uid={0},ou=people,dc=oldap,dc=test,dc=elasticsearch,dc=com",
Locale.ROOT).format(new Object[]{user}, new StringBuffer(), null).toString())));
assertThat(ldap.groups(), hasItem(containsString("Avengers")));
assertThat(groups(ldap), hasItem(containsString("Avengers")));
}
}
} finally {

View File

@ -6,13 +6,16 @@
package org.elasticsearch.xpack.security.authc.ldap;
import com.unboundid.ldap.sdk.LDAPException;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.UncategorizedExecutionException;
import org.elasticsearch.env.Environment;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSession;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapTestCase;
import org.elasticsearch.xpack.security.authc.ldap.support.SessionFactory;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.xpack.security.authc.support.SecuredStringTests;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.junit.annotations.Network;
@ -20,10 +23,13 @@ import org.elasticsearch.xpack.ssl.SSLService;
import org.junit.Before;
import java.nio.file.Path;
import java.util.List;
import java.util.concurrent.ExecutionException;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.instanceOf;
@Network
public class OpenLdapTests extends ESTestCase {
@ -68,8 +74,8 @@ public class OpenLdapTests extends ESTestCase {
String[] users = new String[] { "blackwidow", "cap", "hawkeye", "hulk", "ironman", "thor" };
for (String user : users) {
try (LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(PASSWORD))) {
assertThat(ldap.groups(), hasItem(containsString("Avengers")));
try (LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(PASSWORD))) {
assertThat(groups(ldap), hasItem(containsString("Avengers")));
}
}
}
@ -85,8 +91,8 @@ public class OpenLdapTests extends ESTestCase {
String[] users = new String[] { "blackwidow", "cap", "hawkeye", "hulk", "ironman", "thor" };
for (String user : users) {
LdapSession ldap = sessionFactory.session(user, SecuredStringTests.build(PASSWORD));
assertThat(ldap.groups(), hasItem(containsString("Avengers")));
LdapSession ldap = session(sessionFactory, user, SecuredStringTests.build(PASSWORD));
assertThat(groups(ldap), hasItem(containsString("Avengers")));
ldap.close();
}
}
@ -102,8 +108,8 @@ public class OpenLdapTests extends ESTestCase {
RealmConfig config = new RealmConfig("oldap-test", settings, globalSettings);
LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService);
try (LdapSession ldap = sessionFactory.session("selvig", SecuredStringTests.build(PASSWORD))){
assertThat(ldap.groups(), hasItem(containsString("Geniuses")));
try (LdapSession ldap = session(sessionFactory, "selvig", SecuredStringTests.build(PASSWORD))){
assertThat(groups(ldap), hasItem(containsString("Geniuses")));
}
}
@ -121,7 +127,7 @@ public class OpenLdapTests extends ESTestCase {
LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService);
LDAPException expected = expectThrows(LDAPException.class,
() -> sessionFactory.session("thor", SecuredStringTests.build(PASSWORD)).groups());
() -> session(sessionFactory, "thor", SecuredStringTests.build(PASSWORD)).groups(new PlainActionFuture<>()));
assertThat(expected.getMessage(), containsString("A client-side timeout was encountered while waiting"));
}
@ -138,8 +144,12 @@ public class OpenLdapTests extends ESTestCase {
LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService);
String user = "blackwidow";
LDAPException expected = expectThrows(LDAPException.class, () -> sessionFactory.session(user, SecuredStringTests.build(PASSWORD)));
assertThat(expected.getMessage(), anyOf(containsString("Hostname verification failed"), containsString("peer not authenticated")));
UncategorizedExecutionException e = expectThrows(UncategorizedExecutionException.class,
() -> session(sessionFactory, user, SecuredStringTests.build(PASSWORD)));
assertThat(e.getCause(), instanceOf(ExecutionException.class));
assertThat(e.getCause().getCause(), instanceOf(LDAPException.class));
assertThat(e.getCause().getCause().getMessage(),
anyOf(containsString("Hostname verification failed"), containsString("peer not authenticated")));
}
Settings buildLdapSettings(String ldapUrl, String userTemplate, String groupSearchBase, LdapSearchScope scope) {
@ -153,4 +163,16 @@ public class OpenLdapTests extends ESTestCase {
.put("ssl.truststore.password", "changeit")
.build();
}
protected LdapSession session(SessionFactory factory, String username, SecuredString password) {
PlainActionFuture<LdapSession> future = new PlainActionFuture<>();
factory.session(username, password, future);
return future.actionGet();
}
protected List<String> groups(LdapSession ldapSession) {
PlainActionFuture<List<String>> future = new PlainActionFuture<>();
ldapSession.groups(future);
return future.actionGet();
}
}

View File

@ -5,6 +5,7 @@
*/
package org.elasticsearch.xpack.security.authc.ldap;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.ldap.support.LdapSearchScope;
@ -30,7 +31,8 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
List<String> groups =
resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
containsString("SHIELD"),
@ -45,7 +47,8 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
List<String> groups =
resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
containsString("SHIELD"),
@ -60,7 +63,8 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
List<String> groups =
resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
assertThat(groups, hasItem(containsString("Avengers")));
}
@ -72,7 +76,8 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, "uid=selvig,ou=people,dc=oldap,dc=test,dc=elasticsearch,dc=com",
List<String> groups =
resolveBlocking(resolver, ldapConnection, "uid=selvig,ou=people,dc=oldap,dc=test,dc=elasticsearch,dc=com",
TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
assertThat(groups, hasItem(containsString("Geniuses")));
}
@ -84,7 +89,8 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, "uid=selvig,ou=people,dc=oldap,dc=test,dc=elasticsearch,dc=com",
List<String> groups =
resolveBlocking(resolver, ldapConnection, "uid=selvig,ou=people,dc=oldap,dc=test,dc=elasticsearch,dc=com",
TimeValue.timeValueSeconds(10), NoOpLogger.INSTANCE, null);
assertThat(groups, hasItem(containsString("Geniuses")));
}
@ -107,8 +113,9 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.put("base_dn", "dc=oldap,dc=test,dc=elasticsearch,dc=com")
.put("user_attribute", "uid").build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
assertThat(resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), NoOpLogger.INSTANCE),
is("hulk"));
PlainActionFuture<String> future = new PlainActionFuture<>();
resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), future);
assertThat(future.actionGet(), is("hulk"));
}
public void testReadUserAttributeCn() throws Exception {
@ -117,8 +124,10 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.put("user_attribute", "cn")
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
assertThat(resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), NoOpLogger.INSTANCE),
is("Bruce Banner"));
PlainActionFuture<String> future = new PlainActionFuture<>();
resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), future);
assertThat(future.actionGet(), is("Bruce Banner"));
}
public void testReadNonExistentUserAttribute() throws Exception {
@ -127,7 +136,10 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.put("user_attribute", "doesntExists")
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
assertNull(resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), NoOpLogger.INSTANCE));
PlainActionFuture<String> future = new PlainActionFuture<>();
resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), future);
assertNull(future.actionGet());
}
public void testReadBinaryUserAttribute() throws Exception {
@ -136,7 +148,10 @@ public class SearchGroupsResolverTests extends GroupsResolverTestCase {
.put("user_attribute", "userPassword")
.build();
SearchGroupsResolver resolver = new SearchGroupsResolver(settings);
String attribute = resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), NoOpLogger.INSTANCE);
PlainActionFuture<String> future = new PlainActionFuture<>();
resolver.readUserAttribute(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(5), future);
String attribute = future.actionGet();
assertThat(attribute, is(notNullValue()));
}

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.security.authc.ldap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectorySessionFactoryTests;
import org.elasticsearch.xpack.security.support.NoOpLogger;
import org.elasticsearch.test.junit.annotations.Network;
@ -26,7 +25,8 @@ public class UserAttributeGroupsResolverTests extends GroupsResolverTestCase {
public void testResolve() throws Exception {
//falling back on the 'memberOf' attribute
UserAttributeGroupsResolver resolver = new UserAttributeGroupsResolver(Settings.EMPTY);
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(20), NoOpLogger.INSTANCE, null);
List<String> groups =
resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(20), NoOpLogger.INSTANCE, null);
assertThat(groups, containsInAnyOrder(
containsString("Avengers"),
containsString("SHIELD"),
@ -39,7 +39,8 @@ public class UserAttributeGroupsResolverTests extends GroupsResolverTestCase {
.put("user_group_attribute", "seeAlso")
.build();
UserAttributeGroupsResolver resolver = new UserAttributeGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(20), NoOpLogger.INSTANCE, null);
List<String> groups =
resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(20), NoOpLogger.INSTANCE, null);
assertThat(groups, hasItems(containsString("Avengers"))); //seeAlso only has Avengers
}
@ -48,7 +49,8 @@ public class UserAttributeGroupsResolverTests extends GroupsResolverTestCase {
.put("user_group_attribute", "doesntExist")
.build();
UserAttributeGroupsResolver resolver = new UserAttributeGroupsResolver(settings);
List<String> groups = resolver.resolve(ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(20), NoOpLogger.INSTANCE, null);
List<String> groups =
resolveBlocking(resolver, ldapConnection, BRUCE_BANNER_DN, TimeValue.timeValueSeconds(20), NoOpLogger.INSTANCE, null);
assertThat(groups, empty());
}

View File

@ -9,6 +9,7 @@ import com.unboundid.ldap.listener.InMemoryDirectoryServer;
import com.unboundid.ldap.sdk.Attribute;
import com.unboundid.ldap.sdk.LDAPException;
import com.unboundid.ldap.sdk.LDAPURL;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.security.authc.RealmConfig;
@ -16,12 +17,14 @@ import org.elasticsearch.xpack.security.authc.ldap.LdapRealm;
import org.elasticsearch.xpack.security.authc.support.DnRoleMapper;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import static org.elasticsearch.xpack.security.authc.ldap.LdapSessionFactory.HOSTNAME_VERIFICATION_SETTING;
import static org.elasticsearch.xpack.security.authc.ldap.LdapSessionFactory.URLS_SETTING;
@ -29,7 +32,7 @@ import static org.elasticsearch.xpack.security.authc.ldap.LdapSessionFactory.USE
public abstract class LdapTestCase extends ESTestCase {
protected static int numberOfLdapServers;
static int numberOfLdapServers;
protected InMemoryDirectoryServer[] ldapServers;
@BeforeClass
@ -109,6 +112,25 @@ public abstract class LdapTestCase extends ESTestCase {
Settings global = Settings.builder().put("path.home", createTempDir()).build();
RealmConfig config = new RealmConfig("ldap1", settings, global);
return new DnRoleMapper(LdapRealm.TYPE, config, resourceWatcherService, null);
return new DnRoleMapper(LdapRealm.LDAP_TYPE, config, resourceWatcherService, () -> {});
}
protected LdapSession session(SessionFactory factory, String username, SecuredString password) {
PlainActionFuture<LdapSession> future = new PlainActionFuture<>();
factory.session(username, password, future);
return future.actionGet();
}
protected List<String> groups(LdapSession ldapSession) {
Objects.requireNonNull(ldapSession);
PlainActionFuture<List<String>> future = new PlainActionFuture<>();
ldapSession.groups(future);
return future.actionGet();
}
protected LdapSession unauthenticatedSession(SessionFactory factory, String username) {
PlainActionFuture<LdapSession> future = new PlainActionFuture<>();
factory.unauthenticatedSession(username, future);
return future.actionGet();
}
}

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.security.authc.ldap.support;
import com.unboundid.ldap.listener.InMemoryDirectoryServer;
import com.unboundid.ldap.sdk.LDAPConnection;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
@ -176,8 +177,8 @@ public class SessionFactoryLoadBalancingTests extends LdapTestCase {
}
@Override
protected LdapSession getSession(String user, SecuredString password) throws Exception {
return null;
public void session(String user, SecuredString password, ActionListener<LdapSession> listener) {
listener.onResponse(null);
}
}
}

View File

@ -8,11 +8,14 @@ package org.elasticsearch.xpack.security.authc.ldap.support;
import com.unboundid.ldap.sdk.LDAPConnectionOptions;
import com.unboundid.util.ssl.HostNameSSLSocketVerifier;
import com.unboundid.util.ssl.TrustAllSSLSocketVerifier;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.support.SecuredString;
import org.elasticsearch.test.ESTestCase;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
@ -47,12 +50,9 @@ public class SessionFactoryTests extends ESTestCase {
}
public void testUnauthenticatedSessionThrowsUnsupportedOperationException() throws Exception {
try {
createSessionFactory().unauthenticatedSession(randomAsciiOfLength(5));
fail("session factory should throw an unsupported operation exception");
} catch (UnsupportedOperationException e) {
// expected...
}
UnsupportedOperationException e = expectThrows(UnsupportedOperationException.class,
() -> createSessionFactory().unauthenticatedSession(randomAsciiOfLength(5), new PlainActionFuture<>()));
assertThat(e.getMessage(), containsString("unauthenticated sessions"));
}
private SessionFactory createSessionFactory() {
@ -60,8 +60,8 @@ public class SessionFactoryTests extends ESTestCase {
return new SessionFactory(new RealmConfig("_name", Settings.builder().put("url", "ldap://localhost:389").build(), global), null) {
@Override
protected LdapSession getSession(String user, SecuredString password) {
return null;
public void session(String user, SecuredString password, ActionListener<LdapSession> listener) {
listener.onResponse(null);
}
};
}

View File

@ -8,12 +8,10 @@ package org.elasticsearch.xpack.security.authc.support;
import com.unboundid.ldap.sdk.DN;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.core.LogEvent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.xpack.security.audit.logfile.CapturingLogger;
import org.elasticsearch.xpack.security.authc.RealmConfig;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectoryRealm;
import org.elasticsearch.xpack.security.authc.ldap.LdapRealm;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
@ -101,12 +99,7 @@ public class DnRoleMapperTests extends ESTestCase {
ResourceWatcherService watcherService = new ResourceWatcherService(settings, threadPool);
DnRoleMapper mapper = createMapper(file, watcherService);
mapper.addListener(new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
mapper.addListener(latch::countDown);
Set<String> roles = mapper.resolveRoles("", Collections.singletonList("cn=shield,ou=marvel,o=superheros"));
assertThat(roles, notNullValue());
@ -140,12 +133,7 @@ public class DnRoleMapperTests extends ESTestCase {
ResourceWatcherService watcherService = new ResourceWatcherService(settings, threadPool);
DnRoleMapper mapper = createMapper(file, watcherService);
mapper.addListener(new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
mapper.addListener(latch::countDown);
Set<String> roles = mapper.resolveRoles("", Collections.singletonList("cn=shield,ou=marvel,o=superheros"));
assertThat(roles, notNullValue());
@ -246,7 +234,7 @@ public class DnRoleMapperTests extends ESTestCase {
.build();
RealmConfig config = new RealmConfig("ldap1", ldapSettings, settings);
DnRoleMapper mapper = new DnRoleMapper(LdapRealm.TYPE, config, new ResourceWatcherService(settings, threadPool), null);
DnRoleMapper mapper = new DnRoleMapper(LdapRealm.LDAP_TYPE, config, new ResourceWatcherService(settings, threadPool), null);
Set<String> roles = mapper.resolveRoles("", Arrays.asList(STARK_GROUP_DNS));
@ -260,7 +248,7 @@ public class DnRoleMapperTests extends ESTestCase {
.build();
RealmConfig config = new RealmConfig("ldap1", ldapSettings, settings);
DnRoleMapper mapper = new DnRoleMapper(LdapRealm.TYPE, config, new ResourceWatcherService(settings, threadPool), null);
DnRoleMapper mapper = new DnRoleMapper(LdapRealm.LDAP_TYPE, config, new ResourceWatcherService(settings, threadPool), null);
Set<String> roles = mapper.resolveRoles("", Arrays.asList(STARK_GROUP_DNS));
assertThat(roles, hasItems("genius", "billionaire", "playboy", "philanthropist", "shield", "avengers"));
@ -274,7 +262,7 @@ public class DnRoleMapperTests extends ESTestCase {
.build();
RealmConfig config = new RealmConfig("ldap-userdn-role", ldapSettings, settings);
DnRoleMapper mapper = new DnRoleMapper(LdapRealm.TYPE, config, new ResourceWatcherService(settings, threadPool), null);
DnRoleMapper mapper = new DnRoleMapper(LdapRealm.LDAP_TYPE, config, new ResourceWatcherService(settings, threadPool), null);
Set<String> roles = mapper.resolveRoles("cn=Horatio Hornblower,ou=people,o=sevenSeas", Collections.<String>emptyList());
assertThat(roles, hasItem("avenger"));
@ -285,6 +273,6 @@ public class DnRoleMapperTests extends ESTestCase {
.put("files.role_mapping", file.toAbsolutePath())
.build();
RealmConfig config = new RealmConfig("ad-group-mapper-test", realmSettings, settings, env);
return new DnRoleMapper(randomBoolean() ? ActiveDirectoryRealm.TYPE : LdapRealm.TYPE, config, watcherService, null);
return new DnRoleMapper(randomBoolean() ? LdapRealm.AD_TYPE : LdapRealm.LDAP_TYPE, config, watcherService, () -> {});
}
}

View File

@ -14,7 +14,6 @@ import org.elasticsearch.xpack.security.action.user.AuthenticateRequestBuilder;
import org.elasticsearch.xpack.security.action.user.ChangePasswordRequestBuilder;
import org.elasticsearch.xpack.security.authc.Authentication;
import org.elasticsearch.xpack.security.authc.Authentication.RealmRef;
import org.elasticsearch.xpack.security.authc.activedirectory.ActiveDirectoryRealm;
import org.elasticsearch.xpack.security.authc.esnative.NativeRealm;
import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm;
import org.elasticsearch.xpack.security.authc.file.FileRealm;
@ -36,7 +35,6 @@ import java.util.Iterator;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.verifyZeroInteractions;
@ -165,7 +163,7 @@ public class DefaultRoleTests extends ESTestCase {
when(authentication.getRunAsUser()).thenReturn(user);
when(authentication.isRunAs()).thenReturn(false);
when(authentication.getAuthenticatedBy()).thenReturn(authenticatedBy);
when(authenticatedBy.getType()).thenReturn(randomFrom(LdapRealm.TYPE, FileRealm.TYPE, ActiveDirectoryRealm.TYPE, PkiRealm.TYPE,
when(authenticatedBy.getType()).thenReturn(randomFrom(LdapRealm.LDAP_TYPE, FileRealm.TYPE, LdapRealm.AD_TYPE, PkiRealm.TYPE,
randomAsciiOfLengthBetween(4, 12)));
assertThat(request, instanceOf(UserRequest.class));
@ -190,7 +188,7 @@ public class DefaultRoleTests extends ESTestCase {
when(authentication.isRunAs()).thenReturn(true);
when(authentication.getAuthenticatedBy()).thenReturn(authenticatedBy);
when(authentication.getLookedUpBy()).thenReturn(lookedUpBy);
when(lookedUpBy.getType()).thenReturn(randomFrom(LdapRealm.TYPE, FileRealm.TYPE, ActiveDirectoryRealm.TYPE, PkiRealm.TYPE,
when(lookedUpBy.getType()).thenReturn(randomFrom(LdapRealm.LDAP_TYPE, FileRealm.TYPE, LdapRealm.AD_TYPE, PkiRealm.TYPE,
randomAsciiOfLengthBetween(4, 12)));
assertThat(request, instanceOf(UserRequest.class));

View File

@ -16,7 +16,6 @@ import org.elasticsearch.watcher.ResourceWatcherService;
import org.elasticsearch.xpack.XPackPlugin;
import org.elasticsearch.xpack.XPackSettings;
import org.elasticsearch.xpack.security.audit.logfile.CapturingLogger;
import org.elasticsearch.xpack.security.authc.support.RefreshListener;
import org.elasticsearch.xpack.security.authz.RoleDescriptor;
import org.elasticsearch.xpack.security.authz.permission.ClusterPermission;
import org.elasticsearch.xpack.security.authz.permission.IndicesPermission;
@ -268,12 +267,7 @@ public class FileRolesStoreTests extends ESTestCase {
threadPool = new TestThreadPool("test");
watcherService = new ResourceWatcherService(settings, threadPool);
final CountDownLatch latch = new CountDownLatch(1);
FileRolesStore store = new FileRolesStore(settings, env, watcherService, new RefreshListener() {
@Override
public void onRefresh() {
latch.countDown();
}
});
FileRolesStore store = new FileRolesStore(settings, env, watcherService, latch::countDown);
store.start();
Role role = store.role("role1");