mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-24 23:27:25 -04:00
Fix trappy timeouts in persistent tasks requests (#120514)
Ensure that callers constructing these master-node requests pass in an explicit timeout. Relates #107984
This commit is contained in:
parent
7563e71e90
commit
a1fd7bc374
30 changed files with 188 additions and 111 deletions
|
@ -113,7 +113,7 @@ public class PersistentTaskCreationFailureIT extends ESIntegTestCase {
|
|||
UUIDs.base64UUID(),
|
||||
FailingCreationPersistentTaskExecutor.TASK_NAME,
|
||||
new FailingCreationTaskParams(),
|
||||
null,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
l.map(ignored -> null)
|
||||
)
|
||||
);
|
||||
|
|
|
@ -50,7 +50,7 @@ public class PersistentTaskInitializationFailureIT extends ESIntegTestCase {
|
|||
UUIDs.base64UUID(),
|
||||
FailingInitializationPersistentTaskExecutor.TASK_NAME,
|
||||
new FailingInitializationTaskParams(),
|
||||
null,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
startPersistentTaskFuture
|
||||
);
|
||||
startPersistentTaskFuture.actionGet();
|
||||
|
|
|
@ -45,7 +45,7 @@ public class PersistentTasksExecutorFullRestartIT extends ESIntegTestCase {
|
|||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
futures.add(future);
|
||||
taskIds[i] = UUIDs.base64UUID();
|
||||
service.sendStartRequest(taskIds[i], TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future);
|
||||
service.sendStartRequest(taskIds[i], TestPersistentTasksExecutor.NAME, new TestParams("Blah"), TEST_REQUEST_TIMEOUT, future);
|
||||
}
|
||||
|
||||
for (int i = 0; i < numberOfTasks; i++) {
|
||||
|
|
|
@ -69,7 +69,13 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
public void testPersistentActionFailure() throws Exception {
|
||||
PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class);
|
||||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
UUIDs.base64UUID(),
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams("Blah"),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
long allocationId = future.get().getAllocationId();
|
||||
waitForTaskToStart();
|
||||
TaskInfo firstRunningTask = clusterAdmin().prepareListTasks()
|
||||
|
@ -100,7 +106,13 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class);
|
||||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
String taskId = UUIDs.base64UUID();
|
||||
persistentTasksService.sendStartRequest(taskId, TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
taskId,
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams("Blah"),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
long allocationId = future.get().getAllocationId();
|
||||
waitForTaskToStart();
|
||||
TaskInfo firstRunningTask = clusterAdmin().prepareListTasks()
|
||||
|
@ -119,7 +131,14 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
logger.info("Simulating errant completion notification");
|
||||
// try sending completion request with incorrect allocation id
|
||||
PlainActionFuture<PersistentTask<?>> failedCompletionNotificationFuture = new PlainActionFuture<>();
|
||||
persistentTasksService.sendCompletionRequest(taskId, Long.MAX_VALUE, null, null, null, failedCompletionNotificationFuture);
|
||||
persistentTasksService.sendCompletionRequest(
|
||||
taskId,
|
||||
Long.MAX_VALUE,
|
||||
null,
|
||||
null,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
failedCompletionNotificationFuture
|
||||
);
|
||||
assertFutureThrows(failedCompletionNotificationFuture, ResourceNotFoundException.class);
|
||||
// Make sure that the task is still running
|
||||
assertThat(
|
||||
|
@ -141,7 +160,13 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
TestParams testParams = new TestParams("Blah");
|
||||
testParams.setExecutorNodeAttr("test");
|
||||
persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, testParams, null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
UUIDs.base64UUID(),
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
testParams,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
String taskId = future.get().getId();
|
||||
|
||||
Settings nodeSettings = Settings.builder().put(nodeSettings(0, Settings.EMPTY)).put("node.attr.test_attr", "test").build();
|
||||
|
@ -165,7 +190,7 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
|
||||
// Remove the persistent task
|
||||
PlainActionFuture<PersistentTask<?>> removeFuture = new PlainActionFuture<>();
|
||||
persistentTasksService.sendRemoveRequest(taskId, null, removeFuture);
|
||||
persistentTasksService.sendRemoveRequest(taskId, TEST_REQUEST_TIMEOUT, removeFuture);
|
||||
assertEquals(removeFuture.get().getId(), taskId);
|
||||
}
|
||||
|
||||
|
@ -182,7 +207,13 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class);
|
||||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
TestParams testParams = new TestParams("Blah");
|
||||
persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, testParams, null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
UUIDs.base64UUID(),
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
testParams,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
String taskId = future.get().getId();
|
||||
|
||||
assertThat(clusterAdmin().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]").get().getTasks(), empty());
|
||||
|
@ -197,14 +228,20 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
|
||||
// Remove the persistent task
|
||||
PlainActionFuture<PersistentTask<?>> removeFuture = new PlainActionFuture<>();
|
||||
persistentTasksService.sendRemoveRequest(taskId, null, removeFuture);
|
||||
persistentTasksService.sendRemoveRequest(taskId, TEST_REQUEST_TIMEOUT, removeFuture);
|
||||
assertEquals(removeFuture.get().getId(), taskId);
|
||||
}
|
||||
|
||||
public void testPersistentActionStatusUpdate() throws Exception {
|
||||
PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class);
|
||||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
UUIDs.base64UUID(),
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams("Blah"),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
String taskId = future.get().getId();
|
||||
waitForTaskToStart();
|
||||
TaskInfo firstRunningTask = clusterAdmin().prepareListTasks()
|
||||
|
@ -250,7 +287,7 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
assertFutureThrows(future1, IllegalStateException.class, "timed out after 10ms");
|
||||
|
||||
PlainActionFuture<PersistentTask<?>> failedUpdateFuture = new PlainActionFuture<>();
|
||||
persistentTasksService.sendUpdateStateRequest(taskId, -2, new State("should fail"), null, failedUpdateFuture);
|
||||
persistentTasksService.sendUpdateStateRequest(taskId, -2, new State("should fail"), TEST_REQUEST_TIMEOUT, failedUpdateFuture);
|
||||
assertFutureThrows(
|
||||
failedUpdateFuture,
|
||||
ResourceNotFoundException.class,
|
||||
|
@ -275,11 +312,23 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class);
|
||||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
String taskId = UUIDs.base64UUID();
|
||||
persistentTasksService.sendStartRequest(taskId, TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
taskId,
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams("Blah"),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
future.get();
|
||||
|
||||
PlainActionFuture<PersistentTask<TestParams>> future2 = new PlainActionFuture<>();
|
||||
persistentTasksService.sendStartRequest(taskId, TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future2);
|
||||
persistentTasksService.sendStartRequest(
|
||||
taskId,
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams("Blah"),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future2
|
||||
);
|
||||
assertFutureThrows(future2, ResourceAlreadyExistsException.class);
|
||||
|
||||
waitForTaskToStart();
|
||||
|
@ -315,7 +364,13 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
TestParams testParams = new TestParams("Blah");
|
||||
testParams.setExecutorNodeAttr("test");
|
||||
persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, testParams, null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
UUIDs.base64UUID(),
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
testParams,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
PersistentTask<TestParams> task = future.get();
|
||||
String taskId = task.getId();
|
||||
|
||||
|
@ -366,7 +421,13 @@ public class PersistentTasksExecutorIT extends ESIntegTestCase {
|
|||
persistentTasksClusterService.setRecheckInterval(TimeValue.timeValueMillis(1));
|
||||
PersistentTasksService persistentTasksService = internalCluster().getInstance(PersistentTasksService.class);
|
||||
PlainActionFuture<PersistentTask<TestParams>> future = new PlainActionFuture<>();
|
||||
persistentTasksService.sendStartRequest(UUIDs.base64UUID(), TestPersistentTasksExecutor.NAME, new TestParams("Blah"), null, future);
|
||||
persistentTasksService.sendStartRequest(
|
||||
UUIDs.base64UUID(),
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams("Blah"),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
future
|
||||
);
|
||||
String taskId = future.get().getId();
|
||||
long allocationId = future.get().getAllocationId();
|
||||
waitForTaskToStart();
|
||||
|
|
|
@ -52,7 +52,7 @@ public class EnableAssignmentDeciderIT extends ESIntegTestCase {
|
|||
"task_" + i,
|
||||
TestPersistentTasksExecutor.NAME,
|
||||
new TestParams(randomAlphaOfLength(10)),
|
||||
null,
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
ActionListener.running(latch::countDown)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.indices.SystemIndices;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.persistent.PersistentTasksService;
|
||||
|
@ -95,7 +96,7 @@ public class TransportPostFeatureUpgradeAction extends TransportMasterNodeAction
|
|||
SYSTEM_INDEX_UPGRADE_TASK_NAME,
|
||||
SYSTEM_INDEX_UPGRADE_TASK_NAME,
|
||||
new SystemIndexMigrationTaskParams(),
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be configurable? longer by default? infinite? */,
|
||||
ActionListener.wrap(startedTask -> {
|
||||
listener.onResponse(new PostFeatureUpgradeResponse(true, featuresToMigrate, null, null));
|
||||
}, ex -> {
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
|||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.features.FeatureService;
|
||||
import org.elasticsearch.node.NodeClosedException;
|
||||
import org.elasticsearch.persistent.AllocatedPersistentTask;
|
||||
|
@ -162,7 +163,7 @@ public final class HealthNodeTaskExecutor extends PersistentTasksExecutor<Health
|
|||
TASK_NAME,
|
||||
TASK_NAME,
|
||||
new HealthNodeTaskParams(),
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be configurable? longer by default? infinite? */,
|
||||
ActionListener.wrap(r -> logger.debug("Created the health node task"), e -> {
|
||||
if (e instanceof NodeClosedException) {
|
||||
logger.debug("Failed to create health node task because node is shutting down", e);
|
||||
|
|
|
@ -65,7 +65,13 @@ public class AllocatedPersistentTask extends CancellableTask {
|
|||
final PersistentTaskState state,
|
||||
final ActionListener<PersistentTasksCustomMetadata.PersistentTask<?>> listener
|
||||
) {
|
||||
persistentTasksService.sendUpdateStateRequest(persistentTaskId, allocationId, state, null, listener);
|
||||
persistentTasksService.sendUpdateStateRequest(
|
||||
persistentTaskId,
|
||||
allocationId,
|
||||
state,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be longer? infinite? */,
|
||||
listener
|
||||
);
|
||||
}
|
||||
|
||||
public String getPersistentTaskId() {
|
||||
|
@ -201,7 +207,7 @@ public class AllocatedPersistentTask extends CancellableTask {
|
|||
getAllocationId(),
|
||||
failure,
|
||||
localAbortReason,
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be longer? infinite? */,
|
||||
new ActionListener<>() {
|
||||
@Override
|
||||
public void onResponse(PersistentTasksCustomMetadata.PersistentTask<?> persistentTask) {
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
|||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -56,8 +57,8 @@ public class CompletionPersistentTaskAction {
|
|||
localAbortReason = in.readOptionalString();
|
||||
}
|
||||
|
||||
public Request(String taskId, long allocationId, Exception exception, String localAbortReason) {
|
||||
super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT);
|
||||
public Request(TimeValue masterNodeTimeout, String taskId, long allocationId, Exception exception, String localAbortReason) {
|
||||
super(masterNodeTimeout);
|
||||
this.taskId = taskId;
|
||||
this.exception = exception;
|
||||
this.allocationId = allocationId;
|
||||
|
|
|
@ -18,6 +18,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.gateway.GatewayService;
|
||||
import org.elasticsearch.persistent.PersistentTasksCustomMetadata.PersistentTask;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
|
@ -310,7 +311,7 @@ public class PersistentTasksNodeService implements ClusterStateListener {
|
|||
taskInProgress.getAllocationId(),
|
||||
originalException,
|
||||
null,
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be longer? infinite? */,
|
||||
new ActionListener<>() {
|
||||
@Override
|
||||
public void onResponse(PersistentTask<?> persistentTask) {
|
||||
|
@ -346,7 +347,7 @@ public class PersistentTasksNodeService implements ClusterStateListener {
|
|||
if (task.markAsCancelled()) {
|
||||
// Cancel the local task using the task manager
|
||||
String reason = "task has been removed, cancelling locally";
|
||||
persistentTasksService.sendCancelRequest(task.getId(), reason, null, new ActionListener<>() {
|
||||
persistentTasksService.sendCancelRequest(task.getId(), reason, new ActionListener<>() {
|
||||
@Override
|
||||
public void onResponse(ListTasksResponse cancelTasksResponse) {
|
||||
logger.trace(
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.persistent.PersistentTasksCustomMetadata.PersistentTask
|
|||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
/**
|
||||
|
@ -57,16 +58,16 @@ public class PersistentTasksService {
|
|||
final String taskId,
|
||||
final String taskName,
|
||||
final Params taskParams,
|
||||
final @Nullable TimeValue timeout,
|
||||
final TimeValue timeout,
|
||||
final ActionListener<PersistentTask<Params>> listener
|
||||
) {
|
||||
@SuppressWarnings("unchecked")
|
||||
final ActionListener<PersistentTask<?>> wrappedListener = listener.map(t -> (PersistentTask<Params>) t);
|
||||
StartPersistentTaskAction.Request request = new StartPersistentTaskAction.Request(taskId, taskName, taskParams);
|
||||
if (timeout != null) {
|
||||
request.masterNodeTimeout(timeout);
|
||||
}
|
||||
execute(request, StartPersistentTaskAction.INSTANCE, wrappedListener);
|
||||
execute(
|
||||
new StartPersistentTaskAction.Request(Objects.requireNonNull(timeout), taskId, taskName, taskParams),
|
||||
StartPersistentTaskAction.INSTANCE,
|
||||
wrappedListener
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -85,33 +86,27 @@ public class PersistentTasksService {
|
|||
final @Nullable TimeValue timeout,
|
||||
final ActionListener<PersistentTask<?>> listener
|
||||
) {
|
||||
CompletionPersistentTaskAction.Request request = new CompletionPersistentTaskAction.Request(
|
||||
execute(
|
||||
new CompletionPersistentTaskAction.Request(
|
||||
Objects.requireNonNull(timeout),
|
||||
taskId,
|
||||
taskAllocationId,
|
||||
taskFailure,
|
||||
localAbortReason
|
||||
),
|
||||
CompletionPersistentTaskAction.INSTANCE,
|
||||
listener
|
||||
);
|
||||
if (timeout != null) {
|
||||
request.masterNodeTimeout(timeout);
|
||||
}
|
||||
execute(request, CompletionPersistentTaskAction.INSTANCE, listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cancels a locally running task using the Task Manager API. Accepts operation timeout as optional parameter
|
||||
*/
|
||||
void sendCancelRequest(
|
||||
final long taskId,
|
||||
final String reason,
|
||||
final @Nullable TimeValue timeout,
|
||||
final ActionListener<ListTasksResponse> listener
|
||||
) {
|
||||
void sendCancelRequest(final long taskId, final String reason, final ActionListener<ListTasksResponse> listener) {
|
||||
CancelTasksRequest request = new CancelTasksRequest();
|
||||
request.setTargetTaskId(new TaskId(clusterService.localNode().getId(), taskId));
|
||||
request.setReason(reason);
|
||||
if (timeout != null) {
|
||||
request.setTimeout(timeout);
|
||||
}
|
||||
// TODO set timeout?
|
||||
try {
|
||||
client.admin().cluster().cancelTasks(request, listener);
|
||||
} catch (Exception e) {
|
||||
|
@ -130,33 +125,25 @@ public class PersistentTasksService {
|
|||
final String taskId,
|
||||
final long taskAllocationID,
|
||||
final PersistentTaskState taskState,
|
||||
final @Nullable TimeValue timeout,
|
||||
final TimeValue timeout,
|
||||
final ActionListener<PersistentTask<?>> listener
|
||||
) {
|
||||
UpdatePersistentTaskStatusAction.Request request = new UpdatePersistentTaskStatusAction.Request(
|
||||
taskId,
|
||||
taskAllocationID,
|
||||
taskState
|
||||
execute(
|
||||
new UpdatePersistentTaskStatusAction.Request(Objects.requireNonNull(timeout), taskId, taskAllocationID, taskState),
|
||||
UpdatePersistentTaskStatusAction.INSTANCE,
|
||||
listener
|
||||
);
|
||||
if (timeout != null) {
|
||||
request.masterNodeTimeout(timeout);
|
||||
}
|
||||
execute(request, UpdatePersistentTaskStatusAction.INSTANCE, listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Notifies the master node to remove a persistent task from the cluster state. Accepts operation timeout as optional parameter
|
||||
*/
|
||||
public void sendRemoveRequest(
|
||||
final String taskId,
|
||||
final @Nullable TimeValue timeout,
|
||||
final ActionListener<PersistentTask<?>> listener
|
||||
) {
|
||||
RemovePersistentTaskAction.Request request = new RemovePersistentTaskAction.Request(taskId);
|
||||
if (timeout != null) {
|
||||
request.masterNodeTimeout(timeout);
|
||||
}
|
||||
execute(request, RemovePersistentTaskAction.INSTANCE, listener);
|
||||
public void sendRemoveRequest(final String taskId, final TimeValue timeout, final ActionListener<PersistentTask<?>> listener) {
|
||||
execute(
|
||||
new RemovePersistentTaskAction.Request(Objects.requireNonNull(timeout), taskId),
|
||||
RemovePersistentTaskAction.INSTANCE,
|
||||
listener
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
|||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -44,8 +45,8 @@ public class RemovePersistentTaskAction {
|
|||
taskId = in.readString();
|
||||
}
|
||||
|
||||
public Request(String taskId) {
|
||||
super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT);
|
||||
public Request(TimeValue masterNodeTimeout, String taskId) {
|
||||
super(masterNodeTimeout);
|
||||
this.taskId = taskId;
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
|||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -52,8 +53,8 @@ public class StartPersistentTaskAction {
|
|||
params = in.readNamedWriteable(PersistentTaskParams.class);
|
||||
}
|
||||
|
||||
public Request(String taskId, String taskName, PersistentTaskParams params) {
|
||||
super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT);
|
||||
public Request(TimeValue masterNodeTimeout, String taskId, String taskName, PersistentTaskParams params) {
|
||||
super(masterNodeTimeout);
|
||||
this.taskId = taskId;
|
||||
this.taskName = taskName;
|
||||
this.params = params;
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
|||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -49,8 +50,8 @@ public class UpdatePersistentTaskStatusAction {
|
|||
state = in.readOptionalNamedWriteable(PersistentTaskState.class);
|
||||
}
|
||||
|
||||
public Request(String taskId, long allocationId, PersistentTaskState state) {
|
||||
super(TRAPPY_IMPLICIT_DEFAULT_MASTER_NODE_TIMEOUT);
|
||||
public Request(TimeValue masterNodeTimeout, String taskId, long allocationId, PersistentTaskState state) {
|
||||
super(masterNodeTimeout);
|
||||
this.taskId = taskId;
|
||||
this.allocationId = allocationId;
|
||||
this.state = state;
|
||||
|
|
|
@ -42,6 +42,7 @@ import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
|
|||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.isNotNull;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.times;
|
||||
|
@ -102,7 +103,7 @@ public class HealthNodeTaskExecutorTests extends ESTestCase {
|
|||
eq("health-node"),
|
||||
eq("health-node"),
|
||||
eq(new HealthNodeTaskParams()),
|
||||
eq(null),
|
||||
isNotNull(),
|
||||
any()
|
||||
)
|
||||
);
|
||||
|
@ -121,7 +122,7 @@ public class HealthNodeTaskExecutorTests extends ESTestCase {
|
|||
eq("health-node"),
|
||||
eq("health-node"),
|
||||
eq(new HealthNodeTaskParams()),
|
||||
eq(null),
|
||||
isNotNull(),
|
||||
any()
|
||||
);
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ public class CancelPersistentTaskRequestTests extends AbstractWireSerializingTes
|
|||
|
||||
@Override
|
||||
protected Request createTestInstance() {
|
||||
return new Request(randomAsciiOfLength(10));
|
||||
return new Request(randomTimeValue(), randomAsciiOfLength(10));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -17,9 +17,9 @@ public class CompletionPersistentTaskRequestTests extends AbstractWireSerializin
|
|||
@Override
|
||||
protected Request createTestInstance() {
|
||||
if (randomBoolean()) {
|
||||
return new Request(randomAlphaOfLength(10), randomNonNegativeLong(), null, null);
|
||||
return new Request(randomTimeValue(), randomAlphaOfLength(10), randomNonNegativeLong(), null, null);
|
||||
} else {
|
||||
return new Request(randomAlphaOfLength(10), randomNonNegativeLong(), null, randomAlphaOfLength(20));
|
||||
return new Request(randomTimeValue(), randomAlphaOfLength(10), randomNonNegativeLong(), null, randomAlphaOfLength(20));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -260,12 +260,7 @@ public class PersistentTasksNodeServiceTests extends ESTestCase {
|
|||
when(client.settings()).thenReturn(Settings.EMPTY);
|
||||
PersistentTasksService persistentTasksService = new PersistentTasksService(null, null, client) {
|
||||
@Override
|
||||
void sendCancelRequest(
|
||||
final long taskId,
|
||||
final String reason,
|
||||
final TimeValue timeout,
|
||||
final ActionListener<ListTasksResponse> listener
|
||||
) {
|
||||
void sendCancelRequest(final long taskId, final String reason, final ActionListener<ListTasksResponse> listener) {
|
||||
capturedTaskId.set(taskId);
|
||||
capturedListener.set(listener);
|
||||
}
|
||||
|
@ -356,12 +351,7 @@ public class PersistentTasksNodeServiceTests extends ESTestCase {
|
|||
when(client.settings()).thenReturn(Settings.EMPTY);
|
||||
PersistentTasksService persistentTasksService = new PersistentTasksService(null, null, client) {
|
||||
@Override
|
||||
void sendCancelRequest(
|
||||
final long taskId,
|
||||
final String reason,
|
||||
final TimeValue timeout,
|
||||
final ActionListener<ListTasksResponse> listener
|
||||
) {
|
||||
void sendCancelRequest(final long taskId, final String reason, final ActionListener<ListTasksResponse> listener) {
|
||||
fail("Shouldn't be called during local abort");
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ public class StartPersistentActionRequestTests extends AbstractWireSerializingTe
|
|||
if (randomBoolean()) {
|
||||
testParams.setExecutorNodeAttr(randomAlphaOfLengthBetween(1, 20));
|
||||
}
|
||||
return new Request(UUIDs.base64UUID(), randomAlphaOfLengthBetween(1, 20), testParams);
|
||||
return new Request(randomTimeValue(), UUIDs.base64UUID(), randomAlphaOfLengthBetween(1, 20), testParams);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,7 @@ public class UpdatePersistentTaskRequestTests extends AbstractWireSerializingTes
|
|||
|
||||
@Override
|
||||
protected Request createTestInstance() {
|
||||
return new Request(UUIDs.base64UUID(), randomLong(), new State(randomAlphaOfLength(10)));
|
||||
return new Request(randomTimeValue(), UUIDs.base64UUID(), randomLong(), new State(randomAlphaOfLength(10)));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRunnable;
|
||||
import org.elasticsearch.action.support.master.MasterNodeRequest;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
|
@ -110,6 +111,7 @@ public final class ShardFollowTaskCleaner implements ClusterStateListener {
|
|||
client.execute(
|
||||
CompletionPersistentTaskAction.INSTANCE,
|
||||
new CompletionPersistentTaskAction.Request(
|
||||
MasterNodeRequest.INFINITE_MASTER_NODE_TIMEOUT,
|
||||
persistentTask.getId(),
|
||||
persistentTask.getAllocationId(),
|
||||
new IndexNotFoundException(followerIndex),
|
||||
|
|
|
@ -549,7 +549,7 @@ public class TransportDownsampleAction extends AcknowledgedTransportMasterNodeAc
|
|||
persistentTaskId,
|
||||
DownsampleShardTask.TASK_NAME,
|
||||
params,
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be configurable? longer by default? infinite? */,
|
||||
ActionListener.wrap(
|
||||
startedTask -> persistentTasksService.waitForPersistentTaskCondition(
|
||||
startedTask.getId(),
|
||||
|
|
|
@ -16,6 +16,7 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
|||
import org.elasticsearch.cluster.metadata.DataStream;
|
||||
import org.elasticsearch.cluster.metadata.Metadata;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.injection.guice.Inject;
|
||||
import org.elasticsearch.persistent.PersistentTasksService;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
|
@ -81,7 +82,7 @@ public class ReindexDataStreamTransportAction extends HandledTransportAction<Rei
|
|||
persistentTaskId,
|
||||
ReindexDataStreamTask.TASK_NAME,
|
||||
params,
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be configurable? longer by default? infinite? */,
|
||||
ActionListener.wrap(startedTask -> listener.onResponse(AcknowledgedResponse.TRUE), listener::onFailure)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -308,6 +308,7 @@ public class MlDistributedFailureIT extends BaseMlIntegTestCase {
|
|||
}
|
||||
|
||||
UpdatePersistentTaskStatusAction.Request updatePersistentTaskStatusRequest = new UpdatePersistentTaskStatusAction.Request(
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
task.getId(),
|
||||
task.getAllocationId(),
|
||||
DatafeedState.STOPPING
|
||||
|
|
|
@ -322,7 +322,7 @@ public class TransportPutRollupJobAction extends AcknowledgedTransportMasterNode
|
|||
job.getConfig().getId(),
|
||||
RollupField.TASK_NAME,
|
||||
job,
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be configurable? longer by default? infinite? */,
|
||||
ActionListener.wrap(rollupConfigPersistentTask -> waitForRollupStarted(job, listener, persistentTasksService), e -> {
|
||||
if (e instanceof ResourceAlreadyExistsException) {
|
||||
e = new ElasticsearchStatusException(
|
||||
|
|
|
@ -42,6 +42,7 @@ import static org.hamcrest.Matchers.containsString;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.isNotNull;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
@ -373,10 +374,10 @@ public class PutJobStateMachineTests extends ESTestCase {
|
|||
requestCaptor.getValue().onFailure(new ResourceAlreadyExistsException(job.getConfig().getRollupIndex()));
|
||||
return null;
|
||||
}).when(tasksService)
|
||||
.sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), eq(null), requestCaptor.capture());
|
||||
.sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), isNotNull(), requestCaptor.capture());
|
||||
|
||||
TransportPutRollupJobAction.startPersistentTask(job, testListener, tasksService);
|
||||
verify(tasksService).sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), eq(null), any());
|
||||
verify(tasksService).sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), isNotNull(), any());
|
||||
}
|
||||
|
||||
@SuppressWarnings({ "unchecked", "rawtypes" })
|
||||
|
@ -401,7 +402,7 @@ public class PutJobStateMachineTests extends ESTestCase {
|
|||
requestCaptor.getValue().onResponse(response);
|
||||
return null;
|
||||
}).when(tasksService)
|
||||
.sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), eq(null), requestCaptor.capture());
|
||||
.sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), isNotNull(), requestCaptor.capture());
|
||||
|
||||
ArgumentCaptor<PersistentTasksService.WaitForPersistentTaskListener> requestCaptor2 = ArgumentCaptor.forClass(
|
||||
PersistentTasksService.WaitForPersistentTaskListener.class
|
||||
|
@ -413,7 +414,7 @@ public class PutJobStateMachineTests extends ESTestCase {
|
|||
}).when(tasksService).waitForPersistentTaskCondition(eq(job.getConfig().getId()), any(), any(), requestCaptor2.capture());
|
||||
|
||||
TransportPutRollupJobAction.startPersistentTask(job, testListener, tasksService);
|
||||
verify(tasksService).sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), eq(null), any());
|
||||
verify(tasksService).sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), isNotNull(), any());
|
||||
verify(tasksService).waitForPersistentTaskCondition(eq(job.getConfig().getId()), any(), any(), any());
|
||||
}
|
||||
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.elasticsearch.common.util.set.Sets;
|
|||
import org.elasticsearch.core.IOUtils;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
import org.elasticsearch.core.Releasable;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.env.NodeMetadata;
|
||||
import org.elasticsearch.features.FeatureService;
|
||||
|
@ -1273,7 +1274,7 @@ public class Security extends Plugin
|
|||
SecurityMigrationTaskParams.TASK_NAME,
|
||||
SecurityMigrationTaskParams.TASK_NAME,
|
||||
new SecurityMigrationTaskParams(migrationsVersion, securityMigrationNeeded),
|
||||
null,
|
||||
TimeValue.THIRTY_SECONDS /* TODO should this be configurable? longer by default? infinite? */,
|
||||
ActionListener.wrap((response) -> {
|
||||
logger.debug("Security migration task submitted");
|
||||
}, (exception) -> {
|
||||
|
|
|
@ -183,12 +183,18 @@ public class NodeShutdownTasksIT extends ESIntegTestCase {
|
|||
|
||||
private void startTask() {
|
||||
logger.info("--> sending start request");
|
||||
persistentTasksService.sendStartRequest("task_id", "task_name", new TestTaskParams(), null, ActionListener.wrap(r -> {}, e -> {
|
||||
persistentTasksService.sendStartRequest(
|
||||
"task_id",
|
||||
"task_name",
|
||||
new TestTaskParams(),
|
||||
TEST_REQUEST_TIMEOUT,
|
||||
ActionListener.wrap(r -> {}, e -> {
|
||||
if (e instanceof ResourceAlreadyExistsException == false) {
|
||||
logger.error("failed to create task", e);
|
||||
fail("failed to create task");
|
||||
}
|
||||
}));
|
||||
})
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -266,11 +266,23 @@ public class TransportStopTransformActionTests extends ESTestCase {
|
|||
when(client.threadPool()).thenReturn(threadPool);
|
||||
|
||||
doAnswer(randomBoolean() ? withResponse() : withException(new ResourceNotFoundException("task not found"))).when(client)
|
||||
.execute(same(RemovePersistentTaskAction.INSTANCE), eq(new RemovePersistentTaskAction.Request("task-A")), any());
|
||||
.execute(
|
||||
same(RemovePersistentTaskAction.INSTANCE),
|
||||
eq(new RemovePersistentTaskAction.Request(TEST_REQUEST_TIMEOUT, "task-A")),
|
||||
any()
|
||||
);
|
||||
doAnswer(randomBoolean() ? withResponse() : withException(new ResourceNotFoundException("task not found"))).when(client)
|
||||
.execute(same(RemovePersistentTaskAction.INSTANCE), eq(new RemovePersistentTaskAction.Request("task-B")), any());
|
||||
.execute(
|
||||
same(RemovePersistentTaskAction.INSTANCE),
|
||||
eq(new RemovePersistentTaskAction.Request(TEST_REQUEST_TIMEOUT, "task-B")),
|
||||
any()
|
||||
);
|
||||
doAnswer(withException(new IllegalStateException("real issue while removing task"))).when(client)
|
||||
.execute(same(RemovePersistentTaskAction.INSTANCE), eq(new RemovePersistentTaskAction.Request("task-C")), any());
|
||||
.execute(
|
||||
same(RemovePersistentTaskAction.INSTANCE),
|
||||
eq(new RemovePersistentTaskAction.Request(TEST_REQUEST_TIMEOUT, "task-C")),
|
||||
any()
|
||||
);
|
||||
|
||||
PersistentTasksService persistentTasksService = new PersistentTasksService(mock(ClusterService.class), threadPool, client);
|
||||
Set<String> transformTasks = Set.of("task-A", "task-B", "task-C");
|
||||
|
|
|
@ -80,6 +80,7 @@ import static org.hamcrest.Matchers.sameInstance;
|
|||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyLong;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.isNotNull;
|
||||
import static org.mockito.ArgumentMatchers.isNull;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
|
@ -352,7 +353,7 @@ public class TransformTaskTests extends ESTestCase {
|
|||
eq(42L),
|
||||
isNull(),
|
||||
eq("Node is shutting down."),
|
||||
isNull(),
|
||||
isNotNull(),
|
||||
any()
|
||||
);
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue