Skip to content

Commit

Permalink
Uncouple persistent task state and status (#31031)
Browse files Browse the repository at this point in the history
This pull request removes the relationship between the state 
of persistent task (as stored in the cluster state) and the status 
of the task (as reported by the Task APIs and used in various 
places) that have been confusing for some time (#29608).

In order to do that, a new PersistentTaskState interface is added. 
This interface represents the persisted state of a persistent task. 
The methods used to update the state of persistent tasks are 
renamed: updatePersistentStatus() becomes updatePersistentTaskState() 
and now takes a PersistentTaskState as a parameter. The 
Task.Status type as been changed to PersistentTaskState in all 
places were it make sense (in persistent task customs in cluster 
state and all other methods that deal with the state of an allocated 
persistent task).
  • Loading branch information
tlrx committed Jun 15, 2018
1 parent 8c6ee7d commit 992c788
Show file tree
Hide file tree
Showing 42 changed files with 404 additions and 391 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskManager;

Expand Down Expand Up @@ -77,8 +76,9 @@ public Status getStatus() {
* <p>
* This doesn't affect the status of this allocated task.
*/
public void updatePersistentStatus(Task.Status status, ActionListener<PersistentTasksCustomMetaData.PersistentTask<?>> listener) {
persistentTasksService.updateStatus(persistentTaskId, allocationId, status, listener);
public void updatePersistentTaskState(final PersistentTaskState state,
final ActionListener<PersistentTasksCustomMetaData.PersistentTask<?>> listener) {
persistentTasksService.sendUpdateStateRequest(persistentTaskId, allocationId, state, listener);
}

public String getPersistentTaskId() {
Expand Down Expand Up @@ -116,7 +116,7 @@ public void waitForPersistentTask(final Predicate<PersistentTasksCustomMetaData.
}

protected final boolean isCompleted() {
return state.get() == State.COMPLETED;
return state.get() == State.COMPLETED;
}

boolean markAsCancelled() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;

/**
Expand All @@ -29,16 +28,17 @@
* It abstracts away the execution of tasks and greatly simplifies testing of PersistentTasksNodeService
*/
public class NodePersistentTasksExecutor {

private final ThreadPool threadPool;

public NodePersistentTasksExecutor(ThreadPool threadPool) {
NodePersistentTasksExecutor(ThreadPool threadPool) {
this.threadPool = threadPool;
}

public <Params extends PersistentTaskParams> void executeTask(Params params,
@Nullable Task.Status status,
AllocatedPersistentTask task,
PersistentTasksExecutor<Params> executor) {
public <Params extends PersistentTaskParams> void executeTask(final Params params,
final @Nullable PersistentTaskState state,
final AllocatedPersistentTask task,
final PersistentTasksExecutor<Params> executor) {
threadPool.executor(executor.getExecutor()).execute(new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
Expand All @@ -49,14 +49,12 @@ public void onFailure(Exception e) {
@Override
protected void doRun() throws Exception {
try {
executor.nodeOperation(task, params, status);
executor.nodeOperation(task, params, state);
} catch (Exception ex) {
task.markAsFailed(ex);
}

}
});

}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.persistent;

import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.xcontent.ToXContentObject;

/**
* {@link PersistentTaskState} represents the state of the persistent tasks, as it
* is persisted in the cluster state.
*/
public interface PersistentTaskState extends ToXContentObject, NamedWriteable {
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask;
import org.elasticsearch.persistent.decider.AssignmentDecision;
import org.elasticsearch.persistent.decider.EnableAssignmentDecider;
import org.elasticsearch.tasks.Task;

import java.util.Objects;

Expand Down Expand Up @@ -178,27 +177,30 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS
}

/**
* Update task status
* Update the state of a persistent task
*
* @param id the id of a persistent task
* @param allocationId the expected allocation id of the persistent task
* @param status new status
* @param listener the listener that will be called when task is removed
* @param taskId the id of a persistent task
* @param taskAllocationId the expected allocation id of the persistent task
* @param taskState new state
* @param listener the listener that will be called when task is removed
*/
public void updatePersistentTaskStatus(String id, long allocationId, Task.Status status, ActionListener<PersistentTask<?>> listener) {
clusterService.submitStateUpdateTask("update task status", new ClusterStateUpdateTask() {
public void updatePersistentTaskState(final String taskId,
final long taskAllocationId,
final PersistentTaskState taskState,
final ActionListener<PersistentTask<?>> listener) {
clusterService.submitStateUpdateTask("update task state", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) {
PersistentTasksCustomMetaData.Builder tasksInProgress = builder(currentState);
if (tasksInProgress.hasTask(id, allocationId)) {
return update(currentState, tasksInProgress.updateTaskStatus(id, status));
if (tasksInProgress.hasTask(taskId, taskAllocationId)) {
return update(currentState, tasksInProgress.updateTaskState(taskId, taskState));
} else {
if (tasksInProgress.hasTask(id)) {
logger.warn("trying to update status on task {} with unexpected allocation id {}", id, allocationId);
if (tasksInProgress.hasTask(taskId)) {
logger.warn("trying to update state on task {} with unexpected allocation id {}", taskId, taskAllocationId);
} else {
logger.warn("trying to update status on non-existing task {}", id);
logger.warn("trying to update state on non-existing task {}", taskId);
}
throw new ResourceNotFoundException("the task with id {} and allocation id {} doesn't exist", id, allocationId);
throw new ResourceNotFoundException("the task with id {} and allocation id {} doesn't exist", taskId, taskAllocationId);
}
}

Expand All @@ -209,7 +211,7 @@ public void onFailure(String source, Exception e) {

@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
listener.onResponse(PersistentTasksCustomMetaData.getTaskWithId(newState, id));
listener.onResponse(PersistentTasksCustomMetaData.getTaskWithId(newState, taskId));
}
});
}
Expand Down
Loading

0 comments on commit 992c788

Please sign in to comment.