Add better authorization for scroll requests and remove signing (elastic/x-pack-elasticsearch#1416)

This commit adds better security for scroll requests in that they are now tied to a single user as
we only authorize the request that creates the scroll. This is accomplished by adding a
SearchOperationListener that listens for new scroll contexts and stores the authentication on the
ScrollContext. Then upon
retrieval of the search context for a query or fetch, the current authentication is compared to the
authentication that was present when the scroll context was created. If the current authentication
belongs to a different user, then a SearchContextMissingException will be thrown to prevent leaking
a valid vs invalid scroll id.

Additionally, signing of a scroll id is only performed when there is a older node in the cluster
that would expect the scroll id to be signed. Once this is backported to 5.x, we can remove this
bwc layer for 6.0/master.

Original commit: elastic/x-pack-elasticsearch@0e5dcafd32
This commit is contained in:
Jay Modi 2017-06-06 10:23:18 -06:00 committed by GitHub
parent 15022670e2
commit 2d893df7e9
14 changed files with 834 additions and 162 deletions

View File

@ -121,6 +121,7 @@ import org.elasticsearch.xpack.security.authz.AuthorizationService;
import org.elasticsearch.xpack.security.authz.RoleDescriptor;
import org.elasticsearch.xpack.security.authz.accesscontrol.OptOutQueryCache;
import org.elasticsearch.xpack.security.authz.accesscontrol.SecurityIndexSearcherWrapper;
import org.elasticsearch.xpack.security.authz.SecuritySearchOperationListener;
import org.elasticsearch.xpack.security.authz.accesscontrol.SetSecurityUserProcessor;
import org.elasticsearch.xpack.security.authz.permission.FieldPermissionsCache;
import org.elasticsearch.xpack.security.authz.store.CompositeRolesStore;
@ -206,6 +207,7 @@ public class Security implements ActionPlugin, IngestPlugin, NetworkPlugin {
private final SetOnce<TransportInterceptor> securityInterceptor = new SetOnce<>();
private final SetOnce<IPFilter> ipFilter = new SetOnce<>();
private final SetOnce<AuthenticationService> authcService = new SetOnce<>();
private final SetOnce<AuditTrailService> auditTrailService = new SetOnce<>();
private final SetOnce<SecurityContext> securityContext = new SetOnce<>();
private final SetOnce<ThreadContext> threadContext = new SetOnce<>();
@ -312,6 +314,7 @@ public class Security implements ActionPlugin, IngestPlugin, NetworkPlugin {
final AuditTrailService auditTrailService =
new AuditTrailService(settings, auditTrails.stream().collect(Collectors.toList()), licenseState);
components.add(auditTrailService);
this.auditTrailService.set(auditTrailService);
final SecurityLifecycleService securityLifecycleService =
new SecurityLifecycleService(settings, clusterService, threadPool, client, licenseState, indexAuditTrail);
@ -512,31 +515,34 @@ public class Security implements ActionPlugin, IngestPlugin, NetworkPlugin {
}
public void onIndexModule(IndexModule module) {
if (enabled == false) {
return;
}
if (enabled) {
assert licenseState != null;
if (XPackSettings.DLS_FLS_ENABLED.get(settings)) {
module.setSearcherWrapper(indexService ->
new SecurityIndexSearcherWrapper(indexService.getIndexSettings(),
shardId -> indexService.newQueryShardContext(shardId.id(),
// we pass a null index reader, which is legal and will disable rewrite optimizations
// based on index statistics, which is probably safer...
null,
() -> {
throw new IllegalArgumentException("permission filters are not allowed to use the current timestamp");
assert licenseState != null;
if (XPackSettings.DLS_FLS_ENABLED.get(settings)) {
module.setSearcherWrapper(indexService ->
new SecurityIndexSearcherWrapper(indexService.getIndexSettings(),
shardId -> indexService.newQueryShardContext(shardId.id(),
// we pass a null index reader, which is legal and will disable rewrite optimizations
// based on index statistics, which is probably safer...
null,
() -> {
throw new IllegalArgumentException("permission filters are not allowed to use the current timestamp");
}),
indexService.cache().bitsetFilterCache(),
indexService.getThreadPool().getThreadContext(), licenseState,
indexService.getScriptService()));
}
if (transportClientMode == false) {
/* We need to forcefully overwrite the query cache implementation to use security's opt out query cache implementation.
* This impl. disabled the query cache if field level security is used for a particular request. If we wouldn't do
* forcefully overwrite the query cache implementation then we leave the system vulnerable to leakages of data to
* unauthorized users. */
module.forceQueryCacheProvider((settings, cache) -> new OptOutQueryCache(settings, cache, threadContext.get()));
}),
indexService.cache().bitsetFilterCache(),
indexService.getThreadPool().getThreadContext(), licenseState,
indexService.getScriptService()));
/* We need to forcefully overwrite the query cache implementation to use security's opt out query cache implementation.
* This impl. disabled the query cache if field level security is used for a particular request. If we wouldn't do
* forcefully overwrite the query cache implementation then we leave the system vulnerable to leakages of data to
* unauthorized users. */
module.forceQueryCacheProvider((settings, cache) -> new OptOutQueryCache(settings, cache, threadContext.get()));
}
// in order to prevent scroll ids from being maliciously crafted and/or guessed, a listener is added that
// attaches information to the scroll context so that we can validate the user that created the scroll against
// the user that is executing a scroll operation
module.addSearchOperationListener(
new SecuritySearchOperationListener(threadContext.get(), licenseState, auditTrailService.get()));
}
}

View File

@ -20,6 +20,8 @@ import org.elasticsearch.action.support.ActionFilter;
import org.elasticsearch.action.support.ActionFilterChain;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -69,12 +71,14 @@ public class SecurityActionFilter extends AbstractComponent implements ActionFil
private final ThreadContext threadContext;
private final SecurityContext securityContext;
private final DestructiveOperations destructiveOperations;
private final ClusterService clusterService;
@Inject
public SecurityActionFilter(Settings settings, AuthenticationService authcService, AuthorizationService authzService,
CryptoService cryptoService, AuditTrailService auditTrail, XPackLicenseState licenseState,
Set<RequestInterceptor> requestInterceptors, ThreadPool threadPool,
SecurityContext securityContext, DestructiveOperations destructiveOperations) {
SecurityContext securityContext, DestructiveOperations destructiveOperations,
ClusterService clusterService) {
super(settings);
this.authcService = authcService;
this.authzService = authzService;
@ -85,6 +89,7 @@ public class SecurityActionFilter extends AbstractComponent implements ActionFil
this.threadContext = threadPool.getThreadContext();
this.securityContext = securityContext;
this.destructiveOperations = destructiveOperations;
this.clusterService = clusterService;
}
@Override
@ -196,23 +201,67 @@ public class SecurityActionFilter extends AbstractComponent implements ActionFil
ActionRequest unsign(User user, String action, final ActionRequest request) {
try {
// In order to provide backwards compatibility with previous versions that always signed scroll ids
// we sign the scroll requests and do not allow unsigned requests until all of the nodes in the cluster
// have been upgraded to a version that does not sign scroll ids and instead relies improved scroll
// authorization. It is important to note that older versions do not actually sign if the system key
// does not exist so we need to take that into account as well.
// TODO update to 5.5 on backport and remove any signing from master!
final ClusterState state = clusterService.state();
final boolean signingRequired = state.nodes().getMinNodeVersion().before(Version.V_6_0_0_alpha2) &&
cryptoService.isSystemKeyPresent();
if (request instanceof SearchScrollRequest) {
SearchScrollRequest scrollRequest = (SearchScrollRequest) request;
String scrollId = scrollRequest.scrollId();
scrollRequest.scrollId(cryptoService.unsignAndVerify(scrollId));
if (signingRequired) {
if (cryptoService.isSigned(scrollId)) {
scrollRequest.scrollId(cryptoService.unsignAndVerify(scrollId));
} else {
logger.error("scroll id [{}] is not signed but is expected to be. nodes [{}], minimum node version [{}]",
scrollId, state.nodes(), state.nodes().getMinNodeVersion());
// if we get a unsigned scroll request and not all nodes are up to date, then we cannot trust
// this scroll id and reject it
auditTrail.tamperedRequest(user, action, request);
throw authorizationError("invalid request");
}
} else if (cryptoService.isSigned(scrollId)) {
// if signing isn't required we could still get a signed ID from an already running scroll or
// a node that hasn't received the current cluster state that shows signing isn't required
scrollRequest.scrollId(cryptoService.unsignAndVerify(scrollId));
}
// else the scroll id is fine on the request so don't do anything
} else if (request instanceof ClearScrollRequest) {
ClearScrollRequest clearScrollRequest = (ClearScrollRequest) request;
boolean isClearAllScrollRequest = clearScrollRequest.scrollIds().contains("_all");
if (!isClearAllScrollRequest) {
final boolean isClearAllScrollRequest = clearScrollRequest.scrollIds().contains("_all");
if (isClearAllScrollRequest == false) {
List<String> signedIds = clearScrollRequest.scrollIds();
List<String> unsignedIds = new ArrayList<>(signedIds.size());
for (String signedId : signedIds) {
unsignedIds.add(cryptoService.unsignAndVerify(signedId));
if (signingRequired) {
if (cryptoService.isSigned(signedId)) {
unsignedIds.add(cryptoService.unsignAndVerify(signedId));
} else {
logger.error("scroll id [{}] is not signed but is expected to be. nodes [{}], minimum node version [{}]",
signedId, state.nodes(), state.nodes().getMinNodeVersion());
// if we get a unsigned scroll request and not all nodes are up to date, then we cannot trust
// this scroll id and reject it
auditTrail.tamperedRequest(user, action, request);
throw authorizationError("invalid request");
}
} else if (cryptoService.isSigned(signedId)) {
// if signing isn't required we could still get a signed ID from an already running scroll or
// a node that hasn't received the current cluster state that shows signing isn't required
unsignedIds.add(cryptoService.unsignAndVerify(signedId));
} else {
// the id is not signed and we allow unsigned requests so just add it
unsignedIds.add(signedId);
}
}
clearScrollRequest.scrollIds(unsignedIds);
}
}
} catch (IllegalArgumentException | IllegalStateException e) {
// this can happen when we decode invalid base64 or get a invalid scroll id
auditTrail.tamperedRequest(user, action, request);
throw authorizationError("invalid request. {}", e.getMessage());
}
@ -221,10 +270,17 @@ public class SecurityActionFilter extends AbstractComponent implements ActionFil
private <Response extends ActionResponse> Response sign(Response response) throws IOException {
if (response instanceof SearchResponse) {
SearchResponse searchResponse = (SearchResponse) response;
String scrollId = searchResponse.getScrollId();
if (scrollId != null && !cryptoService.isSigned(scrollId)) {
searchResponse.scrollId(cryptoService.sign(scrollId));
// In order to provide backwards compatibility with previous versions that always signed scroll ids
// we sign the scroll requests and do not allow unsigned requests until all of the nodes in the cluster
// have been upgraded to a version that supports unsigned scroll ids
final boolean sign = clusterService.state().nodes().getMinNodeVersion().before(Version.V_6_0_0_alpha2);
if (sign) {
SearchResponse searchResponse = (SearchResponse) response;
String scrollId = searchResponse.getScrollId();
if (scrollId != null && !cryptoService.isSigned(scrollId)) {
searchResponse.scrollId(cryptoService.sign(scrollId));
}
}
}
return response;

View File

@ -129,7 +129,7 @@ public class AuthorizationService extends AbstractComponent {
}
request = TransportActionProxy.unwrapRequest(request);
// prior to doing any authorization lets set the originating action in the context only
setOriginatingAction(action);
putTransientIfNonExisting(ORIGINATING_ACTION_KEY, action);
// first we need to check if the user is the system. If it is, we'll just authorize the system access
if (SystemUser.is(authentication.getUser())) {
@ -181,7 +181,7 @@ public class AuthorizationService extends AbstractComponent {
throw new IllegalStateException("Composite actions must implement " + CompositeIndicesRequest.class.getSimpleName()
+ ", " + request.getClass().getSimpleName() + " doesn't");
}
//we check if the user can execute the action, without looking at indices, whici will be authorized at the shard level
// we check if the user can execute the action, without looking at indices, which will be authorized at the shard level
if (permission.indices().check(action)) {
grant(authentication, action, request);
return;
@ -203,19 +203,31 @@ public class AuthorizationService extends AbstractComponent {
// some APIs are indices requests that are not actually associated with indices. For example,
// search scroll request, is categorized under the indices context, but doesn't hold indices names
// (in this case, the security check on the indices was done on the search request that initialized
// the scroll... and we rely on the signed scroll id to provide security over this request).
// so we only check indices if indeed the request is an actual IndicesRequest, if it's not,
// we just grant it if it's a scroll, deny otherwise
// the scroll. Given that scroll is implemented using a context on the node holding the shard, we
// piggyback on it and enhance the context with the original authentication. This serves as our method
// to validate the scroll id only stays with the same user!
if (request instanceof IndicesRequest == false && request instanceof IndicesAliasesRequest == false) {
//note that clear scroll shard level actions can originate from a clear scroll all, which doesn't require any
//indices permission as it's categorized under cluster. This is why the scroll check is performed
//even before checking if the user has any indices permission.
if (isScrollRelatedAction(action)) {
//note that clear scroll shard level actions can originate from a clear scroll all, which doesn't require any
//indices permission as it's categorized under cluster. This is why the scroll check is performed
//even before checking if the user has any indices permission.
grant(authentication, action, request);
return;
// if the action is a search scroll action, we first authorize that the user can execute the action for some
// index and if they cannot, we can fail the request early before we allow the execution of the action and in
// turn the shard actions
if (SearchScrollAction.NAME.equals(action) && permission.indices().check(action) == false) {
throw denial(authentication, action, request);
} else {
// we store the request as a transient in the ThreadContext in case of a authorization failure at the shard
// level. If authorization fails we will audit a access_denied message and will use the request to retrieve
// information such as the index and the incoming address of the request
grant(authentication, action, request);
return;
}
} else {
assert false :
"only scroll related requests are known indices api that don't support retrieving the indices they relate to";
throw denial(authentication, action, request);
}
assert false : "only scroll related requests are known indices api that don't support retrieving the indices they relate to";
throw denial(authentication, action, request);
}
if (permission.indices().check(action) == false) {
@ -289,10 +301,10 @@ public class AuthorizationService extends AbstractComponent {
}
}
private void setOriginatingAction(String action) {
String originatingAction = threadContext.getTransient(ORIGINATING_ACTION_KEY);
if (originatingAction == null) {
threadContext.putTransient(ORIGINATING_ACTION_KEY, action);
private void putTransientIfNonExisting(String key, Object value) {
Object existing = threadContext.getTransient(key);
if (existing == null) {
threadContext.putTransient(key, value);
}
}
@ -403,7 +415,7 @@ public class AuthorizationService extends AbstractComponent {
return ReservedRealm.TYPE.equals(realmType) || NativeRealm.TYPE.equals(realmType);
}
private ElasticsearchSecurityException denial(Authentication authentication, String action, TransportRequest request) {
ElasticsearchSecurityException denial(Authentication authentication, String action, TransportRequest request) {
auditTrail.accessDenied(authentication.getUser(), action, request);
return denialException(authentication, action);
}

View File

@ -0,0 +1,97 @@
/*
* 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.authz;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.index.shard.SearchOperationListener;
import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.search.SearchContextMissingException;
import org.elasticsearch.search.internal.ScrollContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.xpack.security.audit.AuditTrailService;
import org.elasticsearch.xpack.security.authc.Authentication;
import static org.elasticsearch.xpack.security.authz.AuthorizationService.ORIGINATING_ACTION_KEY;
/**
* A {@link SearchOperationListener} that is used to provide authorization for scroll requests.
*
* In order to identify the user associated with a scroll request, we replace the {@link ScrollContext}
* on creation with a custom implementation that holds the {@link Authentication} object. When
* this context is accessed again in {@link SearchOperationListener#onPreQueryPhase(SearchContext)}
* the ScrollContext is inspected for the authentication, which is compared to the currently
* authentication.
*/
public final class SecuritySearchOperationListener implements SearchOperationListener {
private final ThreadContext threadContext;
private final XPackLicenseState licenseState;
private final AuditTrailService auditTrailService;
public SecuritySearchOperationListener(ThreadContext threadContext, XPackLicenseState licenseState, AuditTrailService auditTrail) {
this.threadContext = threadContext;
this.licenseState = licenseState;
this.auditTrailService = auditTrail;
}
/**
* Adds the {@link Authentication} to the {@link ScrollContext}
*/
@Override
public void onNewScrollContext(SearchContext searchContext) {
if (licenseState.isAuthAllowed()) {
searchContext.scrollContext().putInContext(Authentication.AUTHENTICATION_KEY, Authentication.getAuthentication(threadContext));
}
}
/**
* Checks for the {@link ScrollContext} if it exists and compares the {@link Authentication}
* object from the scroll context with the current authentication context
*/
@Override
public void validateSearchContext(SearchContext searchContext, TransportRequest request) {
if (licenseState.isAuthAllowed()) {
if (searchContext.scrollContext() != null) {
final Authentication originalAuth = searchContext.scrollContext().getFromContext(Authentication.AUTHENTICATION_KEY);
final Authentication current = Authentication.getAuthentication(threadContext);
final String action = threadContext.getTransient(ORIGINATING_ACTION_KEY);
ensureAuthenticatedUserIsSame(originalAuth, current, auditTrailService, searchContext.id(), action, request);
}
}
}
/**
* Compares the {@link Authentication} that was stored in the {@link ScrollContext} with the
* current authentication. We cannot guarantee that all of the details of the authentication will
* be the same. Some things that could differ include the roles, the name of the authenticating
* (or lookup) realm. To work around this we compare the username and the originating realm type.
*/
static void ensureAuthenticatedUserIsSame(Authentication original, Authentication current, AuditTrailService auditTrailService,
long id, String action, TransportRequest request) {
// this is really a best effort attempt since we cannot guarantee principal uniqueness
// and realm names can change between nodes.
final boolean samePrincipal = original.getUser().principal().equals(current.getUser().principal());
final boolean sameRealmType;
if (original.getUser().isRunAs()) {
if (current.getUser().isRunAs()) {
sameRealmType = original.getLookedUpBy().getType().equals(current.getLookedUpBy().getType());
} else {
sameRealmType = original.getLookedUpBy().getType().equals(current.getAuthenticatedBy().getType());
}
} else if (current.getUser().isRunAs()) {
sameRealmType = original.getAuthenticatedBy().getType().equals(current.getLookedUpBy().getType());
} else {
sameRealmType = original.getAuthenticatedBy().getType().equals(current.getAuthenticatedBy().getType());
}
final boolean sameUser = samePrincipal && sameRealmType;
if (sameUser == false) {
auditTrailService.accessDenied(current.getUser(), action, request);
throw new SearchContextMissingException(id);
}
}
}

View File

@ -255,6 +255,14 @@ public class CryptoService extends AbstractComponent {
return this.encryptionKey != null;
}
/**
* Flag for callers to determine if values will actually be signed or returned as is
* @return true if values will be signed
*/
public boolean isSystemKeyPresent() {
return this.systemKey != null;
}
private byte[] encryptInternal(byte[] bytes, SecretKey key) {
byte[] iv = new byte[ivLength];
secureRandom.nextBytes(iv);

View File

@ -1,101 +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.integration;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.SecurityIntegTestCase;
import org.elasticsearch.xpack.security.crypto.CryptoService;
import java.util.Locale;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.test.SecurityTestsUtils.assertThrowsAuthorizationException;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
public class ScrollIdSigningTests extends SecurityIntegTestCase {
public void testSearchAndClearScroll() throws Exception {
IndexRequestBuilder[] docs = new IndexRequestBuilder[randomIntBetween(20, 100)];
for (int i = 0; i < docs.length; i++) {
docs[i] = client().prepareIndex("idx", "type").setSource("field", "value");
}
indexRandom(true, docs);
SearchResponse response = client().prepareSearch()
.setQuery(matchAllQuery())
.setScroll(TimeValue.timeValueMinutes(2))
.setSize(randomIntBetween(1, 10)).get();
int hits = 0;
try {
while (true) {
assertSigned(response.getScrollId());
assertHitCount(response, docs.length);
hits += response.getHits().getHits().length;
response = client().prepareSearchScroll(response.getScrollId())
.setScroll(TimeValue.timeValueMinutes(2)).get();
if (response.getHits().getHits().length == 0) {
break;
}
}
assertThat(hits, equalTo(docs.length));
} finally {
clearScroll(response.getScrollId());
}
}
public void testSearchScrollWithTamperedScrollId() throws Exception {
IndexRequestBuilder[] docs = new IndexRequestBuilder[randomIntBetween(20, 100)];
for (int i = 0; i < docs.length; i++) {
docs[i] = client().prepareIndex("idx", "type").setSource("field", "value");
}
indexRandom(true, docs);
SearchResponse response = client().prepareSearch()
.setQuery(matchAllQuery())
.setScroll(TimeValue.timeValueMinutes(2))
.setSize(randomIntBetween(1, 10)).get();
String scrollId = response.getScrollId();
String tamperedScrollId = randomBoolean() ? scrollId.substring(randomIntBetween(1, 10)) :
scrollId + randomAlphaOfLength(randomIntBetween(3, 10));
try {
assertThrowsAuthorizationException(client().prepareSearchScroll(tamperedScrollId).setScroll(TimeValue.timeValueMinutes(2))::get,
equalTo("invalid request. tampered signed text"));
} finally {
clearScroll(scrollId);
}
}
public void testClearScrollWithTamperedScrollId() throws Exception {
IndexRequestBuilder[] docs = new IndexRequestBuilder[randomIntBetween(20, 100)];
for (int i = 0; i < docs.length; i++) {
docs[i] = client().prepareIndex("idx", "type").setSource("field", "value");
}
indexRandom(true, docs);
SearchResponse response = client().prepareSearch()
.setQuery(matchAllQuery())
.setScroll(TimeValue.timeValueMinutes(2))
.setSize(5).get();
String scrollId = response.getScrollId();
String tamperedScrollId = randomBoolean() ? scrollId.substring(randomIntBetween(1, 10)) :
scrollId + randomAlphaOfLength(randomIntBetween(3, 10));
try {
assertThrowsAuthorizationException(client().prepareClearScroll().addScrollId(tamperedScrollId)::get,
equalTo("invalid request. tampered signed text"));
} finally {
clearScroll(scrollId);
}
}
private void assertSigned(String scrollId) {
CryptoService cryptoService = internalCluster().getDataNodeInstance(CryptoService.class);
String message = String.format(Locale.ROOT, "Expected scrollId [%s] to be signed, but was not", scrollId);
assertThat(message, cryptoService.isSigned(scrollId), is(true));
}
}

View File

@ -123,12 +123,9 @@ public class SecurityClearScrollTests extends SecurityIntegTestCase {
private void assertThatScrollIdsDoNotExist(List<String> scrollIds) {
for (String scrollId : scrollIds) {
try {
client().prepareSearchScroll(scrollId).get();
fail("Expected SearchPhaseExecutionException but did not happen");
} catch (SearchPhaseExecutionException expectedException) {
assertThat(expectedException.toString(), containsString("SearchContextMissingException"));
}
SearchPhaseExecutionException expectedException =
expectThrows(SearchPhaseExecutionException.class, () -> client().prepareSearchScroll(scrollId).get());
assertThat(expectedException.toString(), containsString("SearchContextMissingException"));
}
}

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.security.action.filter;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ElasticsearchSecurityException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.MockIndicesRequest;
@ -18,10 +19,16 @@ import org.elasticsearch.action.support.ActionFilterChain;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
@ -36,6 +43,7 @@ import org.elasticsearch.xpack.security.crypto.CryptoService;
import org.elasticsearch.xpack.security.user.SystemUser;
import org.elasticsearch.xpack.security.user.User;
import org.junit.Before;
import org.mockito.ArgumentCaptor;
import java.util.Collections;
import java.util.HashSet;
@ -62,6 +70,7 @@ public class SecurityActionFilterTests extends ESTestCase {
private XPackLicenseState licenseState;
private SecurityActionFilter filter;
private ThreadContext threadContext;
private ClusterService clusterService;
private boolean failDestructiveOperations;
@Before
@ -81,10 +90,20 @@ public class SecurityActionFilterTests extends ESTestCase {
.put(DestructiveOperations.REQUIRES_NAME_SETTING.getKey(), failDestructiveOperations).build();
DestructiveOperations destructiveOperations = new DestructiveOperations(settings,
new ClusterSettings(settings, Collections.singleton(DestructiveOperations.REQUIRES_NAME_SETTING)));
clusterService = mock(ClusterService.class);
ClusterState state = mock(ClusterState.class);
when(clusterService.state()).thenReturn(state);
DiscoveryNodes nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("id1",
new TransportAddress(TransportAddress.META_ADDRESS, randomIntBetween(49000, 65500)), Version.CURRENT))
.add(new DiscoveryNode("id2",
new TransportAddress(TransportAddress.META_ADDRESS, randomIntBetween(49000, 65500)), Version.V_5_5_0))
.build();
when(state.nodes()).thenReturn(nodes);
SecurityContext securityContext = new SecurityContext(settings, threadContext);
filter = new SecurityActionFilter(Settings.EMPTY, authcService, authzService, cryptoService, auditTrail,
licenseState, new HashSet<>(), threadPool, securityContext, destructiveOperations);
licenseState, new HashSet<>(), threadPool, securityContext, destructiveOperations, clusterService);
}
public void testApply() throws Exception {
@ -262,6 +281,7 @@ public class SecurityActionFilterTests extends ESTestCase {
callback.onResponse(authentication);
return Void.TYPE;
}).when(authcService).authenticate(eq("_action"), eq(request), eq(SystemUser.INSTANCE), any(ActionListener.class));
when(cryptoService.isSystemKeyPresent()).thenReturn(true);
when(cryptoService.isSigned("signed_scroll_id")).thenReturn(true);
when(cryptoService.unsignAndVerify("signed_scroll_id")).thenReturn("scroll_id");
final Role empty = Role.EMPTY;
@ -278,6 +298,80 @@ public class SecurityActionFilterTests extends ESTestCase {
verify(chain).proceed(eq(task), eq("_action"), eq(request), isA(ContextPreservingActionListener.class));
}
public void testUnsignedWithOldVersionNode() throws Exception {
DiscoveryNodes nodes = DiscoveryNodes.builder(clusterService.state().nodes())
.add(new DiscoveryNode("id3",
new TransportAddress(TransportAddress.META_ADDRESS, randomIntBetween(49000, 65500)), Version.V_5_4_0))
.build();
when(clusterService.state().nodes()).thenReturn(nodes);
SearchScrollRequest request = new SearchScrollRequest("unsigned");
ActionListener listener = mock(ActionListener.class);
ActionFilterChain chain = mock(ActionFilterChain.class);
User user = mock(User.class);
Task task = mock(Task.class);
Authentication authentication = new Authentication(user, new RealmRef("test", "test", "foo"), null);
doAnswer((i) -> {
ActionListener callback =
(ActionListener) i.getArguments()[3];
callback.onResponse(authentication);
return Void.TYPE;
}).when(authcService).authenticate(eq("_action"), eq(request), eq(SystemUser.INSTANCE), any(ActionListener.class));
when(cryptoService.isSigned("unsigned")).thenReturn(false);
when(cryptoService.isSystemKeyPresent()).thenReturn(true);
final Role empty = Role.EMPTY;
doAnswer((i) -> {
ActionListener callback =
(ActionListener) i.getArguments()[1];
callback.onResponse(empty);
return Void.TYPE;
}).when(authzService).roles(any(User.class), any(ActionListener.class));
filter.apply(task, "_action", request, listener, chain);
ArgumentCaptor<ElasticsearchSecurityException> captor = ArgumentCaptor.forClass(ElasticsearchSecurityException.class);
verify(listener).onFailure(captor.capture());
ElasticsearchSecurityException e = captor.getValue();
assertEquals("invalid request", e.getMessage());
assertEquals(RestStatus.FORBIDDEN, e.status());
verify(authzService).authorize(authentication, "_action", request, empty, null);
verifyZeroInteractions(chain);
}
public void testUnsigned() throws Exception {
DiscoveryNodes nodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("id1", new TransportAddress(TransportAddress.META_ADDRESS, randomIntBetween(49000, 65500)),
Version.V_6_0_0_alpha2))
.add(new DiscoveryNode("id2", new TransportAddress(TransportAddress.META_ADDRESS, randomIntBetween(49000, 65500)),
Version.V_6_0_0_alpha2))
.build();
when(clusterService.state().nodes()).thenReturn(nodes);
SearchScrollRequest request = new SearchScrollRequest("unsigned");
ActionListener listener = mock(ActionListener.class);
ActionFilterChain chain = mock(ActionFilterChain.class);
User user = mock(User.class);
Task task = mock(Task.class);
Authentication authentication = new Authentication(user, new RealmRef("test", "test", "foo"), null);
doAnswer((i) -> {
ActionListener callback =
(ActionListener) i.getArguments()[3];
callback.onResponse(authentication);
return Void.TYPE;
}).when(authcService).authenticate(eq("_action"), eq(request), eq(SystemUser.INSTANCE), any(ActionListener.class));
when(cryptoService.isSigned("unsigned")).thenReturn(false);
when(cryptoService.isSystemKeyPresent()).thenReturn(randomBoolean());
final Role empty = Role.EMPTY;
doAnswer((i) -> {
ActionListener callback =
(ActionListener) i.getArguments()[1];
callback.onResponse(empty);
return Void.TYPE;
}).when(authzService).roles(any(User.class), any(ActionListener.class));
filter.apply(task, "_action", request, listener, chain);
assertThat(request.scrollId(), equalTo("unsigned"));
verify(authzService).authorize(authentication, "_action", request, empty, null);
verify(chain).proceed(eq(task), eq("_action"), eq(request), isA(ContextPreservingActionListener.class));
}
public void testActionSignatureError() throws Exception {
SearchScrollRequest request = new SearchScrollRequest("scroll_id");
ActionListener listener = mock(ActionListener.class);
@ -292,6 +386,7 @@ public class SecurityActionFilterTests extends ESTestCase {
callback.onResponse(authentication);
return Void.TYPE;
}).when(authcService).authenticate(eq("_action"), eq(request), eq(SystemUser.INSTANCE), any(ActionListener.class));
when(cryptoService.isSystemKeyPresent()).thenReturn(true);
when(cryptoService.isSigned("scroll_id")).thenReturn(true);
doThrow(sigException).when(cryptoService).unsignAndVerify("scroll_id");
doAnswer((i) -> {

View File

@ -125,7 +125,7 @@ public class TokenAuthIntegTests extends SecurityIntegTestCase {
// superuser since the role used there is a file based role since we cannot guarantee the superuser role is always available
internalClient().admin().indices().prepareDelete(TokenService.INDEX_NAME).get();
} catch (IndexNotFoundException e) {
logger.warn("securirty index does not exist", e);
logger.warn("security index does not exist", e);
}
}
}

View File

@ -0,0 +1,117 @@
/*
* 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.authz;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.settings.SecureString;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.search.SearchContextMissingException;
import org.elasticsearch.test.SecurityIntegTestCase;
import org.elasticsearch.xpack.security.SecurityLifecycleService;
import org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken;
import org.junit.After;
import java.util.Collections;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
public class SecurityScrollTests extends SecurityIntegTestCase {
public void testScrollIsPerUser() throws Exception {
assertSecurityIndexWriteable();
securityClient().preparePutRole("scrollable")
.addIndices(new String[] { randomAlphaOfLengthBetween(4, 12) }, new String[] { "read" }, null, null, null)
.get();
securityClient().preparePutUser("other", "changeme".toCharArray(), "scrollable").get();
final int numDocs = randomIntBetween(4, 16);
IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs];
for (int i = 0; i < docs.length; i++) {
docs[i] = client().prepareIndex("foo", "bar").setSource("doc", i);
}
indexRandom(true, docs);
SearchResponse response = client().prepareSearch("foo")
.setScroll(TimeValue.timeValueSeconds(5L))
.setQuery(matchAllQuery())
.setSize(1)
.get();
assertEquals(numDocs, response.getHits().getTotalHits());
assertEquals(1, response.getHits().getHits().length);
if (randomBoolean()) {
response = client().prepareSearchScroll(response.getScrollId()).setScroll(TimeValue.timeValueSeconds(5L)).get();
assertEquals(numDocs, response.getHits().getTotalHits());
assertEquals(1, response.getHits().getHits().length);
}
final String scrollId = response.getScrollId();
SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () ->
client()
.filterWithHeader(Collections.singletonMap("Authorization",
UsernamePasswordToken.basicAuthHeaderValue("other", new SecureString("changeme".toCharArray()))))
.prepareSearchScroll(scrollId)
.get());
for (ShardSearchFailure failure : e.shardFailures()) {
assertThat(ExceptionsHelper.unwrapCause(failure.getCause()), instanceOf(SearchContextMissingException.class));
}
}
public void testSearchAndClearScroll() throws Exception {
IndexRequestBuilder[] docs = new IndexRequestBuilder[randomIntBetween(20, 100)];
for (int i = 0; i < docs.length; i++) {
docs[i] = client().prepareIndex("idx", "type").setSource("field", "value");
}
indexRandom(true, docs);
SearchResponse response = client().prepareSearch()
.setQuery(matchAllQuery())
.setScroll(TimeValue.timeValueSeconds(5L))
.setSize(randomIntBetween(1, 10)).get();
int hits = 0;
try {
do {
assertHitCount(response, docs.length);
hits += response.getHits().getHits().length;
response = client().prepareSearchScroll(response.getScrollId())
.setScroll(TimeValue.timeValueSeconds(5L)).get();
} while (response.getHits().getHits().length != 0);
assertThat(hits, equalTo(docs.length));
} finally {
clearScroll(response.getScrollId());
}
}
@After
public void wipeSecurityIndex() {
try {
// this is a hack to clean up the .security index since only superusers can delete it and the default test user is not a
// superuser since the role used there is a file based role since we cannot guarantee the superuser role is always available
internalClient().admin().indices().prepareDelete(SecurityLifecycleService.SECURITY_INDEX_NAME).get();
} catch (IndexNotFoundException e) {
logger.warn("security index does not exist", e);
}
}
@Override
public String transportClientUsername() {
return this.nodeClientUsername();
}
@Override
public SecureString transportClientPassword() {
return this.nodeClientPassword();
}
}

View File

@ -0,0 +1,235 @@
/*
* 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.authz;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.common.util.concurrent.ThreadContext.StoredContext;
import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.search.Scroll;
import org.elasticsearch.search.SearchContextMissingException;
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
import org.elasticsearch.search.internal.ScrollContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TestSearchContext;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequest.Empty;
import org.elasticsearch.xpack.security.audit.AuditTrailService;
import org.elasticsearch.xpack.security.authc.Authentication;
import org.elasticsearch.xpack.security.authc.Authentication.RealmRef;
import org.elasticsearch.xpack.security.user.User;
import static org.elasticsearch.mock.orig.Mockito.verifyNoMoreInteractions;
import static org.elasticsearch.xpack.security.authz.AuthorizationService.ORIGINATING_ACTION_KEY;
import static org.elasticsearch.xpack.security.authz.SecuritySearchOperationListener.ensureAuthenticatedUserIsSame;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;
public class SecuritySearchOperationListenerTests extends ESTestCase {
public void testUnlicensed() {
XPackLicenseState licenseState = mock(XPackLicenseState.class);
when(licenseState.isAuthAllowed()).thenReturn(false);
ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
AuditTrailService auditTrailService = mock(AuditTrailService.class);
SearchContext searchContext = mock(SearchContext.class);
when(searchContext.scrollContext()).thenReturn(new ScrollContext());
SecuritySearchOperationListener listener = new SecuritySearchOperationListener(threadContext, licenseState, auditTrailService);
listener.onNewScrollContext(searchContext);
listener.validateSearchContext(searchContext, Empty.INSTANCE);
verify(licenseState, times(2)).isAuthAllowed();
verifyZeroInteractions(auditTrailService, searchContext);
}
public void testOnNewContextSetsAuthentication() throws Exception {
TestScrollSearchContext testSearchContext = new TestScrollSearchContext();
testSearchContext.scrollContext(new ScrollContext());
final Scroll scroll = new Scroll(TimeValue.timeValueSeconds(2L));
testSearchContext.scrollContext().scroll = scroll;
XPackLicenseState licenseState = mock(XPackLicenseState.class);
when(licenseState.isAuthAllowed()).thenReturn(true);
ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
AuditTrailService auditTrailService = mock(AuditTrailService.class);
Authentication authentication = new Authentication(new User("test", "role"), new RealmRef("realm", "file", "node"), null);
authentication.writeToContext(threadContext);
SecuritySearchOperationListener listener = new SecuritySearchOperationListener(threadContext, licenseState, auditTrailService);
listener.onNewScrollContext(testSearchContext);
Authentication contextAuth = testSearchContext.scrollContext().getFromContext(Authentication.AUTHENTICATION_KEY);
assertEquals(authentication, contextAuth);
assertEquals(scroll, testSearchContext.scrollContext().scroll);
verify(licenseState).isAuthAllowed();
verifyZeroInteractions(auditTrailService);
}
public void testValidateSearchContext() throws Exception {
TestScrollSearchContext testSearchContext = new TestScrollSearchContext();
testSearchContext.scrollContext(new ScrollContext());
testSearchContext.scrollContext().putInContext(Authentication.AUTHENTICATION_KEY,
new Authentication(new User("test", "role"), new RealmRef("realm", "file", "node"), null));
testSearchContext.scrollContext().scroll = new Scroll(TimeValue.timeValueSeconds(2L));
XPackLicenseState licenseState = mock(XPackLicenseState.class);
when(licenseState.isAuthAllowed()).thenReturn(true);
ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
AuditTrailService auditTrailService = mock(AuditTrailService.class);
SecuritySearchOperationListener listener = new SecuritySearchOperationListener(threadContext, licenseState, auditTrailService);
try (StoredContext ignore = threadContext.newStoredContext(false)) {
Authentication authentication = new Authentication(new User("test", "role"), new RealmRef("realm", "file", "node"), null);
authentication.writeToContext(threadContext);
listener.validateSearchContext(testSearchContext, Empty.INSTANCE);
verify(licenseState).isAuthAllowed();
verifyZeroInteractions(auditTrailService);
}
try (StoredContext ignore = threadContext.newStoredContext(false)) {
final String nodeName = randomAlphaOfLengthBetween(1, 8);
final String realmName = randomAlphaOfLengthBetween(1, 16);
Authentication authentication = new Authentication(new User("test", "role"), new RealmRef(realmName, "file", nodeName), null);
authentication.writeToContext(threadContext);
listener.validateSearchContext(testSearchContext, Empty.INSTANCE);
verify(licenseState, times(2)).isAuthAllowed();
verifyZeroInteractions(auditTrailService);
}
try (StoredContext ignore = threadContext.newStoredContext(false)) {
final String nodeName = randomBoolean() ? "node" : randomAlphaOfLengthBetween(1, 8);
final String realmName = randomBoolean() ? "realm" : randomAlphaOfLengthBetween(1, 16);
final String type = randomAlphaOfLengthBetween(5, 16);
Authentication authentication = new Authentication(new User("test", "role"), new RealmRef(realmName, type, nodeName), null);
authentication.writeToContext(threadContext);
threadContext.putTransient(ORIGINATING_ACTION_KEY, "action");
final InternalScrollSearchRequest request = new InternalScrollSearchRequest();
SearchContextMissingException expected =
expectThrows(SearchContextMissingException.class, () -> listener.validateSearchContext(testSearchContext, request));
assertEquals(testSearchContext.id(), expected.id());
verify(licenseState, times(3)).isAuthAllowed();
verify(auditTrailService)
.accessDenied(authentication.getUser(), "action", request);
}
// another user running as the original user
try (StoredContext ignore = threadContext.newStoredContext(false)) {
final String nodeName = randomBoolean() ? "node" : randomAlphaOfLengthBetween(1, 8);
final String realmName = randomBoolean() ? "realm" : randomAlphaOfLengthBetween(1, 16);
final String type = randomAlphaOfLengthBetween(5, 16);
User user = new User(new User("test", "role"), new User("authenticated", "runas"));
Authentication authentication = new Authentication(user, new RealmRef(realmName, type, nodeName),
new RealmRef(randomAlphaOfLengthBetween(1, 16), "file", nodeName));
authentication.writeToContext(threadContext);
threadContext.putTransient(ORIGINATING_ACTION_KEY, "action");
final InternalScrollSearchRequest request = new InternalScrollSearchRequest();
listener.validateSearchContext(testSearchContext, request);
verify(licenseState, times(4)).isAuthAllowed();
verifyNoMoreInteractions(auditTrailService);
}
// the user that authenticated for the run as request
try (StoredContext ignore = threadContext.newStoredContext(false)) {
final String nodeName = randomBoolean() ? "node" : randomAlphaOfLengthBetween(1, 8);
final String realmName = randomBoolean() ? "realm" : randomAlphaOfLengthBetween(1, 16);
final String type = randomAlphaOfLengthBetween(5, 16);
Authentication authentication =
new Authentication(new User("authenticated", "runas"), new RealmRef(realmName, type, nodeName), null);
authentication.writeToContext(threadContext);
threadContext.putTransient(ORIGINATING_ACTION_KEY, "action");
final InternalScrollSearchRequest request = new InternalScrollSearchRequest();
SearchContextMissingException expected =
expectThrows(SearchContextMissingException.class, () -> listener.validateSearchContext(testSearchContext, request));
assertEquals(testSearchContext.id(), expected.id());
verify(licenseState, times(5)).isAuthAllowed();
verify(auditTrailService)
.accessDenied(authentication.getUser(), "action", request);
}
}
public void testEnsuredAuthenticatedUserIsSame() {
Authentication original = new Authentication(new User("test", "role"), new RealmRef("realm", "file", "node"), null);
Authentication current =
randomBoolean() ? original : new Authentication(new User("test", "role"), new RealmRef("realm", "file", "node"), null);
long id = randomLong();
final String action = randomAlphaOfLength(4);
TransportRequest request = Empty.INSTANCE;
AuditTrailService auditTrail = mock(AuditTrailService.class);
ensureAuthenticatedUserIsSame(original, current, auditTrail, id, action, request);
verifyZeroInteractions(auditTrail);
// original user being run as
User user = new User(new User("test", "role"), new User("authenticated", "runas"));
current = new Authentication(user, new RealmRef("realm", "file", "node"),
new RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"));
ensureAuthenticatedUserIsSame(original, current, auditTrail, id, action, request);
verifyZeroInteractions(auditTrail);
// both user are run as
current = new Authentication(user, new RealmRef("realm", "file", "node"),
new RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"));
Authentication runAs = current;
ensureAuthenticatedUserIsSame(runAs, current, auditTrail, id, action, request);
verifyZeroInteractions(auditTrail);
// different authenticated by type
Authentication differentRealmType =
new Authentication(new User("test", "role"), new RealmRef("realm", randomAlphaOfLength(5), "node"), null);
SearchContextMissingException e = expectThrows(SearchContextMissingException.class,
() -> ensureAuthenticatedUserIsSame(original, differentRealmType, auditTrail, id, action, request));
assertEquals(id, e.id());
verify(auditTrail).accessDenied(differentRealmType.getUser(), action, request);
// wrong user
Authentication differentUser =
new Authentication(new User("test2", "role"), new RealmRef("realm", "realm", "node"), null);
e = expectThrows(SearchContextMissingException.class,
() -> ensureAuthenticatedUserIsSame(original, differentUser, auditTrail, id, action, request));
assertEquals(id, e.id());
verify(auditTrail).accessDenied(differentUser.getUser(), action, request);
// run as different user
Authentication diffRunAs = new Authentication(new User(new User("test2", "role"), new User("authenticated", "runas")),
new RealmRef("realm", "file", "node1"), new RealmRef("realm", "file", "node1"));
e = expectThrows(SearchContextMissingException.class,
() -> ensureAuthenticatedUserIsSame(original, diffRunAs, auditTrail, id, action, request));
assertEquals(id, e.id());
verify(auditTrail).accessDenied(diffRunAs.getUser(), action, request);
// run as different looked up by type
Authentication runAsDiffType = new Authentication(user, new RealmRef("realm", "file", "node"),
new RealmRef(randomAlphaOfLengthBetween(1, 16), randomAlphaOfLengthBetween(5, 12), "node"));
e = expectThrows(SearchContextMissingException.class,
() -> ensureAuthenticatedUserIsSame(runAs, runAsDiffType, auditTrail, id, action, request));
assertEquals(id, e.id());
verify(auditTrail).accessDenied(runAsDiffType.getUser(), action, request);
}
static class TestScrollSearchContext extends TestSearchContext {
private ScrollContext scrollContext;
TestScrollSearchContext() {
super(null);
}
@Override
public ScrollContext scrollContext() {
return scrollContext;
}
@Override
public SearchContext scrollContext(ScrollContext scrollContext) {
this.scrollContext = scrollContext;
return this;
}
}
}

View File

@ -113,6 +113,7 @@ for (boolean withSystemKey: [true, false]) {
waitCondition = waitWithAuth
setting 'xpack.ssl.keystore.path', 'testnode.jks'
setting 'xpack.ssl.keystore.password', 'testnode'
setting 'node.attr.upgraded', 'first'
extraConfigFile 'testnode.jks', new File(outputDir + '/testnode.jks')
if (withSystemKey) {
setting 'xpack.security.system_key.required', 'true'

View File

@ -1,7 +1,7 @@
---
"Index data and search on the mixed cluster":
- do:
cluster.health:
setup:
- do:
cluster.health:
# if the primary shard of an index with (number_of_replicas > 0) ends up on the new node, the replica cannot be
# allocated to the old node (see NodeVersionAllocationDecider). x-pack automatically creates indices with
# replicas, for example monitoring-data-*.
@ -9,6 +9,8 @@
wait_for_nodes: 2
timeout: 25s
---
"Index data and search on the mixed cluster":
- do:
search:
index: test_index
@ -82,3 +84,119 @@
- do:
indices.flush:
index: test_index
---
"Basic scroll mixed":
- do:
indices.create:
index: test_scroll
- do:
index:
index: test_scroll
type: test
id: 42
body: { foo: 1 }
- do:
index:
index: test_scroll
type: test
id: 43
body: { foo: 2 }
- do:
indices.refresh: {}
- do:
search:
index: test_scroll
size: 1
scroll: 1m
sort: foo
body:
query:
match_all: {}
- set: {_scroll_id: scroll_id}
- match: {hits.total: 2 }
- length: {hits.hits: 1 }
- match: {hits.hits.0._id: "42" }
- do:
index:
index: test_scroll
type: test
id: 44
body: { foo: 3 }
- do:
indices.refresh: {}
- do:
scroll:
body: { "scroll_id": "$scroll_id", "scroll": "1m"}
- match: {hits.total: 2 }
- length: {hits.hits: 1 }
- match: {hits.hits.0._id: "43" }
- do:
scroll:
scroll_id: $scroll_id
scroll: 1m
- match: {hits.total: 2 }
- length: {hits.hits: 0 }
- do:
clear_scroll:
scroll_id: $scroll_id
---
"Start scroll in mixed cluster for upgraded":
- do:
indices.create:
index: upgraded_scroll
wait_for_active_shards: all
body:
settings:
number_of_replicas: "0"
index.routing.allocation.include.upgraded: "first"
- do:
index:
index: upgraded_scroll
type: test
id: 42
body: { foo: 1 }
- do:
index:
index: upgraded_scroll
type: test
id: 43
body: { foo: 2 }
- do:
indices.refresh: {}
- do:
search:
index: upgraded_scroll
size: 1
scroll: 5m
sort: foo
body:
query:
match_all: {}
- set: {_scroll_id: scroll_id}
- match: {hits.total: 2 }
- length: {hits.hits: 1 }
- match: {hits.hits.0._id: "42" }
- do:
index:
index: scroll_index
type: doc
id: 1
body: { value: $scroll_id }

View File

@ -36,3 +36,34 @@
index: test_index
- match: { hits.total: 15 } # 10 docs from previous clusters plus 5 new docs
---
"Get indexed scroll and execute scroll":
- do:
get:
index: scroll_index
type: doc
id: 1
- set: {_source.value: scroll_id}
- do:
scroll:
scroll_id: $scroll_id
scroll: 1m
- match: {hits.total: 2 }
- length: {hits.hits: 1 }
- match: {hits.hits.0._id: "43" }
- do:
scroll:
scroll_id: $scroll_id
scroll: 1m
- match: {hits.total: 2 }
- length: {hits.hits: 0 }
- do:
clear_scroll:
scroll_id: $scroll_id