Răsfoiți Sursa

Introduce eql search status API (#68065)

Introduce eql search status API,
that reports the status of eql stored or async search.

GET _eql/search/status/<id>

The API is restricted to the monitoring_user role.

For a running eql search, a response has the following format:

{
  "id" : <id>,
  "is_running" : true,
  "is_partial" : true,
  "start_time_in_millis" : 1611690235000,
  "expiration_time_in_millis" : 1611690295000
}

For a completed eql search, a response has the following format:

{
  "id" : <id>,
  "is_running" : false,
  "is_partial" : false,
  "expiration_time_in_millis" : 1611690295000,
  "completion_status" : 200
}

Closes #66955
Mayya Sharipova 4 ani în urmă
părinte
comite
6521d2af27
20 a modificat fișierele cu 897 adăugiri și 77 ștergeri
  1. 3 1
      docs/reference/eql/eql-search-api.asciidoc
  2. 25 0
      docs/reference/eql/eql.asciidoc
  3. 120 0
      docs/reference/eql/get-async-eql-status-api.asciidoc
  4. 3 0
      docs/reference/search.asciidoc
  5. 8 4
      x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/AsyncSearchTask.java
  6. 10 49
      x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportGetAsyncStatusAction.java
  7. 105 0
      x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncStatusResponseTests.java
  8. 59 10
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java
  9. 37 5
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/AsyncStatusResponse.java
  10. 19 0
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/SearchStatusResponse.java
  11. 50 0
      x-pack/plugin/eql/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/eql/10_basic.yml
  12. 19 8
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchTask.java
  13. 191 0
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlStatusResponse.java
  14. 19 0
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlAsyncGetStatusAction.java
  15. 2 0
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlPlugin.java
  16. 35 0
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/RestEqlGetAsyncStatusAction.java
  17. 75 0
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/TransportEqlAsyncGetStatusAction.java
  18. 85 0
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlStatusResponseTests.java
  19. 1 0
      x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java
  20. 31 0
      x-pack/plugin/src/test/resources/rest-api-spec/api/eql.get_status.json

+ 3 - 1
docs/reference/eql/eql-search-api.asciidoc

@@ -338,7 +338,9 @@ This search ID is only provided if one of the following conditions is met:
   parameter is `true`.
 
 You can use this ID with the <<get-async-eql-search-api,get async EQL search
-API>> to get the current status and available results for the search.
+API>> to get the current status and available results for the search or
+<<get-async-eql-status-api,get async EQL status API>> to get only
+the current status.
 --
 
 `is_partial`::

+ 25 - 0
docs/reference/eql/eql.asciidoc

@@ -567,6 +567,28 @@ complete.
 // TESTRESPONSE[s/"took": 2000/"took": $body.took/]
 // TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/]
 
+Another more lightweight way to check the progress of an async search is to use
+the <<get-async-eql-status-api,get async EQL status API>> with the search ID.
+
+[source,console]
+----
+GET /_eql/search/status/FmNJRUZ1YWZCU3dHY1BIOUhaenVSRkEaaXFlZ3h4c1RTWFNocDdnY2FSaERnUTozNDE=
+----
+// TEST[skip: no access to search ID]
+
+[source,console-result]
+----
+{
+  "id": "FmNJRUZ1YWZCU3dHY1BIOUhaenVSRkEaaXFlZ3h4c1RTWFNocDdnY2FSaERnUTozNDE=",
+  "is_running": false,
+  "is_partial": false,
+  "expiration_time_in_millis" : 1611690295000,
+  "completion_status": 200
+}
+----
+// TESTRESPONSE[s/FmNJRUZ1YWZCU3dHY1BIOUhaenVSRkEaaXFlZ3h4c1RTWFNocDdnY2FSaERnUTozNDE=/$body.id/]
+// TESTRESPONSE[s/"expiration_time_in_millis": 1611690295000/"expiration_time_in_millis": $body.expiration_time_in_millis/]
+
 [discrete]
 [[eql-search-store-async-eql-search]]
 === Change the search retention period
@@ -660,6 +682,9 @@ GET /_eql/search/FjlmbndxNmJjU0RPdExBTGg0elNOOEEaQk9xSjJBQzBRMldZa1VVQ2pPa01YUTo
 Saved synchronous searches are still subject to the `keep_alive` parameter's
 retention period. When this period ends, the search and its results are deleted.
 
+You can also check only the status of the saved synchronous search without
+results by using <<get-async-eql-status-api,get async EQL status API>>.
+
 You can also manually delete saved synchronous searches using the
 <<delete-async-eql-search-api,delete async EQL search API>>.
 

+ 120 - 0
docs/reference/eql/get-async-eql-status-api.asciidoc

@@ -0,0 +1,120 @@
+[role="xpack"]
+[testenv="basic"]
+
+[[get-async-eql-status-api]]
+=== Get async EQL status API
+++++
+<titleabbrev>Get async EQL search status</titleabbrev>
+++++
+Returns the current status for an <<eql-search-async,async EQL search>> or
+a <<eql-search-store-sync-eql-search,stored synchronous EQL search>>
+without returning results. This is a more lightweight API than
+<<get-async-eql-search-api,get async EQL search API>> as it doesn't return
+search results, and reports only the status.
+
+If the {es} {security-features} are enabled, the access to the get async
+eql status API is restricted to the <<built-in-roles, monitoring_user role>>.
+
+[source,console]
+----
+GET /_eql/search/status/FkpMRkJGS1gzVDRlM3g4ZzMyRGlLbkEaTXlJZHdNT09TU2VTZVBoNDM3cFZMUToxMDM=
+----
+// TEST[skip: no access to search ID]
+
+[[get-async-eql-status-api-request]]
+==== {api-request-title}
+
+`GET /_eql/search/status/<search_id>`
+
+
+[[get-async-eql-status-api-path-params]]
+==== {api-path-parms-title}
+
+`<search_id>`::
+(Required, string)
+Identifier for the search.
++
+A search ID is provided in the <<eql-search-api,EQL search API>>'s response for
+an <<eql-search-async,async search>>. A search ID is also provided if the
+request's <<eql-search-api-keep-on-completion,`keep_on_completion`>> parameter
+is `true`.
+
+[role="child_attributes"]
+[[get-async-eql-status-api-response-body]]
+==== {api-response-body-title}
+
+`id`::
+(string)
+Identifier for the search.
+
+`is_running`::
+(boolean)
+If `true`, the search request is still executing.
+If `false`, the search is completed.
+
+`is_partial`::
+(boolean)
+If `true`, the response does not contain complete search results.
+This could be because either the search is still running
+(`is_running` status is `false`), or because it is already completed
+(`is_running` status is `true`)  and results are partial due to
+failures or timeouts.
+
+`start_time_in_millis`::
+(Long)
+For a running search shows a timestamp when the eql search
+started, in milliseconds since the Unix epoch.
+
+`expiration_time_in_millis`::
+(long)
+Shows a timestamp when the eql search will be expired, in milliseconds
+since the Unix epoch. When this time is reached, the search and its results
+are deleted, even if the search is still ongoing.
+
+`completion_status`::
+(Integer)
+For a completed search shows the http status code of the completed
+search.
+
+
+[[eql-status-api-example]]
+==== {api-examples-title}
+
+[source,console]
+----
+GET /_eql/search/status/FmNJRUZ1YWZCU3dHY1BIOUhaenVSRkEaaXFlZ3h4c1RTWFNocDdnY2FSaERnUTozNDE=?keep_alive=5d
+----
+// TEST[skip: no access to search ID]
+
+If the search is still running, the status response has the following form:
+
+[source,console-result]
+--------------------------------------------------
+{
+  "id" : "FmNJRUZ1YWZCU3dHY1BIOUhaenVSRkEaaXFlZ3h4c1RTWFNocDdnY2FSaERnUTozNDE=",
+  "is_running" : true,
+  "is_partial" : true,
+  "start_time_in_millis" : 1611690235000,
+  "expiration_time_in_millis" : 1611690295000
+
+}
+--------------------------------------------------
+// TEST[skip: no access to search ID]
+
+If the search is completed the status response doesn't have
+`start_time_in_millis`, but has an additional `completion_status`
+field that shows the status code of the completed eql search:
+
+[source,console-result]
+--------------------------------------------------
+{
+  "id" : "FmNJRUZ1YWZCU3dHY1BIOUhaenVSRkEaaXFlZ3h4c1RTWFNocDdnY2FSaERnUTozNDE=",
+  "is_running" : false,
+  "is_partial" : false,
+  "expiration_time_in_millis" : 1611690295000,
+  "completion_status" : 200 <1>
+}
+--------------------------------------------------
+// TEST[skip: no access to search ID]
+
+<1> Indicates that the eql search was successfully completed

+ 3 - 0
docs/reference/search.asciidoc

@@ -45,6 +45,7 @@ For an overview of EQL and related tutorials, see <<eql>>.
 
 * <<eql-search-api>>
 * <<get-async-eql-search-api>>
+* <<get-async-eql-status-api>>
 * <<delete-async-eql-search-api>>
 
 
@@ -70,6 +71,8 @@ include::eql/eql-search-api.asciidoc[]
 
 include::eql/get-async-eql-search-api.asciidoc[]
 
+include::eql/get-async-eql-status-api.asciidoc[]
+
 include::eql/delete-async-eql-search-api.asciidoc[]
 
 include::search/count.asciidoc[]

+ 8 - 4
x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/AsyncSearchTask.java

@@ -350,12 +350,16 @@ final class AsyncSearchTask extends SearchTask implements AsyncTask {
     }
 
     /**
-     * Returns the status of {@link AsyncSearchTask}
+     * Returns the status from {@link AsyncSearchTask}
      */
-    public AsyncStatusResponse getStatusResponse() {
-        MutableSearchResponse mutableSearchResponse = searchResponse.get();
+    public static AsyncStatusResponse getStatusResponse(AsyncSearchTask asyncTask) {
+        MutableSearchResponse mutableSearchResponse = asyncTask.searchResponse.get();
         assert mutableSearchResponse != null;
-        return mutableSearchResponse.toStatusResponse(searchId.getEncoded(), getStartTime(), expirationTimeMillis);
+        return mutableSearchResponse.toStatusResponse(
+            asyncTask.searchId.getEncoded(),
+            asyncTask.getStartTime(),
+            asyncTask.expirationTimeMillis
+        );
     }
 
     class Listener extends SearchProgressActionListener {

+ 10 - 49
x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportGetAsyncStatusAction.java

@@ -6,7 +6,6 @@
  */
 package org.elasticsearch.xpack.search;
 
-import org.elasticsearch.ResourceNotFoundException;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.ActionListenerResponseHandler;
 import org.elasticsearch.action.support.ActionFilters;
@@ -21,7 +20,6 @@ import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.xpack.core.XPackPlugin;
 import org.elasticsearch.xpack.core.async.AsyncExecutionId;
-import org.elasticsearch.xpack.core.async.AsyncTask;
 import org.elasticsearch.xpack.core.async.AsyncTaskIndexService;
 import org.elasticsearch.xpack.core.async.GetAsyncStatusRequest;
 import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
@@ -55,56 +53,19 @@ public class TransportGetAsyncStatusAction extends HandledTransportAction<GetAsy
     protected void doExecute(Task task, GetAsyncStatusRequest request, ActionListener<AsyncStatusResponse> listener) {
         AsyncExecutionId searchId = AsyncExecutionId.decode(request.getId());
         DiscoveryNode node = clusterService.state().nodes().get(searchId.getTaskId().getNodeId());
-        if (node == null || Objects.equals(node, clusterService.localNode())) {
-            retrieveStatus(request, listener);
+        DiscoveryNode localNode = clusterService.state().getNodes().getLocalNode();
+        if (node == null || Objects.equals(node, localNode)) {
+            store.retrieveStatus(
+                request,
+                taskManager,
+                AsyncSearchTask.class,
+                AsyncSearchTask::getStatusResponse,
+                AsyncStatusResponse::getStatusFromStoredSearch,
+                listener
+            );
         } else {
             transportService.sendRequest(node, GetAsyncStatusAction.NAME, request,
                 new ActionListenerResponseHandler<>(listener, AsyncStatusResponse::new, ThreadPool.Names.SAME));
         }
     }
-
-    private void retrieveStatus(GetAsyncStatusRequest request, ActionListener<AsyncStatusResponse> listener) {
-        long nowInMillis = System.currentTimeMillis();
-        AsyncExecutionId searchId = AsyncExecutionId.decode(request.getId());
-        try {
-            AsyncTask task = (AsyncTask) taskManager.getTask(searchId.getTaskId().getId());
-            if ((task instanceof AsyncSearchTask) && (task.getExecutionId().equals(searchId))) {
-                AsyncStatusResponse response = ((AsyncSearchTask) task).getStatusResponse();
-                sendFinalResponse(request, response, nowInMillis, listener);
-            } else {
-                getStatusResponseFromIndex(searchId, request, nowInMillis, listener);
-            }
-        } catch (Exception exc) {
-            listener.onFailure(exc);
-        }
-    }
-
-    /**
-     * Get a status response from index
-     */
-    private void getStatusResponseFromIndex(AsyncExecutionId searchId,
-            GetAsyncStatusRequest request, long nowInMillis, ActionListener<AsyncStatusResponse> listener) {
-        store.getStatusResponse(searchId, AsyncStatusResponse::getStatusFromAsyncSearchResponseWithExpirationTime,
-            new ActionListener<>() {
-                @Override
-                public void onResponse(AsyncStatusResponse asyncStatusResponse) {
-                    sendFinalResponse(request, asyncStatusResponse, nowInMillis, listener);
-                }
-
-                @Override
-                public void onFailure(Exception e) {
-                    listener.onFailure(e);
-                }
-            }
-        );
-    }
-
-    private static void sendFinalResponse(GetAsyncStatusRequest request,
-            AsyncStatusResponse response, long nowInMillis, ActionListener<AsyncStatusResponse> listener) {
-        if (response.getExpirationTime() < nowInMillis) { // check if the result has expired
-            listener.onFailure(new ResourceNotFoundException(request.getId()));
-        } else {
-            listener.onResponse(response);
-        }
-    }
 }

+ 105 - 0
x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncStatusResponseTests.java

@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+import org.elasticsearch.common.Strings;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.xpack.core.search.action.AsyncStatusResponse;
+import org.elasticsearch.test.AbstractWireSerializingTestCase;
+
+import java.io.IOException;
+import java.util.Date;
+import static org.elasticsearch.xpack.core.async.GetAsyncResultRequestTests.randomSearchId;
+
+public class AsyncStatusResponseTests extends AbstractWireSerializingTestCase<AsyncStatusResponse> {
+
+    @Override
+    protected AsyncStatusResponse createTestInstance() {
+        String id = randomSearchId();
+        boolean isRunning = randomBoolean();
+        boolean isPartial = isRunning ? randomBoolean() : false;
+        long startTimeMillis = (new Date(randomLongBetween(0, 3000000000000L))).getTime();
+        long expirationTimeMillis = startTimeMillis + 3600000L;
+        int totalShards = randomIntBetween(10, 150);
+        int successfulShards = randomIntBetween(0, totalShards - 5);
+        int skippedShards = randomIntBetween(0, 5);
+        int failedShards = totalShards - successfulShards - skippedShards;
+        RestStatus completionStatus = isRunning ? null : randomBoolean() ? RestStatus.OK : RestStatus.SERVICE_UNAVAILABLE;
+        return new AsyncStatusResponse(
+            id,
+            isRunning,
+            isPartial,
+            startTimeMillis,
+            expirationTimeMillis,
+            totalShards,
+            successfulShards,
+            skippedShards,
+            failedShards,
+            completionStatus
+        );
+    }
+
+    @Override
+    protected Writeable.Reader<AsyncStatusResponse> instanceReader() {
+        return AsyncStatusResponse::new;
+    }
+
+    @Override
+    protected AsyncStatusResponse mutateInstance(AsyncStatusResponse instance) {
+        // return a response with the opposite running status
+        boolean isRunning = instance.isRunning() == false;
+        boolean isPartial = isRunning ? randomBoolean() : false;
+        RestStatus completionStatus = isRunning ? null : randomBoolean() ? RestStatus.OK : RestStatus.SERVICE_UNAVAILABLE;
+        return new AsyncStatusResponse(
+            instance.getId(),
+            isRunning,
+            isPartial,
+            instance.getStartTime(),
+            instance.getExpirationTime(),
+            instance.getTotalShards(),
+            instance.getSuccessfulShards(),
+            instance.getSkippedShards(),
+            instance.getFailedShards(),
+            completionStatus
+        );
+    }
+
+    public void testToXContent() throws IOException {
+        AsyncStatusResponse response = createTestInstance();
+        try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
+            String expectedJson = "{\n" +
+                "  \"id\" : \"" + response.getId() + "\",\n" +
+                "  \"is_running\" : " + response.isRunning() + ",\n" +
+                "  \"is_partial\" : " + response.isPartial() + ",\n" +
+                "  \"start_time_in_millis\" : " + response.getStartTime() + ",\n" +
+                "  \"expiration_time_in_millis\" : " + response.getExpirationTime() + ",\n" +
+                "  \"_shards\" : {\n" +
+                "    \"total\" : " + response.getTotalShards() + ",\n" +
+                "    \"successful\" : " + response.getSuccessfulShards() + ",\n" +
+                "    \"skipped\" : " + response.getSkippedShards() + ",\n" +
+                "    \"failed\" : " + response.getFailedShards() + "\n";
+            if (response.getCompletionStatus() == null) {
+                expectedJson = expectedJson +
+                    "  }\n" +
+                    "}";
+            } else {
+                expectedJson = expectedJson +
+                    "  },\n" +
+                    "  \"completion_status\" : " + response.getCompletionStatus().getStatus() + "\n" +
+                    "}";
+            }
+            builder.prettyPrint();
+            response.toXContent(builder, ToXContent.EMPTY_PARAMS);
+            assertEquals(expectedJson, Strings.toString(builder));
+        }
+    }
+}

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

@@ -20,6 +20,7 @@ import org.elasticsearch.client.Client;
 import org.elasticsearch.client.OriginSettingClient;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.TriFunction;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.io.stream.ByteBufferStreamInput;
@@ -38,7 +39,7 @@ import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.xpack.core.XPackPlugin;
 import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
-import org.elasticsearch.xpack.core.search.action.AsyncStatusResponse;
+import org.elasticsearch.xpack.core.search.action.SearchStatusResponse;
 import org.elasticsearch.xpack.core.security.SecurityContext;
 import org.elasticsearch.xpack.core.security.authc.Authentication;
 import org.elasticsearch.xpack.core.security.authc.support.AuthenticationContextSerializer;
@@ -51,7 +52,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.function.BiFunction;
+import java.util.function.Function;
 
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME;
@@ -336,18 +337,55 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         ));
     }
 
+    /**
+     * Retrieve the status of the async search or async or stored eql search.
+     * Retrieve from the task if the task is still available or from the index.
+     */
+     public <T extends AsyncTask, SR extends SearchStatusResponse> void retrieveStatus(
+            GetAsyncStatusRequest request,
+            TaskManager taskManager,
+            Class<T> tClass,
+            Function<T, SR> statusProducerFromTask,
+            TriFunction<R, Long, String, SR> statusProducerFromIndex,
+            ActionListener<SR> listener) {
+        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);
+                sendFinalStatusResponse(request, response, listener);
+            } else { // get status response from index
+                getStatusResponseFromIndex(asyncExecutionId, statusProducerFromIndex,
+                    new ActionListener<>() {
+                        @Override
+                        public void onResponse(SR searchStatusResponse) {
+                            sendFinalStatusResponse(request, searchStatusResponse, listener);
+                        }
+                        @Override
+                        public void onFailure(Exception e) {
+                            listener.onFailure(e);
+                        }
+                    }
+                );
+            }
+        } catch (Exception exc) {
+            listener.onFailure(exc);
+        }
+    }
 
     /**
-     * Gets the status response of the async search from the index
-     * @param asyncExecutionId – id of the async search
-     * @param statusProducer – a producer of the status from the stored async search response and expirationTime
+     * Gets the status response of the stored search from the index
+     * @param asyncExecutionId – id of the stored search (async search or stored eql search)
+     * @param statusProducer – a producer of a status from the stored search, expirationTime and async search id
      * @param listener – listener to report result to
      */
-    public void getStatusResponse(AsyncExecutionId asyncExecutionId,
-                                  BiFunction<R, Long, AsyncStatusResponse> statusProducer,
-                                  ActionListener<AsyncStatusResponse> listener) {
+    private <SR extends SearchStatusResponse> void getStatusResponseFromIndex(
+        AsyncExecutionId asyncExecutionId,
+        TriFunction<R, Long, String, SR> statusProducer,
+        ActionListener<SR> listener) {
+        String asyncId = asyncExecutionId.getEncoded();
         GetRequest internalGet = new GetRequest(index)
-            .preference(asyncExecutionId.getEncoded())
+            .preference(asyncId)
             .id(asyncExecutionId.getDocId());
         clientWithOrigin.get(internalGet, ActionListener.wrap(
             get -> {
@@ -358,7 +396,7 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
                 String encoded = (String) get.getSource().get(RESULT_FIELD);
                 if (encoded != null) {
                     Long expirationTime = (Long) get.getSource().get(EXPIRATION_TIME_FIELD);
-                    listener.onResponse(statusProducer.apply(decodeResponse(encoded), expirationTime));
+                    listener.onResponse(statusProducer.apply(decodeResponse(encoded), expirationTime, asyncId));
                 } else {
                     listener.onResponse(null);
                 }
@@ -367,6 +405,17 @@ public final class AsyncTaskIndexService<R extends AsyncResponse<R>> {
         ));
     }
 
+    private static <SR extends SearchStatusResponse> void sendFinalStatusResponse(
+        GetAsyncStatusRequest request,
+        SR response,
+        ActionListener<SR> listener) {
+        if (response.getExpirationTime() < System.currentTimeMillis()) { // check if the result has expired
+            listener.onFailure(new ResourceNotFoundException(request.getId()));
+        } else {
+            listener.onResponse(response);
+        }
+    }
+
     /**
      * Checks if the current user's authentication matches the original authentication stored
      * in the async search index.

+ 37 - 5
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/AsyncStatusResponse.java

@@ -17,13 +17,14 @@ import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.rest.action.RestActions;
 
 import java.io.IOException;
+import java.util.Objects;
 
 import static org.elasticsearch.rest.RestStatus.OK;
 
 /**
- * A response of an async search request.
+ * A response of an async search status request.
  */
-public class AsyncStatusResponse extends ActionResponse implements StatusToXContentObject {
+public class AsyncStatusResponse extends ActionResponse implements SearchStatusResponse, StatusToXContentObject {
     private final String id;
     private final boolean isRunning;
     private final boolean isPartial;
@@ -57,8 +58,15 @@ public class AsyncStatusResponse extends ActionResponse implements StatusToXCont
         this.completionStatus = completionStatus;
     }
 
-    public static AsyncStatusResponse getStatusFromAsyncSearchResponseWithExpirationTime(AsyncSearchResponse asyncSearchResponse,
-            long expirationTimeMillis) {
+    /**
+     * Get status from the stored async search response
+     * @param asyncSearchResponse stored async search response
+     * @param expirationTimeMillis – expiration time in milliseconds
+     * @param id – encoded async search id
+     * @return status response
+     */
+    public static AsyncStatusResponse getStatusFromStoredSearch(AsyncSearchResponse asyncSearchResponse,
+            long expirationTimeMillis, String id) {
         int totalShards = 0;
         int successfulShards = 0;
         int skippedShards = 0;
@@ -82,7 +90,7 @@ public class AsyncStatusResponse extends ActionResponse implements StatusToXCont
             }
         }
         return new AsyncStatusResponse(
-            asyncSearchResponse.getId(),
+            id,
             asyncSearchResponse.isRunning(),
             asyncSearchResponse.isPartial(),
             asyncSearchResponse.getStartTime(),
@@ -145,6 +153,29 @@ public class AsyncStatusResponse extends ActionResponse implements StatusToXCont
         return builder;
     }
 
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null || getClass() != obj.getClass()) return false;
+        AsyncStatusResponse other = (AsyncStatusResponse) obj;
+        return id.equals(other.id)
+            && isRunning == other.isRunning
+            && isPartial == other.isPartial
+            && startTimeMillis == other.startTimeMillis
+            && expirationTimeMillis == other.expirationTimeMillis
+            && totalShards == other.totalShards
+            && successfulShards == other.successfulShards
+            && skippedShards == other.skippedShards
+            && failedShards == other.failedShards
+            && Objects.equals(completionStatus, other.completionStatus);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(id, isRunning, isPartial, startTimeMillis, expirationTimeMillis, totalShards,
+            successfulShards, skippedShards, failedShards, completionStatus);
+    }
+
     /**
      * Returns the id of the async search status request.
      */
@@ -179,6 +210,7 @@ public class AsyncStatusResponse extends ActionResponse implements StatusToXCont
     /**
      * Returns a timestamp when the search will be expired, in milliseconds since epoch.
      */
+    @Override
     public long getExpirationTime() {
         return expirationTimeMillis;
     }

+ 19 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/SearchStatusResponse.java

@@ -0,0 +1,19 @@
+/*
+ * 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.search.action;
+
+/**
+ * An interface for status response of the stored or running async search
+ */
+public interface SearchStatusResponse {
+
+    /**
+     * Returns a timestamp when the search will be expired, in milliseconds since epoch.
+     */
+    long getExpirationTime();
+}

+ 50 - 0
x-pack/plugin/eql/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/eql/10_basic.yml

@@ -158,3 +158,53 @@ setup:
       catch: missing
       eql.delete:
         id: $id
+
+---
+"EQL status API":
+
+  - do:
+      eql.search:
+        index: eql_test
+        keep_on_completion: true
+        wait_for_completion_timeout: "0ms"
+        body:
+          query: 'process where user == "SYSTEM"'
+  - is_true: id
+  - set: { id: id }
+
+  # status for eql search that may be completed or not
+  - do:
+      eql.get_status:
+        id: $id
+  - match: { id: $id }
+  - set: { is_running: run_status }
+  - match: { is_partial: $run_status }
+  - is_true: expiration_time_in_millis
+
+  # wait for the certain completion
+  - do:
+      eql.get:
+        id: $id
+        wait_for_completion_timeout: "10s"
+
+  # status for completed eql search
+  - do:
+      eql.get_status:
+        id: $id
+  - match: { id: $id }
+  - match: { is_running: false }
+  - match: { is_partial: false }
+  - is_false: start_time_in_millis
+  - is_true: expiration_time_in_millis
+  - match: { completion_status: 200 }
+
+  - do:
+      eql.delete:
+        id: $id
+  - match: { acknowledged: true }
+
+  # status for a deleted/missing eql search
+  - do:
+      catch: missing
+      eql.get_status:
+        id: $id

+ 19 - 8
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchTask.java

@@ -13,12 +13,8 @@ import org.elasticsearch.xpack.core.async.AsyncExecutionId;
 import org.elasticsearch.xpack.eql.async.StoredAsyncTask;
 
 import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.atomic.AtomicReference;
 
 public class EqlSearchTask extends StoredAsyncTask<EqlSearchResponse> {
-    public volatile AtomicReference<EqlSearchResponse> finalResponse = new AtomicReference<>();
-
     public EqlSearchTask(long id, String type, String action, String description, TaskId parentTaskId, Map<String, String> headers,
                          Map<String, String> originHeaders, AsyncExecutionId asyncExecutionId, TimeValue keepAlive) {
         super(id, type, action, description, parentTaskId, headers, originHeaders, asyncExecutionId, keepAlive);
@@ -26,9 +22,24 @@ public class EqlSearchTask extends StoredAsyncTask<EqlSearchResponse> {
 
     @Override
     public EqlSearchResponse getCurrentResult() {
-        return Objects.requireNonNullElseGet(finalResponse.get(),
-            // we haven't seen the final response yet sending a initial response
-            () -> new EqlSearchResponse(EqlSearchResponse.Hits.EMPTY, System.currentTimeMillis() - getStartTime(), false,
-            getExecutionId().getEncoded(), true, true));
+        // for eql searches we never store a search response in the task (neither partial, nor final)
+        // we kill the task on final response, so if the task is still present, it means the search is still running
+        return new EqlSearchResponse(EqlSearchResponse.Hits.EMPTY, System.currentTimeMillis() - getStartTime(), false,
+            getExecutionId().getEncoded(), true, true);
+    }
+
+
+    /**
+     * Returns the status from {@link EqlSearchTask}
+     */
+    public static EqlStatusResponse getStatusResponse(EqlSearchTask asyncTask) {
+        return new EqlStatusResponse(
+            asyncTask.getExecutionId().getEncoded(),
+            true,
+            true,
+            asyncTask.getStartTime(),
+            asyncTask.getExpirationTimeMillis(),
+            null
+        );
     }
 }

+ 191 - 0
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlStatusResponse.java

@@ -0,0 +1,191 @@
+/*
+ * 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.eql.action;
+
+import org.elasticsearch.ExceptionsHelper;
+import org.elasticsearch.action.ActionResponse;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.StatusToXContentObject;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.xpack.core.search.action.SearchStatusResponse;
+import org.elasticsearch.xpack.eql.async.StoredAsyncResponse;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.elasticsearch.rest.RestStatus.OK;
+
+/**
+ * A response for eql search status request
+ */
+public class EqlStatusResponse extends ActionResponse implements SearchStatusResponse, StatusToXContentObject {
+    private final String id;
+    private final boolean isRunning;
+    private final boolean isPartial;
+    private final Long startTimeMillis;
+    private final long expirationTimeMillis;
+    private final RestStatus completionStatus;
+
+    public EqlStatusResponse(String id,
+            boolean isRunning,
+            boolean isPartial,
+            Long startTimeMillis,
+            long expirationTimeMillis,
+            RestStatus completionStatus) {
+        this.id = id;
+        this.isRunning = isRunning;
+        this.isPartial = isPartial;
+        this.startTimeMillis = startTimeMillis;
+        this.expirationTimeMillis = expirationTimeMillis;
+        this.completionStatus = completionStatus;
+    }
+
+    /**
+     * Get status from the stored eql search response
+     * @param storedResponse
+     * @param expirationTimeMillis – expiration time in milliseconds
+     * @param id – encoded async search id
+     * @return a status response
+     */
+    public static EqlStatusResponse getStatusFromStoredSearch(StoredAsyncResponse<EqlSearchResponse> storedResponse,
+            long expirationTimeMillis, String id) {
+        EqlSearchResponse searchResponse = storedResponse.getResponse();
+        if (searchResponse != null) {
+            assert searchResponse.isRunning() == false : "Stored eql search response must have a completed status!";
+            return new EqlStatusResponse(
+                searchResponse.id(),
+                false,
+                searchResponse.isPartial(),
+                null,  // we dont' store in the index start time for completed response
+                expirationTimeMillis,
+                RestStatus.OK
+            );
+        } else {
+            Exception exc = storedResponse.getException();
+            assert exc != null : "Stored eql response must either have a search response or an exception!";
+            return new EqlStatusResponse(
+                id,
+                false,
+                false,
+                null, // we dont' store in the index start time for completed response
+                expirationTimeMillis,
+                ExceptionsHelper.status(exc)
+            );
+        }
+    }
+
+    public EqlStatusResponse(StreamInput in) throws IOException {
+        this.id = in.readString();
+        this.isRunning = in.readBoolean();
+        this.isPartial = in.readBoolean();
+        this.startTimeMillis = in.readOptionalLong();
+        this.expirationTimeMillis = in.readLong();
+        this.completionStatus = (this.isRunning == false) ? RestStatus.readFrom(in) : null;
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        out.writeString(id);
+        out.writeBoolean(isRunning);
+        out.writeBoolean(isPartial);
+        out.writeOptionalLong(startTimeMillis);
+        out.writeLong(expirationTimeMillis);
+        if (isRunning == false) {
+            RestStatus.writeTo(out, completionStatus);
+        }
+    }
+
+    @Override
+    public RestStatus status() {
+        return OK;
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject();
+        builder.field("id", id);
+        builder.field("is_running", isRunning);
+        builder.field("is_partial", isPartial);
+        if (startTimeMillis != null) { // start time is available only for a running eql search
+            builder.timeField("start_time_in_millis", "start_time", startTimeMillis);
+        }
+        builder.timeField("expiration_time_in_millis", "expiration_time", expirationTimeMillis);
+        if (isRunning == false) { // completion status is available only for a completed eql search
+            builder.field("completion_status", completionStatus.getStatus());
+        }
+        builder.endObject();
+        return builder;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null || getClass() != obj.getClass()) return false;
+        EqlStatusResponse other = (EqlStatusResponse) obj;
+        return id.equals(other.id)
+            && isRunning == other.isRunning
+            && isPartial == other.isPartial
+            && Objects.equals(startTimeMillis, other.startTimeMillis)
+            && expirationTimeMillis == other.expirationTimeMillis
+            && Objects.equals(completionStatus, other.completionStatus);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(id, isRunning, isPartial, startTimeMillis, expirationTimeMillis, completionStatus);
+    }
+
+    /**
+     * Returns the id of the eql search status request.
+     */
+    public String getId() {
+        return id;
+    }
+
+    /**
+     * Returns {@code true} if the eql search is still running in the cluster,
+     * or {@code false} if the search has been completed.
+     */
+    public boolean isRunning() {
+        return isRunning;
+    }
+
+    /**
+     * Returns {@code true} if the eql search results are partial.
+     * This could be either because eql search hasn't finished yet,
+     * or if it finished and some shards have failed or timed out.
+     */
+    public boolean isPartial() {
+        return isPartial;
+    }
+
+    /**
+     * Returns a timestamp when the eql search task started, in milliseconds since epoch.
+     * For a completed eql search returns {@code null}, as we don't store start time for completed searches.
+     */
+    public Long getStartTime() {
+        return startTimeMillis;
+    }
+
+    /**
+     * Returns a timestamp when the eql search will be expired, in milliseconds since epoch.
+     */
+    @Override
+    public long getExpirationTime() {
+        return expirationTimeMillis;
+    }
+
+    /**
+     * For a completed eql search returns the completion status.
+     * For a still running eql search returns {@code null}.
+     */
+    public RestStatus getCompletionStatus() {
+        return completionStatus;
+    }
+}

+ 19 - 0
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlAsyncGetStatusAction.java

@@ -0,0 +1,19 @@
+/*
+ * 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.eql.plugin;
+
+import org.elasticsearch.action.ActionType;
+import org.elasticsearch.xpack.eql.action.EqlStatusResponse;
+
+public class EqlAsyncGetStatusAction extends ActionType<EqlStatusResponse> {
+    public static final EqlAsyncGetStatusAction INSTANCE = new EqlAsyncGetStatusAction();
+    public static final String NAME = "cluster:monitor/eql/async/status";
+
+    private EqlAsyncGetStatusAction() {
+        super(NAME, EqlStatusResponse::new);
+    }
+}

+ 2 - 0
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlPlugin.java

@@ -88,6 +88,7 @@ public class EqlPlugin extends Plugin implements ActionPlugin {
             new ActionHandler<>(EqlSearchAction.INSTANCE, TransportEqlSearchAction.class),
             new ActionHandler<>(EqlStatsAction.INSTANCE, TransportEqlStatsAction.class),
             new ActionHandler<>(EqlAsyncGetResultAction.INSTANCE, TransportEqlAsyncGetResultAction.class),
+            new ActionHandler<>(EqlAsyncGetStatusAction.INSTANCE, TransportEqlAsyncGetStatusAction.class),
             new ActionHandler<>(XPackUsageFeatureAction.EQL, EqlUsageTransportAction.class),
             new ActionHandler<>(XPackInfoFeatureAction.EQL, EqlInfoTransportAction.class)
         );
@@ -106,6 +107,7 @@ public class EqlPlugin extends Plugin implements ActionPlugin {
             new RestEqlSearchAction(),
             new RestEqlStatsAction(),
             new RestEqlGetAsyncResultAction(),
+            new RestEqlGetAsyncStatusAction(),
             new RestEqlDeleteAsyncResultAction()
         );
     }

+ 35 - 0
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/RestEqlGetAsyncStatusAction.java

@@ -0,0 +1,35 @@
+/*
+ * 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.eql.plugin;
+
+import org.elasticsearch.client.node.NodeClient;
+import org.elasticsearch.rest.BaseRestHandler;
+import org.elasticsearch.rest.RestRequest;
+import org.elasticsearch.rest.action.RestStatusToXContentListener;
+import org.elasticsearch.xpack.core.async.GetAsyncStatusRequest;
+
+import java.util.List;
+
+import static org.elasticsearch.rest.RestRequest.Method.GET;
+
+public class RestEqlGetAsyncStatusAction extends BaseRestHandler {
+    @Override
+    public List<Route> routes() {
+        return List.of(new Route(GET, "/_eql/search/status/{id}"));
+    }
+
+    @Override
+    public String getName() {
+        return "eql_get_async_status";
+    }
+
+    @Override
+    protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) {
+        GetAsyncStatusRequest statusRequest = new GetAsyncStatusRequest(request.param("id"));
+        return channel -> client.execute(EqlAsyncGetStatusAction.INSTANCE, statusRequest, new RestStatusToXContentListener<>(channel));
+    }
+}

+ 75 - 0
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/TransportEqlAsyncGetStatusAction.java

@@ -0,0 +1,75 @@
+/*
+ * 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.eql.plugin;
+
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.ActionListenerResponseHandler;
+import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.action.support.HandledTransportAction;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.tasks.Task;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xpack.core.XPackPlugin;
+import org.elasticsearch.xpack.core.async.AsyncExecutionId;
+import org.elasticsearch.xpack.core.async.AsyncTaskIndexService;
+import org.elasticsearch.xpack.core.async.GetAsyncStatusRequest;
+import org.elasticsearch.xpack.eql.action.EqlSearchResponse;
+import org.elasticsearch.xpack.eql.action.EqlSearchTask;
+import org.elasticsearch.xpack.eql.action.EqlStatusResponse;
+import org.elasticsearch.xpack.eql.async.StoredAsyncResponse;
+
+import java.util.Objects;
+
+import static org.elasticsearch.xpack.core.ClientHelper.ASYNC_SEARCH_ORIGIN;
+
+
+public class TransportEqlAsyncGetStatusAction extends HandledTransportAction<GetAsyncStatusRequest, EqlStatusResponse> {
+    private final TransportService transportService;
+    private final ClusterService clusterService;
+    private final AsyncTaskIndexService<StoredAsyncResponse<EqlSearchResponse>> store;
+
+    @Inject
+    public TransportEqlAsyncGetStatusAction(TransportService transportService,
+             ActionFilters actionFilters,
+             ClusterService clusterService,
+             NamedWriteableRegistry registry,
+             Client client,
+             ThreadPool threadPool) {
+        super(EqlAsyncGetStatusAction.NAME, transportService, actionFilters, GetAsyncStatusRequest::new);
+        this.transportService = transportService;
+        this.clusterService = clusterService;
+        Writeable.Reader<StoredAsyncResponse<EqlSearchResponse>> reader = in -> new StoredAsyncResponse<>(EqlSearchResponse::new, in);
+        this.store = new AsyncTaskIndexService<>(XPackPlugin.ASYNC_RESULTS_INDEX, clusterService,
+            threadPool.getThreadContext(), client, ASYNC_SEARCH_ORIGIN, reader, registry);
+    }
+
+    @Override
+    protected void doExecute(Task task, GetAsyncStatusRequest request, ActionListener<EqlStatusResponse> listener) {
+        AsyncExecutionId searchId = AsyncExecutionId.decode(request.getId());
+        DiscoveryNode node = clusterService.state().nodes().get(searchId.getTaskId().getNodeId());
+        DiscoveryNode localNode = clusterService.state().getNodes().getLocalNode();
+        if (node == null || Objects.equals(node, localNode)) {
+            store.retrieveStatus(
+                request,
+                taskManager,
+                EqlSearchTask.class,
+                EqlSearchTask::getStatusResponse,
+                EqlStatusResponse::getStatusFromStoredSearch,
+                listener
+            );
+        } else {
+            transportService.sendRequest(node, EqlAsyncGetStatusAction.NAME, request,
+                new ActionListenerResponseHandler<>(listener, EqlStatusResponse::new, ThreadPool.Names.SAME));
+        }
+    }
+}

+ 85 - 0
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlStatusResponseTests.java

@@ -0,0 +1,85 @@
+/*
+ * 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.eql.action;
+
+import org.elasticsearch.common.Strings;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.test.AbstractWireSerializingTestCase;
+
+import java.io.IOException;
+import java.util.Date;
+
+import static org.elasticsearch.xpack.core.async.GetAsyncResultRequestTests.randomSearchId;
+
+public class EqlStatusResponseTests extends AbstractWireSerializingTestCase<EqlStatusResponse> {
+
+    @Override
+    protected EqlStatusResponse createTestInstance() {
+        String id = randomSearchId();
+        boolean isRunning = randomBoolean();
+        boolean isPartial = isRunning ? randomBoolean() : false;
+        long randomDate = (new Date(randomLongBetween(0, 3000000000000L))).getTime();
+        Long startTimeMillis = randomBoolean() ? null : randomDate;
+        long expirationTimeMillis = startTimeMillis == null ? randomDate : startTimeMillis + 3600000L;
+        RestStatus completionStatus = isRunning ? null : randomBoolean() ? RestStatus.OK : RestStatus.SERVICE_UNAVAILABLE;
+        return new EqlStatusResponse(id, isRunning, isPartial, startTimeMillis, expirationTimeMillis, completionStatus);
+    }
+
+    @Override
+    protected Writeable.Reader<EqlStatusResponse> instanceReader() {
+        return EqlStatusResponse::new;
+    }
+
+    @Override
+    protected EqlStatusResponse mutateInstance(EqlStatusResponse instance) {
+        // return a response with the opposite running status
+        boolean isRunning = instance.isRunning() == false;
+        boolean isPartial = isRunning ? randomBoolean() : false;
+        RestStatus completionStatus = isRunning ? null : randomBoolean() ? RestStatus.OK : RestStatus.SERVICE_UNAVAILABLE;
+        return new EqlStatusResponse(
+            instance.getId(),
+            isRunning,
+            isPartial,
+            instance.getStartTime(),
+            instance.getExpirationTime(),
+            completionStatus
+        );
+    }
+
+    public void testToXContent() throws IOException {
+        EqlStatusResponse response = createTestInstance();
+        try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
+            String expectedJson = "{\n" +
+                "  \"id\" : \"" + response.getId() + "\",\n" +
+                "  \"is_running\" : " + response.isRunning() + ",\n" +
+                "  \"is_partial\" : " + response.isPartial() + ",\n";
+
+            if (response.getStartTime() != null) {
+                expectedJson = expectedJson +
+                    "  \"start_time_in_millis\" : " + response.getStartTime() + ",\n";
+            }
+            expectedJson = expectedJson +
+                "  \"expiration_time_in_millis\" : " + response.getExpirationTime();
+
+            if (response.getCompletionStatus() == null) {
+                expectedJson = expectedJson + "\n" +
+                    "}";
+            } else {
+                expectedJson = expectedJson + ",\n" +
+                    "  \"completion_status\" : " + response.getCompletionStatus().getStatus() + "\n" +
+                    "}";
+            }
+            builder.prettyPrint();
+            response.toXContent(builder, ToXContent.EMPTY_PARAMS);
+            assertEquals(expectedJson, Strings.toString(builder));
+        }
+    }
+}

+ 1 - 0
x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java

@@ -215,6 +215,7 @@ public class Constants {
         "cluster:monitor/ccr/stats",
         "cluster:monitor/data_frame/get",
         "cluster:monitor/data_frame/stats/get",
+        "cluster:monitor/eql/async/status",
         "cluster:monitor/health",
         "cluster:monitor/main",
         "cluster:monitor/nodes/hot_threads",

+ 31 - 0
x-pack/plugin/src/test/resources/rest-api-spec/api/eql.get_status.json

@@ -0,0 +1,31 @@
+{
+  "eql.get_status": {
+    "documentation": {
+      "url": "https://www.elastic.co/guide/en/elasticsearch/reference/current/eql-search-api.html",
+      "description": "Returns the status of a previously submitted async or stored Event Query Language (EQL) search"
+    },
+    "stability": "stable",
+    "visibility": "public",
+    "headers": {
+      "accept": [
+        "application/json"
+      ]
+    },
+    "url": {
+      "paths": [
+        {
+          "path": "/_eql/search/status/{id}",
+          "methods": [
+            "GET"
+          ],
+          "parts": {
+            "id": {
+              "type": "string",
+              "description": "The async search ID"
+            }
+          }
+        }
+      ]
+    }
+  }
+}