1
0
Эх сурвалжийг харах

Allow users to get status of own async search tasks (#106638)

This consists of 3 changes:

1. Refactoring the code so that all the security logic in the async search code is moved to AsyncSearchSecurity
2. Changing TransportGetAsyncStatusAction to check for ownership if the user does not have explicit access to the GetAsyncStatusAction (if they have such access it means that they can get the status of all async searches)
3. In RBACEngine, if a user does not have permission to GetAsyncStatusAction but does have permission to submit async searches, then let them run the action, relying on point 2 above.

Co-authored-by: Michael Peterson <michael.peterson@elastic.co>
Tim Vernum 1 жил өмнө
parent
commit
547e227ea2

+ 5 - 0
docs/changelog/106638.yaml

@@ -0,0 +1,5 @@
+pr: 106638
+summary: Allow users to get status of own async search tasks
+area: Authorization
+type: enhancement
+issues: []

+ 116 - 36
x-pack/plugin/async-search/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/search/AsyncSearchSecurityIT.java

@@ -105,6 +105,7 @@ public class AsyncSearchSecurityIT extends ESRestTestCase {
         .user("user2", "x-pack-test-password", "user2", false)
         .user("user-dls", "x-pack-test-password", "user-dls", false)
         .user("user-cancel", "x-pack-test-password", "user-cancel", false)
+        .user("user-monitor", "x-pack-test-password", "user-monitor", false)
         .build();
 
     @Override
@@ -169,48 +170,52 @@ public class AsyncSearchSecurityIT extends ESRestTestCase {
         testCase("user2", "user1");
     }
 
+    /**
+     * This test uses a 10-second delay in the search completion so that all actions against that user are done
+     * while the search is still running (which has different code paths from when the search is finished, which
+     * the testWithUsers test is generally testing).
+     * @throws IOException
+     */
+    public void testStatusWithUsersWhileSearchIsRunning() throws IOException {
+        String user = randomFrom("user1", "user2");
+        String other = user.equals("user1") ? "user2" : "user1";
+        String indexName = "index-" + user;
+        String query = """
+            {
+              "query": {
+                "error_query": {
+                  "indices": [
+                    {
+                      "name": "*",
+                      "error_type": "none",
+                      "stall_time_seconds": 10
+                    }
+                  ]
+                }
+              }
+            }""";
+
+        Response submitResp = submitAsyncSearchWithJsonBody(indexName, query, TimeValue.timeValueMillis(10), user);
+        assertOK(submitResp);
+        String id = extractResponseId(submitResp);
+
+        userBasedPermissionsAsserts(user, other, indexName, id);
+
+        ResponseException exc = expectThrows(
+            ResponseException.class,
+            () -> submitAsyncSearch("index-" + other, "*", TimeValue.timeValueSeconds(10), user)
+        );
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(403));
+        assertThat(exc.getMessage(), containsString("unauthorized"));
+    }
+
     private void testCase(String user, String other) throws Exception {
         for (String indexName : new String[] { "index", "index-" + user }) {
             Response submitResp = submitAsyncSearch(indexName, "foo:bar", TimeValue.timeValueSeconds(10), user);
             assertOK(submitResp);
             String id = extractResponseId(submitResp);
-            Response getResp = getAsyncSearch(id, user);
-            assertOK(getResp);
-
-            // other cannot access the result
-            ResponseException exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, other));
-            assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
-
-            // user-cancel cannot access the result
-            exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, "user-cancel"));
-            assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
-
-            // other cannot delete the result
-            exc = expectThrows(ResponseException.class, () -> deleteAsyncSearch(id, other));
-            assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
-
-            // other and user cannot access the result from direct get calls
-            AsyncExecutionId searchId = AsyncExecutionId.decode(id);
-            for (String runAs : new String[] { user, other }) {
-                exc = expectThrows(ResponseException.class, () -> get(ASYNC_RESULTS_INDEX, searchId.getDocId(), runAs));
-                assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(403));
-                assertThat(exc.getMessage(), containsString("unauthorized"));
-            }
-
-            Response delResp = deleteAsyncSearch(id, user);
-            assertOK(delResp);
 
-            // check that users with the 'cancel_task' privilege can delete an async
-            // search submitted by a different user.
-            for (String runAs : new String[] { "user-cancel", "test_kibana_user" }) {
-                Response newResp = submitAsyncSearch(indexName, "foo:bar", TimeValue.timeValueSeconds(10), user);
-                assertOK(newResp);
-                String newId = extractResponseId(newResp);
-                exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, runAs));
-                assertThat(exc.getResponse().getStatusLine().getStatusCode(), greaterThan(400));
-                delResp = deleteAsyncSearch(newId, runAs);
-                assertOK(delResp);
-            }
+            userBasedPermissionsAsserts(user, other, indexName, id);
         }
         ResponseException exc = expectThrows(
             ResponseException.class,
@@ -220,6 +225,64 @@ public class AsyncSearchSecurityIT extends ESRestTestCase {
         assertThat(exc.getMessage(), containsString("unauthorized"));
     }
 
+    private static void userBasedPermissionsAsserts(String user, String other, String indexName, String id) throws IOException {
+        Response statusResp = getAsyncStatus(id, user);
+        assertOK(statusResp);
+
+        Response getResp = getAsyncSearch(id, user);
+        assertOK(getResp);
+
+        // other (user) cannot access the status
+        ResponseException exc = expectThrows(ResponseException.class, () -> getAsyncStatus(id, other));
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+
+        // other (user) cannot access the result
+        exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, other));
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+
+        // user-cancel cannot access the result
+        exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, "user-cancel"));
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+
+        // user-monitor can access the status
+        assertOK(getAsyncStatus(id, "user-monitor"));
+
+        // user-monitor cannot access the result
+        exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, "user-monitor"));
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+
+        // other cannot delete the result
+        exc = expectThrows(ResponseException.class, () -> deleteAsyncSearch(id, other));
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+
+        // user-monitor cannot delete the result
+        exc = expectThrows(ResponseException.class, () -> deleteAsyncSearch(id, "user-monitor"));
+        assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+
+        // none of the users can access the result from direct get calls on the index
+        AsyncExecutionId searchId = AsyncExecutionId.decode(id);
+        for (String runAs : new String[] { user, other, "user-monitor", "user-cancel" }) {
+            exc = expectThrows(ResponseException.class, () -> get(ASYNC_RESULTS_INDEX, searchId.getDocId(), runAs));
+            assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(403));
+            assertThat(exc.getMessage(), containsString("unauthorized"));
+        }
+
+        Response delResp = deleteAsyncSearch(id, user);
+        assertOK(delResp);
+
+        // check that users with the 'cancel_task' privilege can delete an async
+        // search submitted by a different user.
+        for (String runAs : new String[] { "user-cancel", "test_kibana_user" }) {
+            Response newResp = submitAsyncSearch(indexName, "foo:bar", TimeValue.timeValueSeconds(10), user);
+            assertOK(newResp);
+            String newId = extractResponseId(newResp);
+            exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, runAs));
+            assertThat(exc.getResponse().getStatusLine().getStatusCode(), greaterThan(400));
+            delResp = deleteAsyncSearch(newId, runAs);
+            assertOK(delResp);
+        }
+    }
+
     private SearchHit[] getSearchHits(String asyncId, String user) throws IOException {
         final Response resp = getAsyncSearch(asyncId, user);
         assertOK(resp);
@@ -392,6 +455,17 @@ public class AsyncSearchSecurityIT extends ESRestTestCase {
         return client().performRequest(request);
     }
 
+    static Response submitAsyncSearchWithJsonBody(String indexName, String jsonBody, TimeValue waitForCompletion, String user)
+        throws IOException {
+        final Request request = new Request("POST", indexName + "/_async_search");
+        setRunAsHeader(request, user);
+        request.setJsonEntity(jsonBody);
+        request.addParameter("wait_for_completion_timeout", waitForCompletion.toString());
+        // we do the cleanup explicitly
+        request.addParameter("keep_on_completion", "true");
+        return client().performRequest(request);
+    }
+
     static Response submitAsyncSearch(String indexName, String query, TimeValue waitForCompletion, String user) throws IOException {
         final Request request = new Request("POST", indexName + "/_async_search");
         setRunAsHeader(request, user);
@@ -402,6 +476,12 @@ public class AsyncSearchSecurityIT extends ESRestTestCase {
         return client().performRequest(request);
     }
 
+    static Response getAsyncStatus(String id, String user) throws IOException {
+        final Request request = new Request("GET", "/_async_search/status/" + id);
+        setRunAsHeader(request, user);
+        return client().performRequest(request);
+    }
+
     static Response getAsyncSearch(String id, String user) throws IOException {
         final Request request = new Request("GET", "/_async_search/" + id);
         setRunAsHeader(request, user);

+ 4 - 0
x-pack/plugin/async-search/qa/security/src/javaRestTest/resources/roles.yml

@@ -59,3 +59,7 @@ user-dls:
 user-cancel:
   cluster:
     - cancel_task
+
+user-monitor:
+  cluster:
+    - monitor

+ 119 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncSearchSecurity.java

@@ -0,0 +1,119 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.core.async;
+
+import org.elasticsearch.ResourceNotFoundException;
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.get.GetRequest;
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.client.internal.OriginSettingClient;
+import org.elasticsearch.common.Strings;
+import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
+import org.elasticsearch.xpack.core.search.action.GetAsyncStatusAction;
+import org.elasticsearch.xpack.core.security.SecurityContext;
+import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesAction;
+import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesRequest;
+import org.elasticsearch.xpack.core.security.authc.Authentication;
+import org.elasticsearch.xpack.core.security.authz.RoleDescriptor;
+import org.elasticsearch.xpack.core.security.authz.privilege.ClusterPrivilegeResolver;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+public class AsyncSearchSecurity {
+
+    private static final FetchSourceContext FETCH_HEADERS_FIELD_CONTEXT = FetchSourceContext.of(
+        true,
+        new String[] { AsyncTaskIndexService.HEADERS_FIELD },
+        Strings.EMPTY_ARRAY
+    );
+
+    private final String indexName;
+    private final SecurityContext securityContext;
+    private final Client client;
+    private final OriginSettingClient clientWithOrigin;
+
+    public AsyncSearchSecurity(String indexName, SecurityContext securityContext, Client client, String origin) {
+        this.securityContext = securityContext;
+        this.client = client;
+        this.clientWithOrigin = new OriginSettingClient(client, origin);
+        this.indexName = indexName;
+    }
+
+    public void currentUserHasCancelTaskPrivilege(Consumer<Boolean> consumer) {
+        hasClusterPrivilege(
+            ClusterPrivilegeResolver.CANCEL_TASK.name(),
+            ActionListener.wrap(consumer::accept, ex -> consumer.accept(false))
+        );
+    }
+
+    public void currentUserCanSeeStatusOfAllSearches(ActionListener<Boolean> listener) {
+        // If the user has access to the action by-name, then they can get the status of any async search
+        hasClusterPrivilege(GetAsyncStatusAction.NAME, listener);
+    }
+
+    private void hasClusterPrivilege(String privilegeName, ActionListener<Boolean> listener) {
+        final Authentication current = securityContext.getAuthentication();
+        if (current != null) {
+            HasPrivilegesRequest req = new HasPrivilegesRequest();
+            req.username(current.getEffectiveSubject().getUser().principal());
+            req.clusterPrivileges(privilegeName);
+            req.indexPrivileges(new RoleDescriptor.IndicesPrivileges[] {});
+            req.applicationPrivileges(new RoleDescriptor.ApplicationResourcePrivileges[] {});
+            try {
+                client.execute(HasPrivilegesAction.INSTANCE, req, listener.map(resp -> resp.isCompleteMatch()));
+            } catch (Exception exc) {
+                listener.onFailure(exc);
+            }
+        } else {
+            listener.onResponse(false);
+        }
+    }
+
+    public boolean currentUserHasAccessToTask(AsyncTask asyncTask) throws IOException {
+        Objects.requireNonNull(asyncTask, "Task cannot be null");
+        return currentUserHasAccessToTaskWithHeaders(asyncTask.getOriginHeaders());
+    }
+
+    public boolean currentUserHasAccessToTaskWithHeaders(Map<String, String> headers) throws IOException {
+        return securityContext.canIAccessResourcesCreatedWithHeaders(headers);
+    }
+
+    /**
+     * Checks if the current user can access the async search result of the original user.
+     */
+    void ensureAuthenticatedUserCanDeleteFromIndex(AsyncExecutionId executionId, ActionListener<Void> listener) {
+        getTaskHeadersFromIndex(executionId, listener.map(headers -> {
+            if (currentUserHasAccessToTaskWithHeaders(headers)) {
+                return null;
+            } else {
+                throw new ResourceNotFoundException(executionId.getEncoded());
+            }
+        }));
+    }
+
+    private void getTaskHeadersFromIndex(AsyncExecutionId executionId, ActionListener<Map<String, String>> listener) {
+        GetRequest internalGet = new GetRequest(indexName).preference(executionId.getEncoded())
+            .id(executionId.getDocId())
+            .fetchSourceContext(FETCH_HEADERS_FIELD_CONTEXT);
+
+        clientWithOrigin.get(internalGet, ActionListener.wrap(get -> {
+            if (get.isExists() == false) {
+                listener.onFailure(new ResourceNotFoundException(executionId.getEncoded()));
+                return;
+            }
+            // Check authentication for the user
+            @SuppressWarnings("unchecked")
+            Map<String, String> headers = (Map<String, String>) get.getSource().get(AsyncTaskIndexService.HEADERS_FIELD);
+            listener.onResponse(headers);
+        }, exc -> listener.onFailure(new ResourceNotFoundException(executionId.getEncoded()))));
+    }
+
+}

+ 44 - 65
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java

@@ -26,7 +26,6 @@ import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.client.internal.OriginSettingClient;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.TriFunction;
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.bytes.BytesReference;
@@ -49,13 +48,11 @@ import org.elasticsearch.core.Streams;
 import org.elasticsearch.index.engine.DocumentMissingException;
 import org.elasticsearch.index.engine.VersionConflictEngineException;
 import org.elasticsearch.indices.SystemIndexDescriptor;
-import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.xcontent.DeprecationHandler;
 import org.elasticsearch.xcontent.NamedXContentRegistry;
 import org.elasticsearch.xcontent.XContentBuilder;
-import org.elasticsearch.xcontent.XContentFactory;
 import org.elasticsearch.xcontent.XContentParser;
 import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.core.XPackPlugin;
@@ -154,9 +151,10 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
     }
 
     private final String index;
+    private final ThreadContext threadContext;
     private final Client client;
+    final AsyncSearchSecurity security;
     private final Client clientWithOrigin;
-    private final SecurityContext securityContext;
     private final NamedWriteableRegistry registry;
     private final Writeable.Reader<R> reader;
     private final BigArrays bigArrays;
@@ -175,8 +173,14 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         BigArrays bigArrays
     ) {
         this.index = index;
-        this.securityContext = new SecurityContext(clusterService.getSettings(), threadContext);
+        this.threadContext = threadContext;
         this.client = client;
+        this.security = new AsyncSearchSecurity(
+            index,
+            new SecurityContext(clusterService.getSettings(), client.threadPool().getThreadContext()),
+            client,
+            origin
+        );
         this.clientWithOrigin = new OriginSettingClient(client, origin);
         this.registry = registry;
         this.reader = reader;
@@ -202,11 +206,8 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         return client;
     }
 
-    /**
-     * Returns the authentication information, or null if the current context has no authentication info.
-     **/
-    public SecurityContext getSecurityContext() {
-        return securityContext;
+    public AsyncSearchSecurity getSecurity() {
+        return security;
     }
 
     /**
@@ -257,8 +258,7 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         try {
             var buffer = allocateBuffer(limitToMaxResponseSize);
             listener = ActionListener.runBefore(listener, buffer::close);
-            final XContentBuilder source = XContentFactory.jsonBuilder(buffer)
-                .startObject()
+            final XContentBuilder source = jsonBuilder(buffer).startObject()
                 .field(HEADERS_FIELD, headers)
                 .field(EXPIRATION_TIME_FIELD, response.getExpirationTime());
             if (responseHeaders != null) {
@@ -285,7 +285,7 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         ReleasableBytesStreamOutput buffer = null;
         try {
             buffer = allocateBuffer(isFailure == false);
-            final XContentBuilder source = XContentFactory.jsonBuilder(buffer).startObject().field(RESPONSE_HEADERS_FIELD, responseHeaders);
+            final XContentBuilder source = jsonBuilder(buffer).startObject().field(RESPONSE_HEADERS_FIELD, responseHeaders);
             addResultFieldAndFinish(response, source);
             clientWithOrigin.update(
                 new UpdateRequest().index(index).id(docId).doc(buffer.bytes(), source.contentType()).retryOnConflict(5),
@@ -399,7 +399,7 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
             return null;
         }
         // Check authentication for the user
-        if (false == securityContext.canIAccessResourcesCreatedWithHeaders(asyncTask.getOriginHeaders())) {
+        if (false == security.currentUserHasAccessToTask(asyncTask)) {
             throw new ResourceNotFoundException(asyncExecutionId.getEncoded() + " not found");
         }
         return asyncTask;
@@ -472,7 +472,7 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
                         @SuppressWarnings("unchecked")
                         final Map<String, String> headers = (Map<String, String>) XContentParserUtils.parseFieldsValue(parser);
                         // check the authentication of the current user against the user that initiated the async task
-                        if (checkAuthentication && false == securityContext.canIAccessResourcesCreatedWithHeaders(headers)) {
+                        if (checkAuthentication && false == security.currentUserHasAccessToTaskWithHeaders(headers)) {
                             throw new ResourceNotFoundException(asyncExecutionId.getEncoded());
                         }
                     }
@@ -482,7 +482,7 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
                             parser
                         );
                         if (restoreResponseHeaders) {
-                            restoreResponseHeadersContext(securityContext.getThreadContext(), responseHeaders);
+                            restoreResponseHeadersContext(threadContext, responseHeaders);
                         }
                     }
                     default -> XContentParserUtils.parseFieldsValue(parser); // consume and discard unknown fields
@@ -510,64 +510,43 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         Class<T> tClass,
         Function<T, SR> statusProducerFromTask,
         TriFunction<R, Long, String, SR> statusProducerFromIndex,
-        ActionListener<SR> outerListener
+        ActionListener<SR> originalListener
     ) {
         // check if the result has expired
-        outerListener = outerListener.delegateFailure((listener, resp) -> {
+        final ActionListener<SR> outerListener = originalListener.delegateFailure((listener, resp) -> {
             if (resp.getExpirationTime() < System.currentTimeMillis()) {
                 listener.onFailure(new ResourceNotFoundException(request.getId()));
             } else {
                 listener.onResponse(resp);
             }
         });
-        AsyncExecutionId asyncExecutionId = AsyncExecutionId.decode(request.getId());
-        try {
-            T asyncTask = getTask(taskManager, asyncExecutionId, tClass);
-            if (asyncTask != null) { // get status response from task
-                SR response = statusProducerFromTask.apply(asyncTask);
-                outerListener.onResponse(response);
-            } else {
-                // get status response from index
-                getResponseFromIndex(
-                    asyncExecutionId,
-                    false,
-                    false,
-                    outerListener.map(resp -> statusProducerFromIndex.apply(resp, resp.getExpirationTime(), asyncExecutionId.getEncoded()))
-                );
-            }
-        } catch (Exception exc) {
-            outerListener.onFailure(exc);
-        }
-    }
-
-    private static final FetchSourceContext FETCH_HEADERS_FIELD_CONTEXT = FetchSourceContext.of(
-        true,
-        new String[] { HEADERS_FIELD },
-        Strings.EMPTY_ARRAY
-    );
-
-    /**
-     * Checks if the current user can access the async search result of the original user.
-     **/
-    void ensureAuthenticatedUserCanDeleteFromIndex(AsyncExecutionId executionId, ActionListener<Void> listener) {
-        GetRequest internalGet = new GetRequest(index).preference(executionId.getEncoded())
-            .id(executionId.getDocId())
-            .fetchSourceContext(FETCH_HEADERS_FIELD_CONTEXT);
-
-        clientWithOrigin.get(internalGet, ActionListener.wrap(get -> {
-            if (get.isExists() == false) {
-                listener.onFailure(new ResourceNotFoundException(executionId.getEncoded()));
-                return;
-            }
-            // Check authentication for the user
-            @SuppressWarnings("unchecked")
-            Map<String, String> headers = (Map<String, String>) get.getSource().get(HEADERS_FIELD);
-            if (securityContext.canIAccessResourcesCreatedWithHeaders(headers)) {
-                listener.onResponse(null);
-            } else {
-                listener.onFailure(new ResourceNotFoundException(executionId.getEncoded()));
+        security.currentUserCanSeeStatusOfAllSearches(ActionListener.wrap(canSeeAll -> {
+            AsyncExecutionId asyncExecutionId = AsyncExecutionId.decode(request.getId());
+            try {
+                T asyncTask = getTask(taskManager, asyncExecutionId, tClass);
+                if (asyncTask != null) { // get status response from task
+                    if (canSeeAll || security.currentUserHasAccessToTask(asyncTask)) {
+                        var response = statusProducerFromTask.apply(asyncTask);
+                        outerListener.onResponse(response);
+                    } else {
+                        outerListener.onFailure(new ResourceNotFoundException(request.getId()));
+                    }
+                } else {
+                    // get status response from index
+                    final boolean checkAuthentication = canSeeAll == false;
+                    getResponseFromIndex(
+                        asyncExecutionId,
+                        false,
+                        checkAuthentication,
+                        outerListener.map(
+                            resp -> statusProducerFromIndex.apply(resp, resp.getExpirationTime(), asyncExecutionId.getEncoded())
+                        )
+                    );
+                }
+            } catch (Exception exc) {
+                outerListener.onFailure(exc);
             }
-        }, exc -> listener.onFailure(new ResourceNotFoundException(executionId.getEncoded()))));
+        }, outerListener::onFailure));
     }
 
     /**

+ 7 - 28
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/DeleteAsyncResultsService.java

@@ -15,11 +15,6 @@ import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.tasks.TaskManager;
-import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesAction;
-import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesRequest;
-import org.elasticsearch.xpack.core.security.authc.Authentication;
-import org.elasticsearch.xpack.core.security.authz.RoleDescriptor;
-import org.elasticsearch.xpack.core.security.authz.privilege.ClusterPrivilegeResolver;
 
 import java.util.function.Consumer;
 
@@ -29,8 +24,10 @@ import java.util.function.Consumer;
  */
 public class DeleteAsyncResultsService {
     private static final Logger logger = LogManager.getLogger(DeleteAsyncResultsService.class);
-    private final TaskManager taskManager;
+
     private final AsyncTaskIndexService<? extends AsyncResponse<?>> store;
+    private final AsyncSearchSecurity security;
+    private final TaskManager taskManager;
 
     /**
      * Creates async results service
@@ -39,8 +36,9 @@ public class DeleteAsyncResultsService {
      * @param taskManager    task manager
      */
     public DeleteAsyncResultsService(AsyncTaskIndexService<? extends AsyncResponse<?>> store, TaskManager taskManager) {
-        this.taskManager = taskManager;
         this.store = store;
+        this.security = store.getSecurity();
+        this.taskManager = taskManager;
     }
 
     public void deleteResponse(DeleteAsyncResultRequest request, ActionListener<AcknowledgedResponse> listener) {
@@ -52,26 +50,7 @@ public class DeleteAsyncResultsService {
      * delete async search submitted by another user.
      */
     private void hasCancelTaskPrivilegeAsync(Consumer<Boolean> consumer) {
-        final Authentication current = store.getSecurityContext().getAuthentication();
-        if (current != null) {
-            HasPrivilegesRequest req = new HasPrivilegesRequest();
-            req.username(current.getEffectiveSubject().getUser().principal());
-            req.clusterPrivileges(ClusterPrivilegeResolver.CANCEL_TASK.name());
-            req.indexPrivileges(new RoleDescriptor.IndicesPrivileges[] {});
-            req.applicationPrivileges(new RoleDescriptor.ApplicationResourcePrivileges[] {});
-            try {
-                store.getClient()
-                    .execute(
-                        HasPrivilegesAction.INSTANCE,
-                        req,
-                        ActionListener.wrap(resp -> consumer.accept(resp.isCompleteMatch()), exc -> consumer.accept(false))
-                    );
-            } catch (Exception exc) {
-                consumer.accept(false);
-            }
-        } else {
-            consumer.accept(false);
-        }
+        security.currentUserHasCancelTaskPrivilege(consumer);
     }
 
     private void deleteResponseAsync(
@@ -91,7 +70,7 @@ public class DeleteAsyncResultsService {
                 if (hasCancelTaskPrivilege) {
                     deleteResponseFromIndex(searchId, false, listener);
                 } else {
-                    store.ensureAuthenticatedUserCanDeleteFromIndex(
+                    store.security.ensureAuthenticatedUserCanDeleteFromIndex(
                         searchId,
                         listener.delegateFailureAndWrap((l, res) -> deleteResponseFromIndex(searchId, false, l))
                     );

+ 157 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncSearchSecurityTests.java

@@ -0,0 +1,157 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.core.async;
+
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.concurrent.ThreadContext;
+import org.elasticsearch.test.ESSingleNodeTestCase;
+import org.elasticsearch.xpack.core.security.SecurityContext;
+import org.elasticsearch.xpack.core.security.authc.Authentication;
+import org.elasticsearch.xpack.core.security.authc.AuthenticationTestHelper;
+import org.elasticsearch.xpack.core.security.user.User;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+import static org.hamcrest.Matchers.is;
+
+public class AsyncSearchSecurityTests extends ESSingleNodeTestCase {
+
+    public void testEnsuredAuthenticatedUserIsSame() throws IOException {
+        final ThreadContext threadContext = client().threadPool().getThreadContext();
+        final AsyncSearchSecurity security = new AsyncSearchSecurity(
+            ".async-search",
+            new SecurityContext(Settings.EMPTY, threadContext),
+            client(),
+            "async_origin"
+        );
+
+        Authentication original = AuthenticationTestHelper.builder()
+            .user(new User("test", "role"))
+            .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+            .build(false);
+        Authentication current = randomBoolean()
+            ? original
+            : AuthenticationTestHelper.builder()
+                .user(new User("test", "role"))
+                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+                .build(false);
+        current.writeToContext(threadContext);
+        assertThat(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)), is(true));
+
+        // "original" search was unauthenticated (e.g. security was turned off when it was performed)
+        assertThat(security.currentUserHasAccessToTaskWithHeaders(Collections.emptyMap()), is(true));
+
+        // current is not authenticated
+        try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
+            assertThat(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)), is(false));
+            assertThat(security.currentUserHasAccessToTaskWithHeaders(Map.of()), is(true));
+        }
+
+        // original user being run as
+        final User authenticatingUser = new User("authenticated", "runas");
+        final User effectiveUser = new User("test", "role");
+        assertThat(
+            security.currentUserHasAccessToTaskWithHeaders(
+                getAuthenticationAsHeaders(
+                    AuthenticationTestHelper.builder()
+                        .user(authenticatingUser)
+                        .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"))
+                        .runAs()
+                        .user(effectiveUser)
+                        .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+                        .build()
+                )
+            ),
+            is(true)
+        );
+
+        try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
+            // current user being run as
+            current = AuthenticationTestHelper.builder()
+                .user(authenticatingUser)
+                .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"))
+                .runAs()
+                .user(effectiveUser)
+                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+                .build();
+            current.writeToContext(threadContext);
+            assertThat(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)), is(true));
+
+            // both users are run as
+            assertThat(
+                security.currentUserHasAccessToTaskWithHeaders(
+                    getAuthenticationAsHeaders(
+                        AuthenticationTestHelper.builder()
+                            .user(authenticatingUser)
+                            .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"))
+                            .runAs()
+                            .user(effectiveUser)
+                            .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+                            .build()
+                    )
+                ),
+                is(true)
+            );
+        }
+
+        try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
+            // different authenticated by type
+            final Authentication differentRealmType = AuthenticationTestHelper.builder()
+                .user(new User("test", "role"))
+                .realmRef(new Authentication.RealmRef("realm", randomAlphaOfLength(10), "node"))
+                .build(false);
+            differentRealmType.writeToContext(threadContext);
+            assertFalse(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)));
+        }
+
+        // different user
+        try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
+            final Authentication differentUser = AuthenticationTestHelper.builder()
+                .user(new User("test2", "role"))
+                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+                .build(false);
+            differentUser.writeToContext(threadContext);
+            assertFalse(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)));
+        }
+
+        // run as different user
+        try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
+            final Authentication differentRunAs = AuthenticationTestHelper.builder()
+                .user(new User("authenticated", "runas"))
+                .realmRef(new Authentication.RealmRef("realm_runas", "file", "node1"))
+                .runAs()
+                .user(new User("test2", "role"))
+                .realmRef(new Authentication.RealmRef("realm", "file", "node1"))
+                .build();
+            differentRunAs.writeToContext(threadContext);
+            assertFalse(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)));
+        }
+
+        // run as different looked up by type
+        try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
+            final Authentication runAsDiffType = AuthenticationTestHelper.builder()
+                .user(authenticatingUser)
+                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
+                .runAs()
+                .user(effectiveUser)
+                .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), randomAlphaOfLengthBetween(5, 12), "node"))
+                .build();
+            runAsDiffType.writeToContext(threadContext);
+            assertFalse(security.currentUserHasAccessToTaskWithHeaders(getAuthenticationAsHeaders(original)));
+        }
+    }
+
+    private Map<String, String> getAuthenticationAsHeaders(Authentication authentication) throws IOException {
+        ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
+        authentication.writeToContext(threadContext);
+        return threadContext.getHeaders();
+    }
+
+}

+ 0 - 135
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/async/AsyncTaskServiceTests.java

@@ -15,7 +15,6 @@ import org.elasticsearch.action.update.UpdateResponse;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.plugins.SystemIndexPlugin;
@@ -23,17 +22,12 @@ import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.test.ESSingleNodeTestCase;
 import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
-import org.elasticsearch.xpack.core.security.authc.Authentication;
-import org.elasticsearch.xpack.core.security.authc.AuthenticationTestHelper;
-import org.elasticsearch.xpack.core.security.user.User;
 import org.junit.Before;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.hamcrest.Matchers.is;
@@ -88,130 +82,6 @@ public class AsyncTaskServiceTests extends ESSingleNodeTestCase {
         }
     }
 
-    public void testEnsuredAuthenticatedUserIsSame() throws IOException {
-        Authentication original = AuthenticationTestHelper.builder()
-            .user(new User("test", "role"))
-            .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-            .build(false);
-        Authentication current = randomBoolean()
-            ? original
-            : AuthenticationTestHelper.builder()
-                .user(new User("test", "role"))
-                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-                .build(false);
-        current.writeToContext(indexService.getSecurityContext().getThreadContext());
-        assertThat(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)), is(true));
-
-        // original is not authenticated
-        assertThat(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(Collections.emptyMap()), is(true));
-        // current is not authenticated
-        try (ThreadContext.StoredContext ignore = indexService.getSecurityContext().getThreadContext().stashContext()) {
-            assertThat(
-                indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)),
-                is(false)
-            );
-            assertThat(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(Map.of()), is(true));
-        }
-
-        // original user being run as
-        final User authenticatingUser = new User("authenticated", "runas");
-        final User effectiveUser = new User("test", "role");
-        assertThat(
-            indexService.getSecurityContext()
-                .canIAccessResourcesCreatedWithHeaders(
-                    getAuthenticationAsHeaders(
-                        AuthenticationTestHelper.builder()
-                            .user(authenticatingUser)
-                            .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"))
-                            .runAs()
-                            .user(effectiveUser)
-                            .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-                            .build()
-                    )
-                ),
-            is(true)
-        );
-
-        try (ThreadContext.StoredContext ignore = indexService.getSecurityContext().getThreadContext().stashContext()) {
-            // current user being run as
-            current = AuthenticationTestHelper.builder()
-                .user(authenticatingUser)
-                .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"))
-                .runAs()
-                .user(effectiveUser)
-                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-                .build();
-            current.writeToContext(indexService.getSecurityContext().getThreadContext());
-            assertThat(
-                indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)),
-                is(true)
-            );
-
-            // both users are run as
-            assertThat(
-                indexService.getSecurityContext()
-                    .canIAccessResourcesCreatedWithHeaders(
-                        getAuthenticationAsHeaders(
-                            AuthenticationTestHelper.builder()
-                                .user(authenticatingUser)
-                                .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"))
-                                .runAs()
-                                .user(effectiveUser)
-                                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-                                .build()
-                        )
-                    ),
-                is(true)
-            );
-        }
-
-        try (ThreadContext.StoredContext ignore = indexService.getSecurityContext().getThreadContext().stashContext()) {
-            // different authenticated by type
-            final Authentication differentRealmType = AuthenticationTestHelper.builder()
-                .user(new User("test", "role"))
-                .realmRef(new Authentication.RealmRef("realm", randomAlphaOfLength(10), "node"))
-                .build(false);
-            differentRealmType.writeToContext(indexService.getSecurityContext().getThreadContext());
-            assertFalse(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)));
-        }
-
-        // different user
-        try (ThreadContext.StoredContext ignore = indexService.getSecurityContext().getThreadContext().stashContext()) {
-            final Authentication differentUser = AuthenticationTestHelper.builder()
-                .user(new User("test2", "role"))
-                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-                .build(false);
-            differentUser.writeToContext(indexService.getSecurityContext().getThreadContext());
-            assertFalse(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)));
-        }
-
-        // run as different user
-        try (ThreadContext.StoredContext ignore = indexService.getSecurityContext().getThreadContext().stashContext()) {
-            final Authentication differentRunAs = AuthenticationTestHelper.builder()
-                .user(new User("authenticated", "runas"))
-                .realmRef(new Authentication.RealmRef("realm_runas", "file", "node1"))
-                .runAs()
-                .user(new User("test2", "role"))
-                .realmRef(new Authentication.RealmRef("realm", "file", "node1"))
-                .build();
-            differentRunAs.writeToContext(indexService.getSecurityContext().getThreadContext());
-            assertFalse(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)));
-        }
-
-        // run as different looked up by type
-        try (ThreadContext.StoredContext ignore = indexService.getSecurityContext().getThreadContext().stashContext()) {
-            final Authentication runAsDiffType = AuthenticationTestHelper.builder()
-                .user(authenticatingUser)
-                .realmRef(new Authentication.RealmRef("realm", "file", "node"))
-                .runAs()
-                .user(effectiveUser)
-                .realmRef(new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), randomAlphaOfLengthBetween(5, 12), "node"))
-                .build();
-            runAsDiffType.writeToContext(indexService.getSecurityContext().getThreadContext());
-            assertFalse(indexService.getSecurityContext().canIAccessResourcesCreatedWithHeaders(getAuthenticationAsHeaders(original)));
-        }
-    }
-
     public void testAutoCreateIndex() throws Exception {
         // To begin with, the results index should be auto-created.
         AsyncExecutionId id = new AsyncExecutionId("0", new TaskId("N/A", 0));
@@ -267,9 +137,4 @@ public class AsyncTaskServiceTests extends ESSingleNodeTestCase {
         assertThat(expected, is(settings.filter(expected::hasValue)));
     }
 
-    private Map<String, String> getAuthenticationAsHeaders(Authentication authentication) throws IOException {
-        ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
-        authentication.writeToContext(threadContext);
-        return threadContext.getHeaders();
-    }
 }

+ 5 - 0
x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java

@@ -46,6 +46,7 @@ import org.elasticsearch.xpack.core.async.TransportDeleteAsyncResultAction;
 import org.elasticsearch.xpack.core.eql.EqlAsyncActionNames;
 import org.elasticsearch.xpack.core.esql.EsqlAsyncActionNames;
 import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
+import org.elasticsearch.xpack.core.search.action.GetAsyncStatusAction;
 import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
 import org.elasticsearch.xpack.core.security.action.apikey.GetApiKeyAction;
 import org.elasticsearch.xpack.core.security.action.apikey.GetApiKeyRequest;
@@ -188,6 +189,10 @@ public class RBACEngine implements AuthorizationEngine {
                 listener.onResponse(AuthorizationResult.granted());
             } else if (checkSameUserPermissions(requestInfo.getAction(), requestInfo.getRequest(), requestInfo.getAuthentication())) {
                 listener.onResponse(AuthorizationResult.granted());
+            } else if (GetAsyncStatusAction.NAME.equals(requestInfo.getAction()) && role.checkIndicesAction(SubmitAsyncSearchAction.NAME)) {
+                // Users who are allowed to submit async searches are allowed to check the status of those searches
+                // Search ownership will be checked by AsyncSearchSecurity
+                listener.onResponse(AuthorizationResult.granted());
             } else {
                 listener.onResponse(AuthorizationResult.deny());
             }