瀏覽代碼

Rename Task Persistence into Storing Task Results

The term persisted task was used to indicate that a task should store its results upon its completion. We would like to use this term to indicate that a task can survive restart of nodes instead. This commit removes usages of the term "persist" when it means store results.
Igor Motov 9 年之前
父節點
當前提交
10a766704e
共有 17 個文件被更改,包括 133 次插入132 次删除
  1. 2 2
      core/src/main/java/org/elasticsearch/action/ActionRequest.java
  2. 5 5
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskResponse.java
  3. 10 9
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
  4. 7 7
      core/src/main/java/org/elasticsearch/action/support/TransportAction.java
  5. 2 2
      core/src/main/java/org/elasticsearch/cluster/ClusterModule.java
  6. 2 2
      core/src/main/java/org/elasticsearch/node/Node.java
  7. 5 5
      core/src/main/java/org/elasticsearch/tasks/Task.java
  8. 1 1
      core/src/main/java/org/elasticsearch/tasks/TaskInfo.java
  9. 15 15
      core/src/main/java/org/elasticsearch/tasks/TaskManager.java
  10. 20 20
      core/src/main/java/org/elasticsearch/tasks/TaskResult.java
  11. 9 9
      core/src/main/java/org/elasticsearch/tasks/TaskResultsService.java
  12. 23 23
      core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java
  13. 9 9
      core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java
  14. 12 12
      core/src/test/java/org/elasticsearch/tasks/TaskResultTests.java
  15. 1 1
      modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java
  16. 7 7
      modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java
  17. 3 3
      modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java

+ 2 - 2
core/src/main/java/org/elasticsearch/action/ActionRequest.java

@@ -40,9 +40,9 @@ public abstract class ActionRequest<Request extends ActionRequest<Request>> exte
     public abstract ActionRequestValidationException validate();
 
     /**
-     * Should this task persist its result after it has finished?
+     * Should this task store its result after it has finished?
      */
-    public boolean getShouldPersistResult() {
+    public boolean getShouldStoreResult() {
         return false;
     }
 

+ 5 - 5
core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskResponse.java

@@ -25,7 +25,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.tasks.PersistedTaskInfo;
+import org.elasticsearch.tasks.TaskResult;
 
 import java.io.IOException;
 
@@ -35,19 +35,19 @@ import static java.util.Objects.requireNonNull;
  * Returns the list of tasks currently running on the nodes
  */
 public class GetTaskResponse extends ActionResponse implements ToXContent {
-    private PersistedTaskInfo task;
+    private TaskResult task;
 
     public GetTaskResponse() {
     }
 
-    public GetTaskResponse(PersistedTaskInfo task) {
+    public GetTaskResponse(TaskResult task) {
         this.task = requireNonNull(task, "task is required");
     }
 
     @Override
     public void readFrom(StreamInput in) throws IOException {
         super.readFrom(in);
-        task = in.readOptionalWriteable(PersistedTaskInfo::new);
+        task = in.readOptionalWriteable(TaskResult::new);
     }
 
     @Override
@@ -59,7 +59,7 @@ public class GetTaskResponse extends ActionResponse implements ToXContent {
     /**
      * Get the actual result of the fetch.
      */
-    public PersistedTaskInfo getTask() {
+    public TaskResult getTask() {
         return task;
     }
 

+ 10 - 9
core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java

@@ -38,11 +38,11 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.IndexNotFoundException;
-import org.elasticsearch.tasks.PersistedTaskInfo;
+import org.elasticsearch.tasks.TaskResult;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.tasks.TaskInfo;
-import org.elasticsearch.tasks.TaskPersistenceService;
+import org.elasticsearch.tasks.TaskResultsService;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportException;
 import org.elasticsearch.transport.TransportRequestOptions;
@@ -160,7 +160,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
                 });
             } else {
                 TaskInfo info = runningTask.taskInfo(clusterService.localNode(), true);
-                listener.onResponse(new GetTaskResponse(new PersistedTaskInfo(false, info)));
+                listener.onResponse(new GetTaskResponse(new TaskResult(false, info)));
             }
         }
     }
@@ -185,7 +185,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
                  * the error isn't a 404 then we'll just throw it back to the user.
                  */
                 if (ExceptionsHelper.unwrap(e, ResourceNotFoundException.class) != null) {
-                    listener.onResponse(new GetTaskResponse(new PersistedTaskInfo(true, snapshotOfRunningTask)));
+                    listener.onResponse(new GetTaskResponse(new TaskResult(true, snapshotOfRunningTask)));
                 } else {
                     listener.onFailure(e);
                 }
@@ -195,11 +195,11 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
 
     /**
      * Send a {@link GetRequest} to the tasks index looking for a persisted copy of the task completed task. It'll only be found only if the
-     * task's result was persisted. Called on the node that once had the task if that node is still part of the cluster or on the
+     * task's result was stored. Called on the node that once had the task if that node is still part of the cluster or on the
      * coordinating node if the node is no longer part of the cluster.
      */
     void getFinishedTaskFromIndex(Task thisTask, GetTaskRequest request, ActionListener<GetTaskResponse> listener) {
-        GetRequest get = new GetRequest(TaskPersistenceService.TASK_INDEX, TaskPersistenceService.TASK_TYPE,
+        GetRequest get = new GetRequest(TaskResultsService.TASK_INDEX, TaskResultsService.TASK_TYPE,
                 request.getTaskId().toString());
         get.setParentTask(clusterService.localNode().getId(), thisTask.getId());
         client.get(get, new ActionListener<GetResponse>() {
@@ -216,7 +216,8 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
             public void onFailure(Exception e) {
                 if (ExceptionsHelper.unwrap(e, IndexNotFoundException.class) != null) {
                     // We haven't yet created the index for the task results so it can't be found.
-                    listener.onFailure(new ResourceNotFoundException("task [{}] isn't running or persisted", e, request.getTaskId()));
+                    listener.onFailure(new ResourceNotFoundException("task [{}] isn't running or stored its results", e,
+                        request.getTaskId()));
                 } else {
                     listener.onFailure(e);
                 }
@@ -230,7 +231,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
      */
     void onGetFinishedTaskFromIndex(GetResponse response, ActionListener<GetTaskResponse> listener) throws IOException {
         if (false == response.isExists()) {
-            listener.onFailure(new ResourceNotFoundException("task [{}] isn't running or persisted", response.getId()));
+            listener.onFailure(new ResourceNotFoundException("task [{}] isn't running or stored its results", response.getId()));
             return;
         }
         if (response.isSourceEmpty()) {
@@ -238,7 +239,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
             return;
         }
         try (XContentParser parser = XContentHelper.createParser(response.getSourceAsBytesRef())) {
-            PersistedTaskInfo result = PersistedTaskInfo.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT);
+            TaskResult result = TaskResult.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT);
             listener.onResponse(new GetTaskResponse(result));
         }
     }

+ 7 - 7
core/src/main/java/org/elasticsearch/action/support/TransportAction.java

@@ -137,8 +137,8 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
             return;
         }
 
-        if (task != null && request.getShouldPersistResult()) {
-            listener = new PersistentActionListener<>(taskManager, task, listener);
+        if (task != null && request.getShouldStoreResult()) {
+            listener = new TaskResultStoringActionListener<>(taskManager, task, listener);
         }
 
         if (filters.length == 0) {
@@ -256,14 +256,14 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
     }
 
     /**
-     * Wrapper for an action listener that persists the result at the end of the execution
+     * Wrapper for an action listener that stores the result at the end of the execution
      */
-    private static class PersistentActionListener<Response extends ActionResponse> implements ActionListener<Response> {
+    private static class TaskResultStoringActionListener<Response extends ActionResponse> implements ActionListener<Response> {
         private final ActionListener<Response> delegate;
         private final Task task;
         private final TaskManager taskManager;
 
-        private  PersistentActionListener(TaskManager taskManager, Task task, ActionListener<Response> delegate) {
+        private TaskResultStoringActionListener(TaskManager taskManager, Task task, ActionListener<Response> delegate) {
             this.taskManager = taskManager;
             this.task = task;
             this.delegate = delegate;
@@ -272,7 +272,7 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
         @Override
         public void onResponse(Response response) {
             try {
-                taskManager.persistResult(task, response, delegate);
+                taskManager.storeResult(task, response, delegate);
             } catch (Exception e) {
                 delegate.onFailure(e);
             }
@@ -281,7 +281,7 @@ public abstract class TransportAction<Request extends ActionRequest<Request>, Re
         @Override
         public void onFailure(Exception e) {
             try {
-                taskManager.persistResult(task, e, delegate);
+                taskManager.storeResult(task, e, delegate);
             } catch (Exception inner) {
                 inner.addSuppressed(e);
                 delegate.onFailure(inner);

+ 2 - 2
core/src/main/java/org/elasticsearch/cluster/ClusterModule.java

@@ -61,7 +61,7 @@ import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.ExtensionPoint;
 import org.elasticsearch.gateway.GatewayAllocator;
-import org.elasticsearch.tasks.TaskPersistenceService;
+import org.elasticsearch.tasks.TaskResultsService;
 
 import java.util.Arrays;
 import java.util.Collections;
@@ -160,6 +160,6 @@ public class ClusterModule extends AbstractModule {
         bind(ShardStateAction.class).asEagerSingleton();
         bind(NodeMappingRefreshAction.class).asEagerSingleton();
         bind(MappingUpdatedAction.class).asEagerSingleton();
-        bind(TaskPersistenceService.class).asEagerSingleton();
+        bind(TaskResultsService.class).asEagerSingleton();
     }
 }

+ 2 - 2
core/src/main/java/org/elasticsearch/node/Node.java

@@ -108,7 +108,7 @@ import org.elasticsearch.search.SearchModule;
 import org.elasticsearch.search.SearchService;
 import org.elasticsearch.snapshots.SnapshotShardsService;
 import org.elasticsearch.snapshots.SnapshotsService;
-import org.elasticsearch.tasks.TaskPersistenceService;
+import org.elasticsearch.tasks.TaskResultsService;
 import org.elasticsearch.threadpool.ExecutorBuilder;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
@@ -466,7 +466,7 @@ public class Node implements Closeable {
 
         // Start the transport service now so the publish address will be added to the local disco node in ClusterService
         TransportService transportService = injector.getInstance(TransportService.class);
-        transportService.getTaskManager().setTaskResultsService(injector.getInstance(TaskPersistenceService.class));
+        transportService.getTaskManager().setTaskResultsService(injector.getInstance(TaskResultsService.class));
         transportService.start();
 
         validateNodeBeforeAcceptingRequests(settings, transportService.boundAddress());

+ 5 - 5
core/src/main/java/org/elasticsearch/tasks/Task.java

@@ -135,15 +135,15 @@ public class Task {
 
     public interface Status extends ToXContent, NamedWriteable {}
 
-    public PersistedTaskInfo result(DiscoveryNode node, Exception error) throws IOException {
-        return new PersistedTaskInfo(taskInfo(node, true), error);
+    public TaskResult result(DiscoveryNode node, Exception error) throws IOException {
+        return new TaskResult(taskInfo(node, true), error);
     }
 
-    public PersistedTaskInfo result(DiscoveryNode node, ActionResponse response) throws IOException {
+    public TaskResult result(DiscoveryNode node, ActionResponse response) throws IOException {
         if (response instanceof ToXContent) {
-            return new PersistedTaskInfo(taskInfo(node, true), (ToXContent) response);
+            return new TaskResult(taskInfo(node, true), (ToXContent) response);
         } else {
-            throw new IllegalStateException("response has to implement ToXContent for persistence");
+            throw new IllegalStateException("response has to implement ToXContent to be able to store the results");
         }
     }
 }

+ 1 - 1
core/src/main/java/org/elasticsearch/tasks/TaskInfo.java

@@ -207,7 +207,7 @@ public final class TaskInfo implements Writeable, ToXContent {
                 return new TaskInfo(id, type, action, description, status, startTime, runningTimeNanos, cancellable, parentTaskId);
             });
     static {
-        // Note for the future: this has to be backwards compatible with all changes to the task persistence format
+        // Note for the future: this has to be backwards compatible with all changes to the task storage format
         PARSER.declareString(constructorArg(), new ParseField("node"));
         PARSER.declareLong(constructorArg(), new ParseField("id"));
         PARSER.declareString(constructorArg(), new ParseField("type"));

+ 15 - 15
core/src/main/java/org/elasticsearch/tasks/TaskManager.java

@@ -63,7 +63,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
 
     private final Map<TaskId, String> banedParents = new ConcurrentHashMap<>();
 
-    private TaskPersistenceService taskResultsService;
+    private TaskResultsService taskResultsService;
 
     private DiscoveryNodes lastDiscoveryNodes = DiscoveryNodes.EMPTY_NODES;
 
@@ -71,7 +71,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
         super(settings);
     }
 
-    public void setTaskResultsService(TaskPersistenceService taskResultsService) {
+    public void setTaskResultsService(TaskResultsService taskResultsService) {
         assert this.taskResultsService == null;
         this.taskResultsService = taskResultsService;
     }
@@ -155,22 +155,22 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
     /**
      * Stores the task failure
      */
-    public <Response extends ActionResponse> void persistResult(Task task, Exception error, ActionListener<Response> listener) {
+    public <Response extends ActionResponse> void storeResult(Task task, Exception error, ActionListener<Response> listener) {
         DiscoveryNode localNode = lastDiscoveryNodes.getLocalNode();
         if (localNode == null) {
-            // too early to persist anything, shouldn't really be here - just pass the error along
+            // too early to store anything, shouldn't really be here - just pass the error along
             listener.onFailure(error);
             return;
         }
-        final PersistedTaskInfo taskResult;
+        final TaskResult taskResult;
         try {
             taskResult = task.result(localNode, error);
         } catch (IOException ex) {
-            logger.warn("couldn't persist error {}", ex, ExceptionsHelper.detailedMessage(error));
+            logger.warn("couldn't store error {}", ex, ExceptionsHelper.detailedMessage(error));
             listener.onFailure(ex);
             return;
         }
-        taskResultsService.persist(taskResult, new ActionListener<Void>() {
+        taskResultsService.storeResult(taskResult, new ActionListener<Void>() {
             @Override
             public void onResponse(Void aVoid) {
                 listener.onFailure(error);
@@ -178,7 +178,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
 
             @Override
             public void onFailure(Exception e) {
-                logger.warn("couldn't persist error {}", e, ExceptionsHelper.detailedMessage(error));
+                logger.warn("couldn't store error {}", e, ExceptionsHelper.detailedMessage(error));
                 listener.onFailure(e);
             }
         });
@@ -187,24 +187,24 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
     /**
      * Stores the task result
      */
-    public <Response extends ActionResponse> void persistResult(Task task, Response response, ActionListener<Response> listener) {
+    public <Response extends ActionResponse> void storeResult(Task task, Response response, ActionListener<Response> listener) {
         DiscoveryNode localNode = lastDiscoveryNodes.getLocalNode();
         if (localNode == null) {
-            // too early to persist anything, shouldn't really be here - just pass the response along
-            logger.warn("couldn't persist response {}, the node didn't join the cluster yet", response);
+            // too early to store anything, shouldn't really be here - just pass the response along
+            logger.warn("couldn't store response {}, the node didn't join the cluster yet", response);
             listener.onResponse(response);
             return;
         }
-        final PersistedTaskInfo taskResult;
+        final TaskResult taskResult;
         try {
             taskResult = task.result(localNode, response);
         } catch (IOException ex) {
-            logger.warn("couldn't persist response {}", ex, response);
+            logger.warn("couldn't store response {}", ex, response);
             listener.onFailure(ex);
             return;
         }
 
-        taskResultsService.persist(taskResult, new ActionListener<Void>() {
+        taskResultsService.storeResult(taskResult, new ActionListener<Void>() {
             @Override
             public void onResponse(Void aVoid) {
                 listener.onResponse(response);
@@ -212,7 +212,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
 
             @Override
             public void onFailure(Exception e) {
-                logger.warn("couldn't persist response {}", e, response);
+                logger.warn("couldn't store response {}", e, response);
                 listener.onFailure(e);
             }
         });

+ 20 - 20
core/src/main/java/org/elasticsearch/tasks/PersistedTaskInfo.java → core/src/main/java/org/elasticsearch/tasks/TaskResult.java

@@ -45,10 +45,10 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optiona
 import static org.elasticsearch.common.xcontent.XContentHelper.convertToMap;
 
 /**
- * Information about a persisted or running task. Running tasks just have a {@link #getTask()} while persisted tasks will have either a
- * {@link #getError()} or {@link #getResponse()}.
+ * Information about a running task or a task that stored its result. Running tasks just have a {@link #getTask()} while
+ * tasks with stored result will have either a {@link #getError()} or {@link #getResponse()}.
  */
-public final class PersistedTaskInfo implements Writeable, ToXContent {
+public final class TaskResult implements Writeable, ToXContent {
     private final boolean completed;
     private final TaskInfo task;
     @Nullable
@@ -57,28 +57,28 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
     private final BytesReference response;
 
     /**
-     * Construct a {@linkplain PersistedTaskInfo} for a task for which we don't have a result or error. That usually means that the task
+     * Construct a {@linkplain TaskResult} for a task for which we don't have a result or error. That usually means that the task
      * is incomplete, but it could also mean that we waited for the task to complete but it didn't save any error information.
      */
-    public PersistedTaskInfo(boolean completed, TaskInfo task) {
+    public TaskResult(boolean completed, TaskInfo task) {
         this(completed, task, null, null);
     }
 
     /**
-     * Construct a {@linkplain PersistedTaskInfo} for a task that completed with an error.
+     * Construct a {@linkplain TaskResult} for a task that completed with an error.
      */
-    public PersistedTaskInfo(TaskInfo task, Exception error) throws IOException {
+    public TaskResult(TaskInfo task, Exception error) throws IOException {
         this(true, task, toXContent(error), null);
     }
 
     /**
-     * Construct a {@linkplain PersistedTaskInfo} for a task that completed successfully.
+     * Construct a {@linkplain TaskResult} for a task that completed successfully.
      */
-    public PersistedTaskInfo(TaskInfo task, ToXContent response) throws IOException {
+    public TaskResult(TaskInfo task, ToXContent response) throws IOException {
         this(true, task, null, toXContent(response));
     }
 
-    private PersistedTaskInfo(boolean completed, TaskInfo task, @Nullable BytesReference error, @Nullable BytesReference result) {
+    private TaskResult(boolean completed, TaskInfo task, @Nullable BytesReference error, @Nullable BytesReference result) {
         this.completed = completed;
         this.task = requireNonNull(task, "task is required");
         this.error = error;
@@ -88,7 +88,7 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
     /**
      * Read from a stream.
      */
-    public PersistedTaskInfo(StreamInput in) throws IOException {
+    public TaskResult(StreamInput in) throws IOException {
         completed = in.readBoolean();
         task = new TaskInfo(in);
         error = in.readOptionalBytesReference();
@@ -112,7 +112,7 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
 
     /**
      * Get the error that finished this task. Will return null if the task didn't finish with an error, it hasn't yet finished, or didn't
-     * persist its result.
+     * store its result.
      */
     public BytesReference getError() {
         return error;
@@ -120,7 +120,7 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
 
     /**
      * Convert {@link #getError()} from XContent to a Map for easy processing. Will return an empty map if the task didn't finish with an
-     * error, hasn't yet finished, or didn't persist its result.
+     * error, hasn't yet finished, or didn't store its result.
      */
     public Map<String, Object> getErrorAsMap() {
         if (error == null) {
@@ -131,7 +131,7 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
 
     /**
      * Get the response that this task finished with. Will return null if the task was finished by an error, it hasn't yet finished, or
-     * didn't persist its result.
+     * didn't store its result.
      */
     public BytesReference getResponse() {
         return response;
@@ -139,7 +139,7 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
 
     /**
      * Convert {@link #getResponse()} from XContent to a Map for easy processing. Will return an empty map if the task was finished with an
-     * error, hasn't yet finished, or didn't persist its result.
+     * error, hasn't yet finished, or didn't store its result.
      */
     public Map<String, Object> getResponseAsMap() {
         if (response == null) {
@@ -171,14 +171,14 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
         return builder;
     }
 
-    public static final ConstructingObjectParser<PersistedTaskInfo, ParseFieldMatcherSupplier> PARSER = new ConstructingObjectParser<>(
-            "persisted_task_info", a -> {
+    public static final ConstructingObjectParser<TaskResult, ParseFieldMatcherSupplier> PARSER = new ConstructingObjectParser<>(
+            "stored_task_result", a -> {
                 int i = 0;
                 boolean completed = (boolean) a[i++];
                 TaskInfo task = (TaskInfo) a[i++];
                 BytesReference error = (BytesReference) a[i++];
                 BytesReference response = (BytesReference) a[i++];
-                return new PersistedTaskInfo(completed, task, error, response);
+                return new TaskResult(completed, task, error, response);
             });
     static {
         PARSER.declareBoolean(constructorArg(), new ParseField("completed"));
@@ -195,10 +195,10 @@ public final class PersistedTaskInfo implements Writeable, ToXContent {
     // Implements equals and hashcode for testing
     @Override
     public boolean equals(Object obj) {
-        if (obj == null || obj.getClass() != PersistedTaskInfo.class) {
+        if (obj == null || obj.getClass() != TaskResult.class) {
             return false;
         }
-        PersistedTaskInfo other = (PersistedTaskInfo) obj;
+        TaskResult other = (TaskResult) obj;
         /*
          * Equality of error and result is done by converting them to a map first. Not efficient but ignores field order and spacing
          * differences so perfect for testing.

+ 9 - 9
core/src/main/java/org/elasticsearch/tasks/TaskPersistenceService.java → core/src/main/java/org/elasticsearch/tasks/TaskResultsService.java

@@ -47,9 +47,9 @@ import java.io.IOException;
 import java.io.InputStream;
 
 /**
- * Service that can persist tasks and their results.
+ * Service that can store task results.
  */
-public class TaskPersistenceService extends AbstractComponent {
+public class TaskResultsService extends AbstractComponent {
 
     public static final String TASK_INDEX = ".tasks";
 
@@ -64,7 +64,7 @@ public class TaskPersistenceService extends AbstractComponent {
     private final TransportCreateIndexAction createIndexAction;
 
     @Inject
-    public TaskPersistenceService(Settings settings, Client client, ClusterService clusterService,
+    public TaskResultsService(Settings settings, Client client, ClusterService clusterService,
                               TransportCreateIndexAction createIndexAction) {
         super(settings);
         this.client = client;
@@ -72,7 +72,7 @@ public class TaskPersistenceService extends AbstractComponent {
         this.createIndexAction = createIndexAction;
     }
 
-    public void persist(PersistedTaskInfo taskResult, ActionListener<Void> listener) {
+    public void storeResult(TaskResult taskResult, ActionListener<Void> listener) {
 
         ClusterState state = clusterService.state();
 
@@ -86,7 +86,7 @@ public class TaskPersistenceService extends AbstractComponent {
             createIndexAction.execute(null, createIndexRequest, new ActionListener<CreateIndexResponse>() {
                 @Override
                 public void onResponse(CreateIndexResponse result) {
-                    doPersist(taskResult, listener);
+                    doStoreResult(taskResult, listener);
                 }
 
                 @Override
@@ -94,7 +94,7 @@ public class TaskPersistenceService extends AbstractComponent {
                     if (ExceptionsHelper.unwrapCause(e) instanceof IndexAlreadyExistsException) {
                         // we have the index, do it
                         try {
-                            doPersist(taskResult, listener);
+                            doStoreResult(taskResult, listener);
                         } catch (Exception inner) {
                             inner.addSuppressed(e);
                             listener.onFailure(inner);
@@ -112,7 +112,7 @@ public class TaskPersistenceService extends AbstractComponent {
                     .execute(new ActionListener<PutMappingResponse>() {
                                  @Override
                                  public void onResponse(PutMappingResponse putMappingResponse) {
-                                     doPersist(taskResult, listener);
+                                     doStoreResult(taskResult, listener);
                                  }
 
                                  @Override
@@ -122,13 +122,13 @@ public class TaskPersistenceService extends AbstractComponent {
                              }
                     );
             } else {
-                doPersist(taskResult, listener);
+                doStoreResult(taskResult, listener);
             }
         }
     }
 
 
-    private void doPersist(PersistedTaskInfo taskResult, ActionListener<Void> listener) {
+    private void doStoreResult(TaskResult taskResult, ActionListener<Void> listener) {
         IndexRequestBuilder index = client.prepareIndex(TASK_INDEX, TASK_TYPE, taskResult.getTask().getTaskId().toString());
         try (XContentBuilder builder = XContentFactory.contentBuilder(Requests.INDEX_CONTENT_TYPE)) {
             taskResult.toXContent(builder, ToXContent.EMPTY_PARAMS);

+ 23 - 23
core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java

@@ -46,11 +46,11 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.tasks.PersistedTaskInfo;
+import org.elasticsearch.tasks.TaskResult;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.tasks.TaskInfo;
-import org.elasticsearch.tasks.TaskPersistenceService;
+import org.elasticsearch.tasks.TaskResultsService;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.tasks.MockTaskManager;
 import org.elasticsearch.test.tasks.MockTaskManagerListener;
@@ -452,39 +452,39 @@ public class TasksIT extends ESIntegTestCase {
         });
     }
 
-    public void testGetTaskWaitForCompletionNoPersist() throws Exception {
+    public void testGetTaskWaitForCompletionWithoutStoringResult() throws Exception {
         waitForCompletionTestCase(false, id -> {
             return client().admin().cluster().prepareGetTask(id).setWaitForCompletion(true).execute();
         }, response -> {
             assertNotNull(response.getTask().getTask());
             assertTrue(response.getTask().isCompleted());
-            // We didn't persist the result so it won't come back when we wait
+            // We didn't store the result so it won't come back when we wait
             assertNull(response.getTask().getResponse());
         });
     }
 
-    public void testGetTaskWaitForCompletionWithPersist() throws Exception {
+    public void testGetTaskWaitForCompletionWithStoringResult() throws Exception {
         waitForCompletionTestCase(true, id -> {
             return client().admin().cluster().prepareGetTask(id).setWaitForCompletion(true).execute();
         }, response -> {
             assertNotNull(response.getTask().getTask());
             assertTrue(response.getTask().isCompleted());
-            // We persisted the task so we should get its results
+            // We stored the task so we should get its results
             assertEquals(0, response.getTask().getResponseAsMap().get("failure_count"));
         });
     }
 
     /**
      * Test wait for completion.
-     * @param persist should the task persist its results
+     * @param storeResult should the task store its results
      * @param wait start waiting for a task. Accepts that id of the task to wait for and returns a future waiting for it.
      * @param validator validate the response and return the task ids that were found
      */
-    private <T> void waitForCompletionTestCase(boolean persist, Function<TaskId, ListenableActionFuture<T>> wait, Consumer<T> validator)
+    private <T> void waitForCompletionTestCase(boolean storeResult, Function<TaskId, ListenableActionFuture<T>> wait, Consumer<T> validator)
             throws Exception {
         // Start blocking test task
         ListenableActionFuture<TestTaskPlugin.NodesResponse> future = TestTaskPlugin.TestTaskAction.INSTANCE.newRequestBuilder(client())
-                .setShouldPersistResult(persist).execute();
+                .setShouldStoreResult(storeResult).execute();
 
         ListenableActionFuture<T> waitResponseFuture;
         TaskId taskId;
@@ -622,17 +622,17 @@ public class TasksIT extends ESIntegTestCase {
         assertThat(response.getTasks().size(), greaterThanOrEqualTo(1));
     }
 
-    public void testTaskResultPersistence() throws Exception {
+    public void testTaskStoringSuccesfulResult() throws Exception {
         // Randomly create an empty index to make sure the type is created automatically
         if (randomBoolean()) {
             logger.info("creating an empty results index with custom settings");
-            assertAcked(client().admin().indices().prepareCreate(TaskPersistenceService.TASK_INDEX));
+            assertAcked(client().admin().indices().prepareCreate(TaskResultsService.TASK_INDEX));
         }
 
         registerTaskManageListeners(TestTaskPlugin.TestTaskAction.NAME);  // we need this to get task id of the process
 
         // Start non-blocking test task
-        TestTaskPlugin.TestTaskAction.INSTANCE.newRequestBuilder(client()).setShouldPersistResult(true).setShouldBlock(false).get();
+        TestTaskPlugin.TestTaskAction.INSTANCE.newRequestBuilder(client()).setShouldStoreResult(true).setShouldBlock(false).get();
 
         List<TaskInfo> events = findEvents(TestTaskPlugin.TestTaskAction.NAME, Tuple::v1);
 
@@ -641,7 +641,7 @@ public class TasksIT extends ESIntegTestCase {
         TaskId taskId = taskInfo.getTaskId();
 
         GetResponse resultDoc = client()
-                .prepareGet(TaskPersistenceService.TASK_INDEX, TaskPersistenceService.TASK_TYPE, taskId.toString()).get();
+                .prepareGet(TaskResultsService.TASK_INDEX, TaskResultsService.TASK_TYPE, taskId.toString()).get();
         assertTrue(resultDoc.isExists());
 
         Map<String, Object> source = resultDoc.getSource();
@@ -657,16 +657,16 @@ public class TasksIT extends ESIntegTestCase {
 
         assertNull(source.get("failure"));
 
-        assertNoFailures(client().admin().indices().prepareRefresh(TaskPersistenceService.TASK_INDEX).get());
+        assertNoFailures(client().admin().indices().prepareRefresh(TaskResultsService.TASK_INDEX).get());
 
-        SearchResponse searchResponse = client().prepareSearch(TaskPersistenceService.TASK_INDEX)
-            .setTypes(TaskPersistenceService.TASK_TYPE)
+        SearchResponse searchResponse = client().prepareSearch(TaskResultsService.TASK_INDEX)
+            .setTypes(TaskResultsService.TASK_TYPE)
             .setSource(SearchSourceBuilder.searchSource().query(QueryBuilders.termQuery("task.action", taskInfo.getAction())))
             .get();
 
         assertEquals(1L, searchResponse.getHits().totalHits());
 
-        searchResponse = client().prepareSearch(TaskPersistenceService.TASK_INDEX).setTypes(TaskPersistenceService.TASK_TYPE)
+        searchResponse = client().prepareSearch(TaskResultsService.TASK_INDEX).setTypes(TaskResultsService.TASK_TYPE)
                 .setSource(SearchSourceBuilder.searchSource().query(QueryBuilders.termQuery("task.node", taskInfo.getTaskId().getNodeId())))
                 .get();
 
@@ -677,14 +677,14 @@ public class TasksIT extends ESIntegTestCase {
         assertNull(getResponse.getTask().getError());
     }
 
-    public void testTaskFailurePersistence() throws Exception {
+    public void testTaskStoringFailureResult() throws Exception {
         registerTaskManageListeners(TestTaskPlugin.TestTaskAction.NAME);  // we need this to get task id of the process
 
         // Start non-blocking test task that should fail
         assertThrows(
             TestTaskPlugin.TestTaskAction.INSTANCE.newRequestBuilder(client())
                 .setShouldFail(true)
-                .setShouldPersistResult(true)
+                .setShouldStoreResult(true)
                 .setShouldBlock(false),
             IllegalStateException.class
         );
@@ -695,7 +695,7 @@ public class TasksIT extends ESIntegTestCase {
         TaskId failedTaskId = failedTaskInfo.getTaskId();
 
         GetResponse failedResultDoc = client()
-            .prepareGet(TaskPersistenceService.TASK_INDEX, TaskPersistenceService.TASK_TYPE, failedTaskId.toString())
+            .prepareGet(TaskResultsService.TASK_INDEX, TaskResultsService.TASK_TYPE, failedTaskId.toString())
             .get();
         assertTrue(failedResultDoc.isExists());
 
@@ -729,9 +729,9 @@ public class TasksIT extends ESIntegTestCase {
     public void testNodeNotFoundButTaskFound() throws Exception {
         // Save a fake task that looks like it is from a node that isn't part of the cluster
         CyclicBarrier b = new CyclicBarrier(2);
-        TaskPersistenceService resultsService = internalCluster().getInstance(TaskPersistenceService.class);
-        resultsService.persist(
-                new PersistedTaskInfo(new TaskInfo(new TaskId("fake", 1), "test", "test", "", null, 0, 0, false, TaskId.EMPTY_TASK_ID),
+        TaskResultsService resultsService = internalCluster().getInstance(TaskResultsService.class);
+        resultsService.storeResult(
+                new TaskResult(new TaskInfo(new TaskId("fake", 1), "test", "test", "", null, 0, 0, false, TaskId.EMPTY_TASK_ID),
                         new RuntimeException("test")),
                 new ActionListener<Void>() {
                     @Override

+ 9 - 9
core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java

@@ -179,7 +179,7 @@ public class TestTaskPlugin extends Plugin implements ActionPlugin {
 
     public static class NodesRequest extends BaseNodesRequest<NodesRequest> {
         private String requestName;
-        private boolean shouldPersistResult = false;
+        private boolean shouldStoreResult = false;
         private boolean shouldBlock = true;
         private boolean shouldFail = false;
 
@@ -192,13 +192,13 @@ public class TestTaskPlugin extends Plugin implements ActionPlugin {
             this.requestName = requestName;
         }
 
-        public void setShouldPersistResult(boolean shouldPersistResult) {
-            this.shouldPersistResult = shouldPersistResult;
+        public void setShouldStoreResult(boolean shouldStoreResult) {
+            this.shouldStoreResult = shouldStoreResult;
         }
 
         @Override
-        public boolean getShouldPersistResult() {
-            return shouldPersistResult;
+        public boolean getShouldStoreResult() {
+            return shouldStoreResult;
         }
 
         public void setShouldBlock(boolean shouldBlock) {
@@ -221,7 +221,7 @@ public class TestTaskPlugin extends Plugin implements ActionPlugin {
         public void readFrom(StreamInput in) throws IOException {
             super.readFrom(in);
             requestName = in.readString();
-            shouldPersistResult = in.readBoolean();
+            shouldStoreResult = in.readBoolean();
             shouldBlock = in.readBoolean();
             shouldFail = in.readBoolean();
         }
@@ -230,7 +230,7 @@ public class TestTaskPlugin extends Plugin implements ActionPlugin {
         public void writeTo(StreamOutput out) throws IOException {
             super.writeTo(out);
             out.writeString(requestName);
-            out.writeBoolean(shouldPersistResult);
+            out.writeBoolean(shouldStoreResult);
             out.writeBoolean(shouldBlock);
             out.writeBoolean(shouldFail);
         }
@@ -336,8 +336,8 @@ public class TestTaskPlugin extends Plugin implements ActionPlugin {
         }
 
 
-        public NodesRequestBuilder setShouldPersistResult(boolean shouldPersistResult) {
-            request().setShouldPersistResult(shouldPersistResult);
+        public NodesRequestBuilder setShouldStoreResult(boolean shouldStoreResult) {
+            request().setShouldStoreResult(shouldStoreResult);
             return this;
         }
 

+ 12 - 12
core/src/test/java/org/elasticsearch/tasks/PersistedTaskInfoTests.java → core/src/test/java/org/elasticsearch/tasks/TaskResultTests.java

@@ -38,18 +38,18 @@ import java.util.Map;
 import java.util.TreeMap;
 
 /**
- * Round trip tests for {@link PersistedTaskInfo} and those classes that it includes like {@link TaskInfo} and {@link RawTaskStatus}.
+ * Round trip tests for {@link TaskResult} and those classes that it includes like {@link TaskInfo} and {@link RawTaskStatus}.
  */
-public class PersistedTaskInfoTests extends ESTestCase {
+public class TaskResultTests extends ESTestCase {
     public void testBinaryRoundTrip() throws IOException {
         NamedWriteableRegistry registry = new NamedWriteableRegistry(Collections.singletonList(
             new NamedWriteableRegistry.Entry(Task.Status.class, RawTaskStatus.NAME, RawTaskStatus::new)));
-        PersistedTaskInfo result = randomTaskResult();
-        PersistedTaskInfo read;
+        TaskResult result = randomTaskResult();
+        TaskResult read;
         try (BytesStreamOutput out = new BytesStreamOutput()) {
             result.writeTo(out);
             try (StreamInput in = new NamedWriteableAwareStreamInput(out.bytes().streamInput(), registry)) {
-                read = new PersistedTaskInfo(in);
+                read = new TaskResult(in);
             }
         } catch (IOException e) {
             throw new IOException("Error processing [" + result + "]", e);
@@ -62,13 +62,13 @@ public class PersistedTaskInfoTests extends ESTestCase {
          * Note that this round trip isn't 100% perfect - status will always be read as RawTaskStatus. Since this test uses RawTaskStatus
          * as the status we randomly generate then we can assert the round trip with .equals.
          */
-        PersistedTaskInfo result = randomTaskResult();
-        PersistedTaskInfo read;
+        TaskResult result = randomTaskResult();
+        TaskResult read;
         try (XContentBuilder builder = XContentBuilder.builder(randomFrom(XContentType.values()).xContent())) {
             result.toXContent(builder, ToXContent.EMPTY_PARAMS);
             try (XContentBuilder shuffled = shuffleXContent(builder);
                     XContentParser parser = XContentHelper.createParser(shuffled.bytes())) {
-                read = PersistedTaskInfo.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT);
+                read = TaskResult.PARSER.apply(parser, () -> ParseFieldMatcher.STRICT);
             }
         } catch (IOException e) {
             throw new IOException("Error processing [" + result + "]", e);
@@ -76,14 +76,14 @@ public class PersistedTaskInfoTests extends ESTestCase {
         assertEquals(result, read);
     }
 
-    private static PersistedTaskInfo randomTaskResult() throws IOException {
+    private static TaskResult randomTaskResult() throws IOException {
         switch (between(0, 2)) {
         case 0:
-            return new PersistedTaskInfo(randomBoolean(), randomTaskInfo());
+            return new TaskResult(randomBoolean(), randomTaskInfo());
         case 1:
-            return new PersistedTaskInfo(randomTaskInfo(), new RuntimeException("error"));
+            return new TaskResult(randomTaskInfo(), new RuntimeException("error"));
         case 2:
-            return new PersistedTaskInfo(randomTaskInfo(), randomTaskResponse());
+            return new TaskResult(randomTaskInfo(), randomTaskResponse());
         default:
             throw new UnsupportedOperationException("Unsupported random TaskResult constructor");
         }

+ 1 - 1
modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java

@@ -77,7 +77,7 @@ public abstract class AbstractBaseReindexRestHandler<
             client.executeLocally(action, internal, new BulkIndexByScrollResponseContentListener(channel, params));
             return;
         } else {
-            internal.setShouldPersistResult(true);
+            internal.setShouldStoreResult(true);
         }
 
         /*

+ 7 - 7
modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java

@@ -94,9 +94,9 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
     private float requestsPerSecond = Float.POSITIVE_INFINITY;
 
     /**
-     * Should this task persist its result?
+     * Should this task store its result?
      */
-    private boolean shouldPersistResult;
+    private boolean shouldStoreResult;
 
     public AbstractBulkByScrollRequest() {
     }
@@ -301,16 +301,16 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
     }
 
     /**
-     * Should this task persist its result after it has finished?
+     * Should this task store its result after it has finished?
      */
-    public Self setShouldPersistResult(boolean shouldPersistResult) {
-        this.shouldPersistResult = shouldPersistResult;
+    public Self setShouldStoreResult(boolean shouldStoreResult) {
+        this.shouldStoreResult = shouldStoreResult;
         return self();
     }
 
     @Override
-    public boolean getShouldPersistResult() {
-        return shouldPersistResult;
+    public boolean getShouldStoreResult() {
+        return shouldStoreResult;
     }
 
     @Override

+ 3 - 3
modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java

@@ -135,10 +135,10 @@ public abstract class AbstractBulkByScrollRequestBuilder<
     }
 
     /**
-     * Should this task persist its result after it has finished?
+     * Should this task store its result after it has finished?
      */
-    public Self setShouldPersistResult(boolean shouldPersistResult) {
-        request.setShouldPersistResult(shouldPersistResult);
+    public Self setShouldStoreResult(boolean shouldStoreResult) {
+        request.setShouldStoreResult(shouldStoreResult);
         return self();
     }
 }