diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/UpdateTimeSeriesRangeService.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/UpdateTimeSeriesRangeService.java index abd6ba93d3d16..457ca7ae5ce4c 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/UpdateTimeSeriesRangeService.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/UpdateTimeSeriesRangeService.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.LocalNodeMasterListener; @@ -18,6 +17,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; @@ -50,13 +50,14 @@ public class UpdateTimeSeriesRangeService extends AbstractLifecycleComponent imp volatile TimeValue pollInterval; volatile Scheduler.Cancellable job; private final AtomicBoolean running = new AtomicBoolean(false); - private final ClusterStateTaskExecutor taskExecutor = new UpdateTimeSeriesExecutor(); + private final MasterServiceTaskQueue taskQueue; UpdateTimeSeriesRangeService(Settings settings, ThreadPool threadPool, ClusterService clusterService) { this.pollInterval = DataStreamsPlugin.TIME_SERIES_POLL_INTERVAL.get(settings); this.threadPool = threadPool; this.clusterService = clusterService; clusterService.getClusterSettings().addSettingsUpdateConsumer(DataStreamsPlugin.TIME_SERIES_POLL_INTERVAL, this::setPollInterval); + this.taskQueue = clusterService.getTaskQueue("update-time-series-range", Priority.URGENT, new UpdateTimeSeriesExecutor()); } void perform(Runnable onComplete) { @@ -69,8 +70,7 @@ void perform(Runnable onComplete) { running.set(false); onComplete.run(); }); - var config = ClusterStateTaskConfig.build(Priority.URGENT); - clusterService.submitStateUpdateTask("update_tsdb_data_stream_end_times", task, config, taskExecutor); + taskQueue.submitTask("update_tsdb_data_stream_end_times", task, null); } else { LOGGER.debug("not starting tsdb update task, because another execution is still running"); } diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java index e583fc0146a23..42f05b650888e 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/threadpool/EvilThreadPoolTests.java @@ -98,8 +98,7 @@ public void testExecutionErrorOnSinglePrioritizingThreadPoolExecutor() throws In "test", EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - threadPool.scheduler(), - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + threadPool.scheduler() ); try { checkExecutionError(getExecuteRunner(prioritizedExecutor)); @@ -208,8 +207,7 @@ public void testExecutionExceptionOnSinglePrioritizingThreadPoolExecutor() throw "test", EsExecutors.daemonThreadFactory("test"), threadPool.getThreadContext(), - threadPool.scheduler(), - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + threadPool.scheduler() ); try { checkExecutionException(getExecuteRunner(prioritizedExecutor), true); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java index d0e151d506341..f9cd9649600fa 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/support/AutoCreateIndexIT.java @@ -10,7 +10,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.test.ESIntegTestCase; @@ -28,17 +27,12 @@ public class AutoCreateIndexIT extends ESIntegTestCase { public void testBatchingWithDeprecationWarnings() throws Exception { final var masterNodeClusterService = internalCluster().getCurrentMasterNodeInstance(ClusterService.class); final var barrier = new CyclicBarrier(2); - masterNodeClusterService.submitStateUpdateTask( - "block", - e -> { assert false : e; }, - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - barrier.await(10, TimeUnit.SECONDS); - barrier.await(10, TimeUnit.SECONDS); - batchExecutionContext.taskContexts().forEach(c -> c.success(() -> {})); - return batchExecutionContext.initialState(); - } - ); + masterNodeClusterService.getTaskQueue("block", Priority.NORMAL, batchExecutionContext -> { + barrier.await(10, TimeUnit.SECONDS); + barrier.await(10, TimeUnit.SECONDS); + batchExecutionContext.taskContexts().forEach(c -> c.success(() -> {})); + return batchExecutionContext.initialState(); + }).submitTask("block", e -> { assert false : e; }, null); barrier.await(10, TimeUnit.SECONDS); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/service/ClusterServiceIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/service/ClusterServiceIT.java index cd2f3ebf561d5..24950fe160e86 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/service/ClusterServiceIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/service/ClusterServiceIT.java @@ -17,6 +17,7 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; +import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; import java.util.HashSet; @@ -25,6 +26,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.StreamSupport; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -414,11 +416,7 @@ public void onFailure(Exception e) { }); } - final var startNanoTime = System.nanoTime(); - while (TimeUnit.MILLISECONDS.convert(System.nanoTime() - startNanoTime, TimeUnit.NANOSECONDS) <= 0) { - // noinspection BusyWait - Thread.sleep(100); - } + waitForTimeToElapse(); pendingClusterTasks = clusterService.getMasterService().pendingTasks(); assertThat(pendingClusterTasks.size(), greaterThanOrEqualTo(5)); @@ -441,4 +439,28 @@ public void onFailure(Exception e) { block2.countDown(); } } + + private static void waitForTimeToElapse() throws InterruptedException { + final ThreadPool[] threadPools = StreamSupport.stream(internalCluster().getInstances(ClusterService.class).spliterator(), false) + .map(ClusterService::threadPool) + .toArray(ThreadPool[]::new); + final long[] startTimes = Arrays.stream(threadPools).mapToLong(ThreadPool::relativeTimeInMillis).toArray(); + + final var startNanoTime = System.nanoTime(); + while (TimeUnit.MILLISECONDS.convert(System.nanoTime() - startNanoTime, TimeUnit.NANOSECONDS) <= 100) { + // noinspection BusyWait + Thread.sleep(100); + } + + outer: do { + for (int i = 0; i < threadPools.length; i++) { + if (threadPools[i].relativeTimeInMillis() <= startTimes[i]) { + // noinspection BusyWait + Thread.sleep(100); + continue outer; + } + } + return; + } while (true); + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportDeleteDesiredNodesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportDeleteDesiredNodesAction.java index 86e0d9893ee27..aa0d31499177f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportDeleteDesiredNodesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportDeleteDesiredNodesAction.java @@ -13,8 +13,6 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.SimpleBatchedExecutor; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -22,6 +20,7 @@ import org.elasticsearch.cluster.metadata.DesiredNodesMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.core.Tuple; @@ -31,7 +30,7 @@ public class TransportDeleteDesiredNodesAction extends TransportMasterNodeAction { - private final ClusterStateTaskExecutor taskExecutor = new DeleteDesiredNodesExecutor(); + private final MasterServiceTaskQueue taskQueue; @Inject public TransportDeleteDesiredNodesAction( @@ -52,6 +51,7 @@ public TransportDeleteDesiredNodesAction( in -> ActionResponse.Empty.INSTANCE, ThreadPool.Names.SAME ); + this.taskQueue = clusterService.getTaskQueue("delete-desired-nodes", Priority.HIGH, new DeleteDesiredNodesExecutor()); } @Override @@ -61,12 +61,7 @@ protected void masterOperation( ClusterState state, ActionListener listener ) throws Exception { - clusterService.submitStateUpdateTask( - "delete-desired-nodes", - new DeleteDesiredNodesTask(listener), - ClusterStateTaskConfig.build(Priority.HIGH, request.masterNodeTimeout()), - taskExecutor - ); + taskQueue.submitTask("delete-desired-nodes", new DeleteDesiredNodesTask(listener), request.masterNodeTimeout()); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportUpdateDesiredNodesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportUpdateDesiredNodesAction.java index b0d7000afa8ac..0d292069b0579 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportUpdateDesiredNodesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/desirednodes/TransportUpdateDesiredNodesAction.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -27,6 +26,7 @@ import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.tasks.Task; @@ -41,7 +41,7 @@ public class TransportUpdateDesiredNodesAction extends TransportMasterNodeAction private static final Logger logger = LogManager.getLogger(TransportUpdateDesiredNodesAction.class); private final DesiredNodesSettingsValidator settingsValidator; - private final ClusterStateTaskExecutor taskExecutor; + private final MasterServiceTaskQueue taskQueue; @Inject public TransportUpdateDesiredNodesAction( @@ -66,7 +66,11 @@ public TransportUpdateDesiredNodesAction( ThreadPool.Names.SAME ); this.settingsValidator = settingsValidator; - this.taskExecutor = new UpdateDesiredNodesExecutor(clusterService.getRerouteService(), allocationService); + this.taskQueue = clusterService.getTaskQueue( + "update-desired-nodes", + Priority.URGENT, + new UpdateDesiredNodesExecutor(clusterService.getRerouteService(), allocationService) + ); } @Override @@ -83,12 +87,7 @@ protected void masterOperation( ) throws Exception { try { settingsValidator.validate(request.getNodes()); - clusterService.submitStateUpdateTask( - "update-desired-nodes", - new UpdateDesiredNodesTask(request, listener), - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - taskExecutor - ); + taskQueue.submitTask("update-desired-nodes", new UpdateDesiredNodesTask(request, listener), request.masterNodeTimeout()); } catch (Exception e) { listener.onFailure(e); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java index ead1641cb2595..dd682481d7fb4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.LocalMasterServiceTask; import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException; @@ -31,6 +32,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexNotFoundException; @@ -163,13 +165,20 @@ public void onFailure(Exception e) { if (e instanceof ProcessClusterEventTimeoutException) { listener.onResponse(getResponse(request, clusterService.state(), waitCount, TimeoutState.TIMED_OUT)); } else { - final Level level = e instanceof NotMasterException ? Level.TRACE : Level.ERROR; - assert e instanceof NotMasterException : e; // task cannot fail, nor will it trigger a publication which fails + final Level level = isExpectedFailure(e) ? Level.TRACE : Level.ERROR; logger.log(level, () -> "unexpected failure during [" + source + "]", e); + assert isExpectedFailure(e) : e; // task cannot fail, nor will it trigger a publication which fails // TransportMasterNodeAction implements the retry logic, which is triggered by passing a NotMasterException listener.onFailure(e); } } + + static boolean isExpectedFailure(Exception e) { + return e instanceof NotMasterException + || e instanceof FailedToCommitClusterStateException + && e.getCause()instanceof EsRejectedExecutionException esre + && esre.isExecutorShutdown(); + } }); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java index 88d6ddca28af4..7063303db41cb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java @@ -20,7 +20,6 @@ import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -36,6 +35,7 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.AllocationActionMultiListener; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; @@ -77,7 +77,7 @@ public static final class TransportAction extends TransportMasterNodeAction executor; + private final MasterServiceTaskQueue taskQueue; @Inject public TransportAction( @@ -107,7 +107,7 @@ public TransportAction( this.createIndexService = createIndexService; this.metadataCreateDataStreamService = metadataCreateDataStreamService; this.autoCreateIndex = autoCreateIndex; - this.executor = batchExecutionContext -> { + this.taskQueue = clusterService.getTaskQueue("auto-create", Priority.URGENT, batchExecutionContext -> { final var listener = new AllocationActionMultiListener(threadPool.getThreadContext()); final var taskContexts = batchExecutionContext.taskContexts(); final var successfulRequests = Maps.newMapWithExpectedSize(taskContexts.size()); @@ -129,7 +129,7 @@ public TransportAction( listener.noRerouteNeeded(); } return state; - }; + }); } @Override @@ -139,11 +139,10 @@ protected void masterOperation( ClusterState state, ActionListener listener ) { - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "auto create [" + request.index() + "]", new CreateIndexTask(request, listener), - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - executor + request.masterNodeTimeout() ); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java index c53f34a8a8b69..50e0aacecc1df 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java @@ -22,7 +22,6 @@ import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -35,6 +34,7 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.AllocationActionMultiListener; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; @@ -64,7 +64,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction rolloverTaskQueue; @Inject public TransportRolloverAction( @@ -89,7 +89,11 @@ public TransportRolloverAction( ThreadPool.Names.SAME ); this.client = client; - this.rolloverTaskExecutor = new RolloverExecutor(clusterService, allocationService, rolloverService, threadPool); + this.rolloverTaskQueue = clusterService.getTaskQueue( + "rollover", + Priority.NORMAL, + new RolloverExecutor(clusterService, allocationService, rolloverService, threadPool) + ); } @Override @@ -173,8 +177,7 @@ protected void masterOperation( if (rolloverRequest.areConditionsMet(trialConditionResults)) { String source = "rollover_index source [" + trialRolloverIndexName + "] to target [" + trialRolloverIndexName + "]"; RolloverTask rolloverTask = new RolloverTask(rolloverRequest, statsResponse, trialRolloverResponse, listener); - ClusterStateTaskConfig config = ClusterStateTaskConfig.build(Priority.NORMAL, rolloverRequest.masterNodeTimeout()); - clusterService.submitStateUpdateTask(source, rolloverTask, config, rolloverTaskExecutor); + rolloverTaskQueue.submitTask(source, rolloverTask, rolloverRequest.masterNodeTimeout()); } else { // conditions not met listener.onResponse(trialRolloverResponse); diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index 3ece7e345ffd6..774da137ec76b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -70,8 +70,8 @@ * across master elections (and therefore is preserved in a rolling restart). *

* Updates are triggered by submitting tasks to the {@link MasterService} on the elected master, typically using a {@link - * TransportMasterNodeAction} to route a request to the master on which the task is submitted with {@link - * ClusterService#submitStateUpdateTask}. Submitted tasks have an associated {@link ClusterStateTaskConfig} which defines a priority and a + * TransportMasterNodeAction} to route a request to the master on which the task is submitted via a queue obtained with {@link + * ClusterService#getTaskQueue}, which has an associated priority. Submitted tasks have an associated * timeout. Tasks are processed in priority order, so a flood of higher-priority tasks can starve lower-priority ones from running. * Therefore, avoid priorities other than {@link Priority#NORMAL} where possible. Tasks associated with client actions should typically have * a timeout, or otherwise be sensitive to client cancellations, to avoid surprises caused by the execution of stale tasks long after they diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterStateTaskConfig.java b/server/src/main/java/org/elasticsearch/cluster/ClusterStateTaskConfig.java deleted file mode 100644 index c31172620767c..0000000000000 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterStateTaskConfig.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ -package org.elasticsearch.cluster; - -import org.elasticsearch.common.Priority; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.core.TimeValue; - -/** - * Cluster state update task configuration for timeout and priority - */ -public interface ClusterStateTaskConfig { - /** - * The timeout for this cluster state update task configuration. If the cluster state update task isn't processed within this timeout, - * the associated {@link ClusterStateTaskListener#onFailure(Exception)} is invoked. Tasks arising from client requests should - * have a timeout which clients can adjust via the {@code ?master_timeout} query parameter, and typically defaults to {@code 30s}. In - * contrast, internal tasks can reasonably have an infinite timeout, especially if a timeout would simply trigger a retry. - * - * @return the timeout, or null if one is not set - */ - @Nullable - TimeValue timeout(); - - /** - * The {@link Priority} for this cluster state update task configuration. Avoid priorities other than {@link Priority#NORMAL} where - * possible. A stream of higher-priority tasks can starve lower-priority ones from running. Higher-priority tasks should definitely - * share a {@link ClusterStateTaskExecutor} instance so that they are executed in batches. - * - * @return the priority - */ - Priority priority(); - - /** - * Build a cluster state update task configuration with the specified {@link Priority} and no timeout. - * - * @param priority the priority for the associated cluster state update task - * @return the resulting cluster state update task configuration - */ - static ClusterStateTaskConfig build(Priority priority) { - return new Basic(priority, null); - } - - /** - * Build a cluster state update task configuration with the specified {@link Priority} and timeout. - * - * @param priority the priority for the associated cluster state update task - * @param timeout the timeout for the associated cluster state update task - * @return the result cluster state update task configuration - */ - static ClusterStateTaskConfig build(Priority priority, TimeValue timeout) { - return new Basic(priority, timeout); - } - - record Basic(Priority priority, TimeValue timeout) implements ClusterStateTaskConfig {} -} diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterStateUpdateTask.java b/server/src/main/java/org/elasticsearch/cluster/ClusterStateUpdateTask.java index ac31d87ee67ca..4f038132e7dfa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterStateUpdateTask.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterStateUpdateTask.java @@ -15,7 +15,7 @@ /** * A task that can update the cluster state. */ -public abstract class ClusterStateUpdateTask implements ClusterStateTaskConfig, ClusterStateTaskListener { +public abstract class ClusterStateUpdateTask implements ClusterStateTaskListener { private final Priority priority; @@ -66,7 +66,6 @@ public final TimeValue timeout() { return timeout; } - @Override public final Priority priority() { return priority; } diff --git a/server/src/main/java/org/elasticsearch/cluster/LocalMasterServiceTask.java b/server/src/main/java/org/elasticsearch/cluster/LocalMasterServiceTask.java index 5b1d0ff22e201..21c31007540eb 100644 --- a/server/src/main/java/org/elasticsearch/cluster/LocalMasterServiceTask.java +++ b/server/src/main/java/org/elasticsearch/cluster/LocalMasterServiceTask.java @@ -28,37 +28,32 @@ protected void execute(ClusterState currentState) {} protected void onPublicationComplete() {} public void submit(MasterService masterService, String source) { - masterService.submitStateUpdateTask( - source, - this, - ClusterStateTaskConfig.build(priority), - // Uses a new executor each time so that these tasks are not batched, but they never change the cluster state anyway so they - // don't trigger the publication process and hence batching isn't really needed. - new ClusterStateTaskExecutor<>() { + // Uses a new queue each time so that these tasks are not batched, but they never change the cluster state anyway so they don't + // trigger the publication process and hence batching isn't really needed. + masterService.getTaskQueue("local-master-service-task", priority, new ClusterStateTaskExecutor() { - @Override - public boolean runOnlyOnMaster() { - return false; - } + @Override + public boolean runOnlyOnMaster() { + return false; + } - @Override - public String describeTasks(List tasks) { - return ""; // only one task in the batch so the source is enough - } + @Override + public String describeTasks(List tasks) { + return ""; // only one task in the batch so the source is enough + } - @Override - public ClusterState execute(BatchExecutionContext batchExecutionContext) { - final var thisTask = LocalMasterServiceTask.this; - final var taskContexts = batchExecutionContext.taskContexts(); - assert taskContexts.size() == 1 && taskContexts.get(0).getTask() == thisTask - : "expected one-element task list containing current object but was " + taskContexts; - try (var ignored = taskContexts.get(0).captureResponseHeaders()) { - thisTask.execute(batchExecutionContext.initialState()); - } - taskContexts.get(0).success(() -> onPublicationComplete()); - return batchExecutionContext.initialState(); + @Override + public ClusterState execute(BatchExecutionContext batchExecutionContext) { + final var thisTask = LocalMasterServiceTask.this; + final var taskContexts = batchExecutionContext.taskContexts(); + assert taskContexts.size() == 1 && taskContexts.get(0).getTask() == thisTask + : "expected one-element task list containing current object but was " + taskContexts; + try (var ignored = taskContexts.get(0).captureResponseHeaders()) { + thisTask.execute(batchExecutionContext.initialState()); } + taskContexts.get(0).success(() -> onPublicationComplete()); + return batchExecutionContext.initialState(); } - ); + }).submitTask(source, this, null); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index 5e7e3aef43c15..1bae8e34a5caa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -17,7 +17,6 @@ import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.NotMasterException; @@ -32,6 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.StaleShard; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; @@ -276,15 +276,13 @@ public void onTimeout(TimeValue timeout) { // TODO: Make this a TransportMasterNodeAction and remove duplication of master failover retrying from upstream code private static class ShardFailedTransportHandler implements TransportRequestHandler { - private final ClusterService clusterService; - private final ShardFailedClusterStateTaskExecutor shardFailedClusterStateTaskExecutor; + private final MasterServiceTaskQueue taskQueue; ShardFailedTransportHandler( ClusterService clusterService, ShardFailedClusterStateTaskExecutor shardFailedClusterStateTaskExecutor ) { - this.clusterService = clusterService; - this.shardFailedClusterStateTaskExecutor = shardFailedClusterStateTaskExecutor; + taskQueue = clusterService.getTaskQueue(TASK_SOURCE, Priority.HIGH, shardFailedClusterStateTaskExecutor); } private static final String TASK_SOURCE = "shard-failed"; @@ -293,12 +291,7 @@ private static class ShardFailedTransportHandler implements TransportRequestHand public void messageReceived(FailedShardEntry request, TransportChannel channel, Task task) throws Exception { logger.debug(() -> format("%s received shard failed for [%s]", request.getShardId(), request), request.failure); var update = new FailedShardUpdateTask(request, new ChannelActionListener<>(channel, TASK_SOURCE, request)); - clusterService.submitStateUpdateTask( - TASK_SOURCE, - update, - ClusterStateTaskConfig.build(Priority.HIGH), - shardFailedClusterStateTaskExecutor - ); + taskQueue.submitTask(TASK_SOURCE, update, null); } } @@ -572,15 +565,13 @@ public void shardStarted( // TODO: Make this a TransportMasterNodeAction and remove duplication of master failover retrying from upstream code private static class ShardStartedTransportHandler implements TransportRequestHandler { - private final ClusterService clusterService; - private final ShardStartedClusterStateTaskExecutor shardStartedClusterStateTaskExecutor; + private final MasterServiceTaskQueue taskQueue; ShardStartedTransportHandler( ClusterService clusterService, ShardStartedClusterStateTaskExecutor shardStartedClusterStateTaskExecutor ) { - this.clusterService = clusterService; - this.shardStartedClusterStateTaskExecutor = shardStartedClusterStateTaskExecutor; + taskQueue = clusterService.getTaskQueue("shard-started", Priority.URGENT, shardStartedClusterStateTaskExecutor); } @Override @@ -593,13 +584,7 @@ public void messageReceived(StartedShardEntry request, TransportChannel channel, ); var update = new StartedShardUpdateTask(request, listener); - - clusterService.submitStateUpdateTask( - "shard-started " + request, - update, - ClusterStateTaskConfig.build(Priority.URGENT), - shardStartedClusterStateTaskExecutor - ); + taskQueue.submitTask("shard-started " + request, update, null); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 33e3528246d12..6282ff7a3f9ac 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -19,7 +19,6 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStatePublicationEvent; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.LocalMasterServiceTask; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -37,6 +36,7 @@ import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractLifecycleComponent; @@ -137,7 +137,7 @@ public class Coordinator extends AbstractLifecycleComponent implements ClusterSt private final AllocationService allocationService; private final JoinHelper joinHelper; private final JoinValidationService joinValidationService; - private final NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor; + private final MasterServiceTaskQueue nodeRemovalQueue; private final Supplier persistedStateSupplier; private final NoMasterBlockService noMasterBlockService; final Object mutex = new Object(); // package-private to allow tests to call methods that assert that the mutex is held @@ -272,7 +272,11 @@ public Coordinator( this::removeNode, nodeHealthService ); - this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService); + this.nodeRemovalQueue = masterService.getTaskQueue( + "node-left", + Priority.IMMEDIATE, + new NodeRemovalClusterStateTaskExecutor(allocationService) + ); this.clusterApplier = clusterApplier; masterService.setClusterStateSupplier(this::getStateForMasterService); this.reconfigurator = new Reconfigurator(settings, clusterSettings); @@ -344,12 +348,7 @@ private void removeNode(DiscoveryNode discoveryNode, String reason) { reason, () -> joinReasonService.onNodeRemoved(discoveryNode, reason) ); - masterService.submitStateUpdateTask( - "node-left", - task, - ClusterStateTaskConfig.build(Priority.IMMEDIATE), - nodeRemovalExecutor - ); + nodeRemovalQueue.submitTask("node-left", task, null); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 48d6ba553cbbe..a42e6edd4c3fe 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -13,7 +13,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ChannelActionListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.coordination.Coordinator.Mode; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RerouteService; @@ -21,6 +20,7 @@ import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; @@ -64,10 +64,9 @@ public class JoinHelper { public static final String JOIN_ACTION_NAME = "internal:cluster/coordination/join"; public static final String JOIN_PING_ACTION_NAME = "internal:cluster/coordination/join/ping"; - private final MasterService masterService; private final ClusterApplier clusterApplier; private final TransportService transportService; - private final JoinTaskExecutor joinTaskExecutor; + private final MasterServiceTaskQueue joinTaskQueue; private final LongSupplier currentTermSupplier; private final NodeHealthService nodeHealthService; private final JoinReasonService joinReasonService; @@ -90,11 +89,14 @@ public class JoinHelper { JoinReasonService joinReasonService, CircuitBreakerService circuitBreakerService ) { - this.masterService = masterService; + this.joinTaskQueue = masterService.getTaskQueue( + "node-join", + Priority.URGENT, + new JoinTaskExecutor(allocationService, rerouteService) + ); this.clusterApplier = clusterApplier; this.transportService = transportService; this.circuitBreakerService = circuitBreakerService; - this.joinTaskExecutor = new JoinTaskExecutor(allocationService, rerouteService); this.currentTermSupplier = currentTermSupplier; this.nodeHealthService = nodeHealthService; this.joinReasonService = joinReasonService; @@ -395,7 +397,7 @@ public void handleJoinRequest(DiscoveryNode sender, ActionListener joinLis joinListener, currentTermSupplier.getAsLong() ); - masterService.submitStateUpdateTask("node-join", task, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); + joinTaskQueue.submitTask("node-join", task, null); } @Override @@ -457,13 +459,7 @@ public void close(Mode newMode) { listener ); }), currentTermSupplier.getAsLong()); - masterService.submitStateUpdateTask( - "elected-as-master ([" + joinTask.nodeCount() + "] nodes joined)", - joinTask, - ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor - - ); + joinTaskQueue.submitTask("elected-as-master ([" + joinTask.nodeCount() + "] nodes joined)", joinTask, null); } else { assert newMode == Mode.FOLLOWER : newMode; joinRequestAccumulator.values() diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 773ebff7d1622..6fa069dba14c3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.admin.indices.delete.DeleteIndexClusterStateUpdateRequest; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.SimpleBatchedAckListenerTaskExecutor; @@ -21,6 +20,7 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; @@ -48,15 +48,14 @@ public class MetadataDeleteIndexService { private static final Logger logger = LogManager.getLogger(MetadataDeleteIndexService.class); private final Settings settings; - private final ClusterService clusterService; // package private for tests final ClusterStateTaskExecutor executor; + private final MasterServiceTaskQueue taskQueue; @Inject public MetadataDeleteIndexService(Settings settings, ClusterService clusterService, AllocationService allocationService) { this.settings = settings; - this.clusterService = clusterService; executor = new SimpleBatchedAckListenerTaskExecutor<>() { @Override public Tuple executeTask( @@ -78,15 +77,14 @@ public ClusterState afterBatchExecution(ClusterState clusterState, boolean clust return clusterState; } }; + taskQueue = clusterService.getTaskQueue("delete-index", Priority.URGENT, executor); } - private static final ClusterStateTaskConfig URGENT_CONFIG = ClusterStateTaskConfig.build(Priority.URGENT); - public void deleteIndices(final DeleteIndexClusterStateUpdateRequest request) { if (request.indices() == null || request.indices().length == 0) { throw new IllegalArgumentException("Index name is required"); } - clusterService.submitStateUpdateTask("delete-index " + Arrays.toString(request.indices()), request, URGENT_CONFIG, executor); + taskQueue.submitTask("delete-index " + Arrays.toString(request.indices()), request, request.masterNodeTimeout()); } /** diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexStateService.java index 592654ac8c125..2da3784b9c2cf 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexStateService.java @@ -33,7 +33,6 @@ import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.SimpleBatchedExecutor; @@ -48,6 +47,7 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.AllocationActionMultiListener; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; @@ -120,11 +120,11 @@ public class MetadataIndexStateService { private final ShardLimitValidator shardLimitValidator; private final NodeClient client; private final ThreadPool threadPool; - private final ClusterStateTaskExecutor opensExecutor; - private final ClusterStateTaskExecutor addBlocksToCloseExecutor; - private final ClusterStateTaskExecutor closesExecutor; - private final ClusterStateTaskExecutor addBlocksExecutor; - private final ClusterStateTaskExecutor finalizeBlocksExecutor; + private final MasterServiceTaskQueue opensQueue; + private final MasterServiceTaskQueue addBlocksToCloseQueue; + private final MasterServiceTaskQueue closesQueue; + private final MasterServiceTaskQueue addBlocksQueue; + private final MasterServiceTaskQueue finalizeBlocksQueue; @Inject public MetadataIndexStateService( @@ -143,11 +143,12 @@ public MetadataIndexStateService( this.shardLimitValidator = shardLimitValidator; this.client = client; this.threadPool = threadPool; - this.opensExecutor = new OpenIndicesExecutor(); - this.addBlocksToCloseExecutor = new AddBlocksToCloseExecutor(); - this.closesExecutor = new CloseIndicesExecutor(); - this.addBlocksExecutor = new AddBlocksExecutor(); - this.finalizeBlocksExecutor = new FinalizeBlocksExecutor(); + + opensQueue = clusterService.getTaskQueue("open-index", Priority.URGENT, new OpenIndicesExecutor()); + addBlocksToCloseQueue = clusterService.getTaskQueue("add-blocks-to-close", Priority.URGENT, new AddBlocksToCloseExecutor()); + closesQueue = clusterService.getTaskQueue("close-index", Priority.URGENT, new CloseIndicesExecutor()); + addBlocksQueue = clusterService.getTaskQueue("add-blocks", Priority.URGENT, new AddBlocksExecutor()); + finalizeBlocksQueue = clusterService.getTaskQueue("finalize-blocks", Priority.URGENT, new FinalizeBlocksExecutor()); } /** @@ -161,11 +162,10 @@ public void closeIndices(final CloseIndexClusterStateUpdateRequest request, fina throw new IllegalArgumentException("Index name is required"); } - clusterService.submitStateUpdateTask( + addBlocksToCloseQueue.submitTask( "add-block-index-to-close " + Arrays.toString(request.indices()), new AddBlocksToCloseTask(request, listener), - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - this.addBlocksToCloseExecutor + request.masterNodeTimeout() ); } @@ -189,14 +189,14 @@ public void taskSucceeded(AddBlocksToCloseTask task, Map bl new WaitForClosedBlocksApplied( blockedIndices, task.request, - task.listener().delegateFailure((delegate2, verifyResults) -> { - clusterService.submitStateUpdateTask( - "close-indices", - new CloseIndicesTask(task.request, blockedIndices, verifyResults, delegate2), - ClusterStateTaskConfig.build(Priority.URGENT), - closesExecutor - ); - }) + task.listener() + .delegateFailure( + (delegate2, verifyResults) -> closesQueue.submitTask( + "close-indices", + new CloseIndicesTask(task.request, blockedIndices, verifyResults, delegate2), + null + ) + ) ) ); } @@ -467,11 +467,10 @@ public void addIndexBlock(AddIndexBlockClusterStateUpdateRequest request, Action ); } - clusterService.submitStateUpdateTask( + addBlocksQueue.submitTask( "add-index-block-[" + request.getBlock().name + "]-" + Arrays.toString(concreteIndices), new AddBlocksTask(request, listener), - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - addBlocksExecutor + request.masterNodeTimeout() ); } @@ -492,18 +491,18 @@ public void taskSucceeded(AddBlocksTask task, Map blockedIn new WaitForBlocksApplied( blockedIndices, task.request, - task.listener().delegateFailure((delegate2, verifyResults) -> { - clusterService.submitStateUpdateTask( - "finalize-index-block-[" - + task.request.getBlock().name - + "]-[" - + blockedIndices.keySet().stream().map(Index::getName).collect(Collectors.joining(", ")) - + "]", - new FinalizeBlocksTask(task.request, blockedIndices, verifyResults, delegate2), - ClusterStateTaskConfig.build(Priority.URGENT), - finalizeBlocksExecutor - ); - }) + task.listener() + .delegateFailure( + (delegate2, verifyResults) -> finalizeBlocksQueue.submitTask( + "finalize-index-block-[" + + task.request.getBlock().name + + "]-[" + + blockedIndices.keySet().stream().map(Index::getName).collect(Collectors.joining(", ")) + + "]", + new FinalizeBlocksTask(task.request, blockedIndices, verifyResults, delegate2), + null + ) + ) ) ); } @@ -935,11 +934,10 @@ private void onlyOpenIndices(final OpenIndexClusterStateUpdateRequest request, f throw new IllegalArgumentException("Index name is required"); } - clusterService.submitStateUpdateTask( + opensQueue.submitTask( "open-indices " + Arrays.toString(request.indices()), new OpenIndicesTask(request, listener), - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - this.opensExecutor + request.masterNodeTimeout() ); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java index a4f3f9e5573a1..99b500ca11315 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java @@ -19,10 +19,10 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.SimpleBatchedExecutor; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; @@ -115,7 +115,9 @@ public class MetadataIndexTemplateService { } private static final Logger logger = LogManager.getLogger(MetadataIndexTemplateService.class); + private final ClusterService clusterService; + private final MasterServiceTaskQueue taskQueue; private final IndicesService indicesService; private final MetadataCreateIndexService metadataCreateIndexService; private final IndexScopedSettings indexScopedSettings; @@ -169,6 +171,7 @@ public MetadataIndexTemplateService( IndexSettingProviders indexSettingProviders ) { this.clusterService = clusterService; + this.taskQueue = clusterService.getTaskQueue("index-templates", Priority.URGENT, TEMPLATE_TASK_EXECUTOR); this.indicesService = indicesService; this.metadataCreateIndexService = metadataCreateIndexService; this.indexScopedSettings = indexScopedSettings; @@ -178,7 +181,7 @@ public MetadataIndexTemplateService( } public void removeTemplates(final RemoveRequest request, final ActionListener listener) { - clusterService.submitStateUpdateTask("remove-index-template [" + request.name + "]", new TemplateClusterStateUpdateTask(listener) { + taskQueue.submitTask("remove-index-template [" + request.name + "]", new TemplateClusterStateUpdateTask(listener) { @Override public ClusterState execute(ClusterState currentState) { Set templateNames = new HashSet<>(); @@ -203,7 +206,7 @@ public ClusterState execute(ClusterState currentState) { } return ClusterState.builder(currentState).metadata(metadata).build(); } - }, ClusterStateTaskConfig.build(Priority.URGENT, request.masterTimeout), TEMPLATE_TASK_EXECUTOR); + }, request.masterTimeout); } /** @@ -218,7 +221,7 @@ public void putComponentTemplate( final ComponentTemplate template, final ActionListener listener ) { - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "create-component-template [" + name + "], cause [" + cause + "]", new TemplateClusterStateUpdateTask(listener) { @Override @@ -226,8 +229,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { return addComponentTemplate(currentState, create, name, template); } }, - ClusterStateTaskConfig.build(Priority.URGENT, masterTimeout), - TEMPLATE_TASK_EXECUTOR + masterTimeout ); } @@ -373,17 +375,12 @@ public void removeComponentTemplate( final ActionListener listener ) { validateNotInUse(state.metadata(), names); - clusterService.submitStateUpdateTask( - "remove-component-template [" + String.join(",", names) + "]", - new TemplateClusterStateUpdateTask(listener) { - @Override - public ClusterState execute(ClusterState currentState) { - return innerRemoveComponentTemplate(currentState, names); - } - }, - ClusterStateTaskConfig.build(Priority.URGENT, masterTimeout), - TEMPLATE_TASK_EXECUTOR - ); + taskQueue.submitTask("remove-component-template [" + String.join(",", names) + "]", new TemplateClusterStateUpdateTask(listener) { + @Override + public ClusterState execute(ClusterState currentState) { + return innerRemoveComponentTemplate(currentState, names); + } + }, masterTimeout); } // Exposed for ReservedComponentTemplateAction @@ -481,7 +478,7 @@ public void putIndexTemplateV2( final ActionListener listener ) { validateV2TemplateRequest(clusterService.state().metadata(), name, template); - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "create-index-template-v2 [" + name + "], cause [" + cause + "]", new TemplateClusterStateUpdateTask(listener) { @Override @@ -489,8 +486,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { return addIndexTemplateV2(currentState, create, name, template); } }, - ClusterStateTaskConfig.build(Priority.URGENT, masterTimeout), - TEMPLATE_TASK_EXECUTOR + masterTimeout ); } @@ -900,17 +896,12 @@ public void removeIndexTemplateV2( final TimeValue masterTimeout, final ActionListener listener ) { - clusterService.submitStateUpdateTask( - "remove-index-template-v2 [" + String.join(",", names) + "]", - new TemplateClusterStateUpdateTask(listener) { - @Override - public ClusterState execute(ClusterState currentState) { - return innerRemoveIndexTemplateV2(currentState, names); - } - }, - ClusterStateTaskConfig.build(Priority.URGENT, masterTimeout), - TEMPLATE_TASK_EXECUTOR - ); + taskQueue.submitTask("remove-index-template-v2 [" + String.join(",", names) + "]", new TemplateClusterStateUpdateTask(listener) { + @Override + public ClusterState execute(ClusterState currentState) { + return innerRemoveIndexTemplateV2(currentState, names); + } + }, masterTimeout); } // Public because it's used by ReservedComposableIndexTemplateAction @@ -1024,7 +1015,7 @@ public void putTemplate(final PutRequest request, final ActionListener taskQueue; @Inject public MetadataMappingService(ClusterService clusterService, IndicesService indicesService) { this.clusterService = clusterService; this.indicesService = indicesService; + taskQueue = clusterService.getTaskQueue("put-mapping", Priority.HIGH, new PutMappingExecutor()); } record PutMappingClusterStateUpdateTask(PutMappingClusterStateUpdateRequest request, ActionListener listener) @@ -246,12 +247,10 @@ public void putMapping(final PutMappingClusterStateUpdateRequest request, final return; } - final PutMappingClusterStateUpdateTask task = new PutMappingClusterStateUpdateTask(request, listener); - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "put-mapping " + Strings.arrayToCommaDelimitedString(request.indices()), - task, - ClusterStateTaskConfig.build(Priority.HIGH, request.masterNodeTimeout()), - putMappingExecutor + new PutMappingClusterStateUpdateTask(request, listener), + request.masterNodeTimeout() ); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataUpdateSettingsService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataUpdateSettingsService.java index 0c138087e6e52..6a9dd94d03b2d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataUpdateSettingsService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataUpdateSettingsService.java @@ -16,8 +16,6 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -26,6 +24,7 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.AllocationActionMultiListener; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.IndexScopedSettings; @@ -53,12 +52,10 @@ public class MetadataUpdateSettingsService { private static final Logger logger = LogManager.getLogger(MetadataUpdateSettingsService.class); - private final ClusterService clusterService; - private final AllocationService allocationService; private final IndexScopedSettings indexScopedSettings; private final IndicesService indicesService; private final ShardLimitValidator shardLimitValidator; - private final ClusterStateTaskExecutor executor; + private final MasterServiceTaskQueue taskQueue; public MetadataUpdateSettingsService( ClusterService clusterService, @@ -68,12 +65,10 @@ public MetadataUpdateSettingsService( ShardLimitValidator shardLimitValidator, ThreadPool threadPool ) { - this.clusterService = clusterService; - this.allocationService = allocationService; this.indexScopedSettings = indexScopedSettings; this.indicesService = indicesService; this.shardLimitValidator = shardLimitValidator; - this.executor = batchExecutionContext -> { + this.taskQueue = clusterService.getTaskQueue("update-settings", Priority.URGENT, batchExecutionContext -> { var listener = new AllocationActionMultiListener(threadPool.getThreadContext()); var state = batchExecutionContext.initialState(); for (final var taskContext : batchExecutionContext.taskContexts()) { @@ -97,7 +92,7 @@ public MetadataUpdateSettingsService( listener.noRerouteNeeded(); } return state; - }; + }); } private final class UpdateSettingsTask implements ClusterStateTaskListener { @@ -305,11 +300,10 @@ ClusterState execute(ClusterState currentState) { } public void updateSettings(final UpdateSettingsClusterStateUpdateRequest request, final ActionListener listener) { - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "update-settings " + Arrays.toString(request.indices()), new UpdateSettingsTask(request, listener), - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - this.executor + request.masterNodeTimeout() ); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java index 1e1495e0aadaa..c9144cf4795c9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java @@ -12,7 +12,6 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.routing.RoutingNode; @@ -26,8 +25,10 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.metrics.CounterMetric; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.threadpool.ThreadPool; @@ -53,13 +54,13 @@ public class DesiredBalanceShardsAllocator implements ShardsAllocator { private final ShardsAllocator delegateAllocator; private final ThreadPool threadPool; - private final ClusterService clusterService; private final DesiredBalanceReconcilerAction reconciler; private final DesiredBalanceComputer desiredBalanceComputer; private final ContinuousComputation desiredBalanceComputation; private final PendingListenersQueue queue; private final AtomicLong indexGenerator = new AtomicLong(-1); private final ConcurrentLinkedQueue> pendingDesiredBalanceMoves = new ConcurrentLinkedQueue<>(); + private final MasterServiceTaskQueue masterServiceTaskQueue; private final ReconcileDesiredBalanceExecutor executor = new ReconcileDesiredBalanceExecutor(); private final NodeAllocationOrdering allocationOrdering = new NodeAllocationOrdering(); private volatile DesiredBalance currentDesiredBalance = DesiredBalance.INITIAL; @@ -94,7 +95,6 @@ public DesiredBalanceShardsAllocator( ) { this.delegateAllocator = delegateAllocator; this.threadPool = threadPool; - this.clusterService = clusterService; this.reconciler = reconciler; this.desiredBalanceComputer = desiredBalanceComputer; this.desiredBalanceComputation = new ContinuousComputation<>(threadPool) { @@ -132,6 +132,11 @@ public String toString() { } }; this.queue = new PendingListenersQueue(threadPool); + this.masterServiceTaskQueue = clusterService.getTaskQueue( + "reconcile-desired-balance", + Priority.URGENT, + new ReconcileDesiredBalanceExecutor() + ); } @Override @@ -195,12 +200,7 @@ private void setCurrentDesiredBalance(DesiredBalance newDesiredBalance) { } protected void submitReconcileTask(DesiredBalance desiredBalance) { - clusterService.submitStateUpdateTask( - "reconcile-desired-balance", - new ReconcileDesiredBalanceTask(desiredBalance), - ClusterStateTaskConfig.build(Priority.URGENT), - executor - ); + masterServiceTaskQueue.submitTask("reconcile-desired-balance", new ReconcileDesiredBalanceTask(desiredBalance), null); } protected void reconcile(DesiredBalance desiredBalance, RoutingAllocation allocation) { @@ -249,7 +249,12 @@ private ReconcileDesiredBalanceTask(DesiredBalance desiredBalance) { @Override public void onFailure(Exception e) { assert MasterService.isPublishFailureException(e) : e; - onNoLongerMaster(); + if (e.getCause() != null && e.getCause()instanceof EsRejectedExecutionException esRejectedExecutionException) { + assert esRejectedExecutionException.isExecutorShutdown(); + // TODO now what? onNoLongerMaster() asserts it's on the master thread but we could be anywhere here + } else { + onNoLongerMaster(); + } } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/service/BatchSummary.java b/server/src/main/java/org/elasticsearch/cluster/service/BatchSummary.java index 8c2285a2c764b..190506e3a017c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/BatchSummary.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/BatchSummary.java @@ -8,41 +8,16 @@ package org.elasticsearch.cluster.service; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.util.LazyInitializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.function.Supplier; public class BatchSummary { - static final int MAX_TASK_DESCRIPTION_CHARS = 8 * 1024; - private final LazyInitializable lazyDescription; - public BatchSummary(TaskBatcher.BatchedTask firstTask, List allTasks) { - lazyDescription = new LazyInitializable<>(() -> { - final Map> processTasksBySource = new HashMap<>(); - for (final var task : allTasks) { - processTasksBySource.computeIfAbsent(task.source, s -> new ArrayList<>()).add(task); - } - final StringBuilder output = new StringBuilder(); - Strings.collectionToDelimitedStringWithLimit((Iterable) () -> processTasksBySource.entrySet().stream().map(entry -> { - String tasks = firstTask.describeTasks(entry.getValue()); - return tasks.isEmpty() ? entry.getKey() : entry.getKey() + "[" + tasks + "]"; - }).filter(s -> s.isEmpty() == false).iterator(), ", ", "", "", MAX_TASK_DESCRIPTION_CHARS, output); - if (output.length() > MAX_TASK_DESCRIPTION_CHARS) { - output.append(" (").append(allTasks.size()).append(" tasks in total)"); - } - return output.toString(); - }); - } - - // for tests - public BatchSummary(String string) { - lazyDescription = new LazyInitializable<>(() -> string); + public BatchSummary(Supplier stringSupplier) { + lazyDescription = new LazyInitializable<>(stringSupplier::get); } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java index dedb7e2ee452f..6d7ace67efa02 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java @@ -134,8 +134,7 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { nodeName + "/" + CLUSTER_UPDATE_THREAD_NAME, daemonThreadFactory(nodeName, CLUSTER_UPDATE_THREAD_NAME), threadPool.getThreadContext(), - threadPool.scheduler(), - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + threadPool.scheduler() ); } diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index 00f986ba4d54b..21137f36399c6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -12,7 +12,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -21,6 +20,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.OperationRouting; import org.elasticsearch.cluster.routing.RerouteService; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -243,28 +243,27 @@ public void submitUnbatchedStateUpdateTask(String source, ClusterStateUpdateTask } /** - * Submits a cluster state update task; submitted updates will be - * batched across the same instance of executor. The exact batching - * semantics depend on the underlying implementation but a rough - * guideline is that if the update task is submitted while there - * are pending update tasks for the same executor, these update - * tasks will all be executed on the executor in a single batch + * Create a new task queue which can be used to submit tasks for execution by the master service. Tasks submitted to the same queue + * (while the master service is otherwise busy) will be batched together into a single cluster state update. You should therefore re-use + * each queue as much as possible. * - * @param source the source of the cluster state update task - * @param task the state and the callback needed for the cluster state update task - * @param config the cluster state update task configuration - * @param executor the cluster state update task executor; tasks - * that share the same executor will be executed - * batches on this executor - * @param the type of the cluster state update task state + * @param name The name of the queue, which is mostly useful for debugging. * + * @param priority The priority at which tasks submitted to the queue are executed. Avoid priorites other than {@link Priority#NORMAL} + * where possible. A stream of higher-priority tasks can starve lower-priority ones from running. Higher-priority tasks + * should definitely re-use the same {@link MasterServiceTaskQueue} so that they are executed in batches. + * + * @param executor The executor which processes each batch of tasks. + * + * @param The type of the tasks + * + * @return A new batching task queue. */ - public void submitStateUpdateTask( - String source, - T task, - ClusterStateTaskConfig config, + public MasterServiceTaskQueue getTaskQueue( + String name, + Priority priority, ClusterStateTaskExecutor executor ) { - masterService.submitStateUpdateTask(source, task, config, executor); + return masterService.getTaskQueue(name, priority, executor); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/service/MasterService.java b/server/src/main/java/org/elasticsearch/cluster/service/MasterService.java index 06c4df9a87a31..9befe06248192 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/MasterService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/MasterService.java @@ -12,11 +12,11 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState.Builder; import org.elasticsearch.cluster.ClusterStateAckListener; import org.elasticsearch.cluster.ClusterStatePublicationEvent; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -27,12 +27,14 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; @@ -52,15 +54,23 @@ import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.EnumMap; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.function.Supplier; +import java.util.stream.Stream; import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory; import static org.elasticsearch.core.Strings.format; @@ -85,6 +95,8 @@ public class MasterService extends AbstractLifecycleComponent { public static final String STATE_UPDATE_ACTION_NAME = "publish_cluster_state_update"; + private final ClusterStateTaskExecutor unbatchedExecutor; + ClusterStatePublisher clusterStatePublisher; private final String nodeName; @@ -98,9 +110,14 @@ public class MasterService extends AbstractLifecycleComponent { private final TaskManager taskManager; private volatile PrioritizedEsThreadPoolExecutor threadPoolExecutor; - private volatile Batcher taskBatcher; + private final CountedQueue[] queues; + private final AtomicInteger totalQueueSize = new AtomicInteger(); + private volatile Batch currentlyExecutingBatch; + private final Map queuesByPriority; + private final LongSupplier insertionIndexSupplier = new AtomicLong()::incrementAndGet; private final ClusterStateUpdateStatsTracker clusterStateUpdateStatsTracker = new ClusterStateUpdateStatsTracker(); + private final StarvationWatcher starvationWatcher = new StarvationWatcher(); public MasterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool, TaskManager taskManager) { this.nodeName = Objects.requireNonNull(Node.NODE_NAME_SETTING.get(settings)); @@ -112,6 +129,18 @@ public MasterService(Settings settings, ClusterSettings clusterSettings, ThreadP this.threadPool = threadPool; this.taskManager = taskManager; + + final var queuesByPriorityBuilder = new EnumMap(Priority.class); + final var priorities = Priority.values(); + queues = new CountedQueue[priorities.length]; + var queueIndex = 0; + for (final var priority : priorities) { + final var queue = new CountedQueue(priority); + queues[queueIndex++] = queue; + queuesByPriorityBuilder.put(priority, queue); + } + this.queuesByPriority = Collections.unmodifiableMap(queuesByPriorityBuilder); + this.unbatchedExecutor = new UnbatchedExecutor(); } private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) { @@ -131,7 +160,6 @@ protected synchronized void doStart() { Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting"); Objects.requireNonNull(clusterStateSupplier, "please set a cluster state supplier before starting"); threadPoolExecutor = createThreadPoolExecutor(); - taskBatcher = new Batcher(logger, threadPoolExecutor); } protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { @@ -139,12 +167,7 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { nodeName + "/" + MASTER_UPDATE_THREAD_NAME, daemonThreadFactory(nodeName, MASTER_UPDATE_THREAD_NAME), threadPool.getThreadContext(), - threadPool.scheduler(), - new MasterServiceStarvationWatcher( - starvationLoggingThreshold.getMillis(), - threadPool::relativeTimeInMillis, - () -> threadPoolExecutor - ) + threadPool.scheduler() ); } @@ -152,77 +175,6 @@ public ClusterStateUpdateStats getClusterStateUpdateStats() { return clusterStateUpdateStatsTracker.getStatistics(); } - @SuppressWarnings("unchecked") - class Batcher extends TaskBatcher { - - Batcher(Logger logger, PrioritizedEsThreadPoolExecutor threadExecutor) { - super(logger, threadExecutor); - } - - @Override - protected void onTimeout(BatchedTask task, TimeValue timeout) { - threadPool.generic() - .execute(() -> ((UpdateTask) task).onFailure(new ProcessClusterEventTimeoutException(timeout, task.source), () -> {})); - } - - @Override - protected void run(Object batchingKey, List tasks, BatchSummary tasksSummary) { - runTasks((ClusterStateTaskExecutor) batchingKey, (List) tasks, tasksSummary); - } - - class UpdateTask extends BatchedTask { - private final ClusterStateTaskListener listener; - private final Supplier threadContextSupplier; - - UpdateTask( - Priority priority, - String source, - ClusterStateTaskListener task, - Supplier threadContextSupplier, - ClusterStateTaskExecutor executor - ) { - super(priority, source, executor, task); - this.threadContextSupplier = threadContextSupplier; - this.listener = task; - } - - @Override - public String describeTasks(List tasks) { - return ((ClusterStateTaskExecutor) batchingKey).describeTasks( - tasks.stream().map(task -> (ClusterStateTaskListener) task.task).toList() - ); - } - - public void onFailure(Exception e, Runnable restoreResponseHeaders) { - try (ThreadContext.StoredContext ignore = threadContextSupplier.get()) { - restoreResponseHeaders.run(); - listener.onFailure(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.error("exception thrown by listener notifying of failure", inner); - } - } - - @Nullable - public ContextPreservingAckListener wrapInTaskContext( - @Nullable ClusterStateAckListener clusterStateAckListener, - Runnable restoreResponseHeaders - ) { - return clusterStateAckListener == null - ? null - : new ContextPreservingAckListener( - Objects.requireNonNull(clusterStateAckListener), - threadContextSupplier, - restoreResponseHeaders - ); - } - - ThreadContext getThreadContext() { - return threadPool.getThreadContext(); - } - } - } - @Override protected synchronized void doStop() { ThreadPool.terminate(threadPoolExecutor, 10, TimeUnit.SECONDS); @@ -252,10 +204,10 @@ assert isMasterUpdateThread() == false return true; } - private void runTasks( - ClusterStateTaskExecutor executor, - List updateTasks, - BatchSummary summary + private void executeAndPublishBatch( + final ClusterStateTaskExecutor executor, + final List> executionResults, + final BatchSummary summary ) { if (lifecycle.started() == false) { logger.debug("processing [{}]: ignoring, master service not started", summary); @@ -267,21 +219,21 @@ private void runTasks( if (previousClusterState.nodes().isLocalNodeElectedMaster() == false && executor.runOnlyOnMaster()) { logger.debug("failing [{}]: local node is no longer master", summary); - updateTasks.forEach(t -> t.onFailure(new NotMasterException("no longer master, failing [" + t.source() + "]"), () -> {})); + for (ExecutionResult executionResult : executionResults) { + executionResult.onBatchFailure(new NotMasterException("no longer master")); + executionResult.notifyOnFailure(); + } return; } final long computationStartTime = threadPool.rawRelativeTimeInMillis(); - final var executionResults = updateTasks.stream().map(ExecutionResult::new).toList(); final var newClusterState = patchVersions( previousClusterState, executeTasks(previousClusterState, executionResults, executor, summary, threadPool.getThreadContext()) ); // fail all tasks that have failed for (final var executionResult : executionResults) { - if (executionResult.failure != null) { - executionResult.updateTask.onFailure(executionResult.failure, executionResult::restoreResponseHeaders); - } + executionResult.notifyOnFailure(); } final TimeValue computationTime = getTimeSince(computationStartTime); logExecutionTime(computationTime, "compute cluster state update", summary); @@ -306,11 +258,11 @@ private void runTasks( } } - private void publishClusterStateUpdate( - ClusterStateTaskExecutor executor, + private void publishClusterStateUpdate( + ClusterStateTaskExecutor executor, BatchSummary summary, ClusterState previousClusterState, - List> executionResults, + List> executionResults, ClusterState newClusterState, TimeValue computationTime ) { @@ -447,6 +399,7 @@ public void onFailure(Exception exception) { } ); } catch (Exception e) { + assert publicationMayFail() : e; handleException(summary, publicationStartTime, newClusterState, e); } } finally { @@ -482,22 +435,17 @@ protected boolean blockingAllowed() { } private void handleException(BatchSummary summary, long startTimeMillis, ClusterState newClusterState, Exception e) { - final TimeValue executionTime = getTimeSince(startTimeMillis); - final long version = newClusterState.version(); - final String stateUUID = newClusterState.stateUUID(); - final String fullState = newClusterState.toString(); logger.warn( () -> format( "took [%s] and then failed to publish updated cluster state (version: %s, uuid: %s) for [%s]:\n%s", - executionTime, - version, - stateUUID, + getTimeSince(startTimeMillis), + newClusterState.version(), + newClusterState.stateUUID(), summary, - fullState + newClusterState ), e ); - // TODO: do we want to call updateTask.onFailure here? } private ClusterState patchVersions(ClusterState previousClusterState, ClusterState newClusterState) { @@ -534,8 +482,97 @@ public Builder incrementVersion(ClusterState clusterState) { */ @Deprecated public void submitUnbatchedStateUpdateTask(String source, ClusterStateUpdateTask updateTask) { - // NB new executor each time so as to avoid batching - submitStateUpdateTask(source, updateTask, updateTask, new UnbatchedExecutor()); + final var summary = new BatchSummary(() -> source); + final var restorableContext = threadPool.getThreadContext().newRestorableContext(true); + final var executed = new AtomicBoolean(false); + final Scheduler.Cancellable timeoutCancellable; + final var timeout = updateTask.timeout(); + if (timeout != null && timeout.millis() > 0) { + timeoutCancellable = threadPool.schedule( + new TaskTimeoutHandler(timeout, source, executed, updateTask), + timeout, + ThreadPool.Names.GENERIC + ); + } else { + timeoutCancellable = null; + } + final long insertionIndex = insertionIndexSupplier.getAsLong(); + final long insertionTime = threadPool.relativeTimeInMillis(); + + queuesByPriority.get(updateTask.priority()).execute(new Batch() { + private volatile boolean isRunning; + + @Override + public Stream getPending(long currentTimeMillis) { + if (isTimedOut()) { + return Stream.of(); + } + return Stream.of( + new PendingClusterTask( + insertionIndex, + updateTask.priority(), + new Text(source), + currentTimeMillis - insertionTime, + executed.get() + ) + ); + } + + @Override + public int getPendingCount() { + return isTimedOut() ? 0 : 1; + } + + @Override + public long getCreationTimeMillis() { + return isTimedOut() ? Long.MAX_VALUE : insertionTime; + } + + private boolean isTimedOut() { + return executed.get() && isRunning == false; + } + + @Override + public void onRejection(FailedToCommitClusterStateException e) { + try { + if (acquireForExecution()) { + try (var ignored = restorableContext.get()) { + updateTask.onFailure(e); + } + } + } catch (Exception e2) { + e2.addSuppressed(e); + logger.error(() -> format("unexpected exception failing task [%s]", source), e2); + assert false : e2; + } + } + + @Override + public void run() { + isRunning = true; + try { + if (acquireForExecution()) { + executeAndPublishBatch( + unbatchedExecutor, + List.of(new ExecutionResult<>(source, updateTask, threadPool.getThreadContext(), restorableContext)), + summary + ); + } + } finally { + isRunning = false; + } + } + + private boolean acquireForExecution() { + if (executed.compareAndSet(false, true) == false) { + return false; + } + if (timeoutCancellable != null) { + timeoutCancellable.cancel(); + } + return true; + } + }); } private static class UnbatchedExecutor implements ClusterStateTaskExecutor { @@ -568,71 +605,31 @@ public String describeTasks(List tasks) { } } - /** - * Submits a cluster state update task; submitted updates will be - * batched across the same instance of executor. The exact batching - * semantics depend on the underlying implementation but a rough - * guideline is that if the update task is submitted while there - * are pending update tasks for the same executor, these update - * tasks will all be executed on the executor in a single batch - * - * @param source the source of the cluster state update task - * @param task the state needed for the cluster state update task, which implements {@link ClusterStateTaskListener} so that it is - * notified when it is executed. - * @param config the cluster state update task configuration - * @param executor the cluster state update task executor; tasks - * that share the same executor will be executed - * batches on this executor - * @param the type of the cluster state update task state - * - */ - public void submitStateUpdateTask( - String source, - T task, - ClusterStateTaskConfig config, - ClusterStateTaskExecutor executor - ) { - if (lifecycle.started() == false) { - return; - } - final ThreadContext threadContext = threadPool.getThreadContext(); - final Supplier supplier = threadContext.newRestorableContext(true); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - threadContext.markAsSystemContext(); - taskBatcher.submitTask(taskBatcher.new UpdateTask(config.priority(), source, task, supplier, executor), config.timeout()); - } catch (EsRejectedExecutionException e) { - // ignore cases where we are shutting down..., there is really nothing interesting - // to be done here... - if (lifecycle.stoppedOrClosed() == false) { - throw e; - } - } - } - /** * Returns the tasks that are pending. */ public List pendingTasks() { - return Arrays.stream(threadPoolExecutor.getPending()).map(pending -> { - assert pending.task instanceof SourcePrioritizedRunnable - : "thread pool executor should only use SourcePrioritizedRunnable instances but found: " - + pending.task.getClass().getName(); - SourcePrioritizedRunnable task = (SourcePrioritizedRunnable) pending.task; - return new PendingClusterTask( - pending.insertionOrder, - pending.priority, - new Text(task.source()), - task.getAgeInMillis(), - pending.executing - ); - }).toList(); + final var currentTimeMillis = threadPool.relativeTimeInMillis(); + return Stream.concat(Stream.ofNullable(currentlyExecutingBatch), Arrays.stream(queues).flatMap(q -> q.queue.stream())) + .flatMap(e -> e.getPending(currentTimeMillis)) + .toList(); } /** * Returns the number of currently pending tasks. */ public int numberOfPendingTasks() { - return threadPoolExecutor.getNumberOfPendingTasks(); + var result = getPendingCountOrZero(currentlyExecutingBatch); // single volatile read + for (final var queue : queues) { + for (final var entry : queue.queue) { + result += entry.getPendingCount(); + } + } + return result; + } + + private static int getPendingCountOrZero(@Nullable Batch batch) { + return batch == null ? 0 : batch.getPendingCount(); } /** @@ -641,7 +638,16 @@ public int numberOfPendingTasks() { * @return A zero time value if the queue is empty, otherwise the time value oldest task waiting in the queue */ public TimeValue getMaxTaskWaitTime() { - return threadPoolExecutor.getMaxTaskWaitTime(); + final var oldestTaskTimeMillis = Stream.concat( + Stream.ofNullable(currentlyExecutingBatch), + Arrays.stream(queues).flatMap(q -> q.queue.stream()) + ).mapToLong(Batch::getCreationTimeMillis).min().orElse(Long.MAX_VALUE); + + if (oldestTaskTimeMillis == Long.MAX_VALUE) { + return TimeValue.ZERO; + } + + return TimeValue.timeValueMillis(threadPool.relativeTimeInMillis() - oldestTaskTimeMillis); } private void logExecutionTime(TimeValue executionTime, String activity, BatchSummary summary) { @@ -820,7 +826,10 @@ public void onNodeAck(DiscoveryNode node, @Nullable Exception e) { } private static class ExecutionResult implements ClusterStateTaskExecutor.TaskContext { - final Batcher.UpdateTask updateTask; + private final String source; + private final T task; + private final ThreadContext threadContext; + private final Supplier threadContextSupplier; @Nullable // if the task is incomplete or failed or onPublicationSuccess supplied Consumer publishedStateConsumer; @@ -837,14 +846,20 @@ private static class ExecutionResult impleme @Nullable Map> responseHeaders; - ExecutionResult(Batcher.UpdateTask updateTask) { - this.updateTask = updateTask; + ExecutionResult(String source, T task, ThreadContext threadContext, Supplier threadContextSupplier) { + this.source = source; + this.task = task; + this.threadContext = threadContext; + this.threadContextSupplier = threadContextSupplier; + } + + public String getSource() { + return source; } - @SuppressWarnings("unchecked") // trust us this is ok @Override public T getTask() { - return (T) updateTask.getTask(); + return task; } private boolean incomplete() { @@ -909,7 +924,6 @@ public void onFailure(Exception failure) { @Override public Releasable captureResponseHeaders() { - final var threadContext = updateTask.getThreadContext(); final var storedContext = threadContext.newStoredContext(); return Releasables.wrap(() -> { final var newResponseHeaders = threadContext.getResponseHeaders(); @@ -935,7 +949,7 @@ private void restoreResponseHeaders() { if (responseHeaders != null) { for (final var responseHeader : responseHeaders.entrySet()) { for (final var value : responseHeader.getValue()) { - updateTask.getThreadContext().addResponseHeader(responseHeader.getKey(), value); + threadContext.addResponseHeader(responseHeader.getKey(), value); } } } @@ -953,7 +967,7 @@ void onPublishSuccess(ClusterState newClusterState) { assert failure != null; return; } - try (ThreadContext.StoredContext ignored = updateTask.threadContextSupplier.get()) { + try (ThreadContext.StoredContext ignored = threadContextSupplier.get()) { restoreResponseHeaders(); if (onPublicationSuccess == null) { publishedStateConsumer.accept(newClusterState); @@ -970,7 +984,7 @@ void onClusterStateUnchanged(ClusterState clusterState) { assert failure != null; return; } - try (ThreadContext.StoredContext ignored = updateTask.threadContextSupplier.get()) { + try (ThreadContext.StoredContext ignored = threadContextSupplier.get()) { restoreResponseHeaders(); if (onPublicationSuccess == null) { publishedStateConsumer.accept(clusterState); @@ -987,7 +1001,7 @@ void onPublishFailure(FailedToCommitClusterStateException e) { assert failure != null; return; } - try (ThreadContext.StoredContext ignored = updateTask.threadContextSupplier.get()) { + try (ThreadContext.StoredContext ignored = threadContextSupplier.get()) { restoreResponseHeaders(); getTask().onFailure(e); } catch (Exception inner) { @@ -998,19 +1012,35 @@ void onPublishFailure(FailedToCommitClusterStateException e) { ContextPreservingAckListener getContextPreservingAckListener() { assert incomplete() == false; - return updateTask.wrapInTaskContext(clusterStateAckListener, this::restoreResponseHeaders); + if (clusterStateAckListener == null) { + return null; + } else { + return new ContextPreservingAckListener(clusterStateAckListener, threadContextSupplier, this::restoreResponseHeaders); + } } @Override public String toString() { - return "TaskContextImpl[" + updateTask.getTask() + "]"; + return "ExecutionResult[" + task + "]"; + } + + void notifyOnFailure() { + if (failure != null) { + try (ThreadContext.StoredContext ignore = threadContextSupplier.get()) { + restoreResponseHeaders(); + getTask().onFailure(failure); + } catch (Exception inner) { + inner.addSuppressed(failure); + logger.error("exception thrown by listener notifying of failure", inner); + } + } } } - private static ClusterState executeTasks( + private static ClusterState executeTasks( ClusterState previousClusterState, - List> executionResults, - ClusterStateTaskExecutor executor, + List> executionResults, + ClusterStateTaskExecutor executor, BatchSummary summary, ThreadContext threadContext ) { @@ -1024,26 +1054,29 @@ private static ClusterState executeTasks( return resultingState; } - private static boolean assertAllTasksComplete(List> executionResults) { + private static boolean assertAllTasksComplete(List> executionResults) { for (final var executionResult : executionResults) { assert executionResult.incomplete() == false : "missing result for " + executionResult; } return true; } - @SuppressWarnings("unchecked") // the input is unmodifiable so it is ok to cast to a more general element type - private static List> castTaskContexts(List executionResults) { - return (List>) executionResults; + @SuppressWarnings("unchecked") + private static List> castTaskContexts( + List executionResults + ) { + // the input is unmodifiable so it is ok to cast to a more general element type + return (List>) executionResults; } - private static ClusterState innerExecuteTasks( + private static ClusterState innerExecuteTasks( ClusterState previousClusterState, - List> executionResults, - ClusterStateTaskExecutor executor, + List> executionResults, + ClusterStateTaskExecutor executor, BatchSummary summary, ThreadContext threadContext ) { - final var taskContexts = castTaskContexts(executionResults); + final List> taskContexts = castTaskContexts(executionResults); try (var ignored = threadContext.newStoredContext()) { // if the executor leaks a response header then this will cause a test failure, but we also store the context here to be sure // to avoid leaking headers in production that were missed by tests @@ -1086,35 +1119,18 @@ Batched task executors must marshal response headers to the appropriate task con } } - private static class MasterServiceStarvationWatcher implements PrioritizedEsThreadPoolExecutor.StarvationWatcher { - - private final long warnThreshold; - private final LongSupplier nowMillisSupplier; - private final Supplier threadPoolExecutorSupplier; - + private class StarvationWatcher { // accesses of these mutable fields are synchronized (on this) private long lastLogMillis; private long nonemptySinceMillis; private boolean isEmpty = true; - MasterServiceStarvationWatcher( - long warnThreshold, - LongSupplier nowMillisSupplier, - Supplier threadPoolExecutorSupplier - ) { - this.nowMillisSupplier = nowMillisSupplier; - this.threadPoolExecutorSupplier = threadPoolExecutorSupplier; - this.warnThreshold = warnThreshold; - } - - @Override - public synchronized void onEmptyQueue() { + synchronized void onEmptyQueue() { isEmpty = true; } - @Override - public void onNonemptyQueue() { - final long nowMillis = nowMillisSupplier.getAsLong(); + void onNonemptyQueue() { + final long nowMillis = threadPool.relativeTimeInMillis(); final long nonemptyDurationMillis; synchronized (this) { if (isEmpty) { @@ -1124,7 +1140,7 @@ public void onNonemptyQueue() { return; } - if (nowMillis - lastLogMillis < warnThreshold) { + if (nowMillis - lastLogMillis < starvationLoggingThreshold.millis()) { return; } @@ -1132,15 +1148,14 @@ public void onNonemptyQueue() { nonemptyDurationMillis = nowMillis - nonemptySinceMillis; } - final PrioritizedEsThreadPoolExecutor threadPoolExecutor = threadPoolExecutorSupplier.get(); - final TimeValue maxTaskWaitTime = threadPoolExecutor.getMaxTaskWaitTime(); + final TimeValue maxTaskWaitTime = getMaxTaskWaitTime(); logger.warn( "pending task queue has been nonempty for [{}/{}ms] which is longer than the warn threshold of [{}ms];" + " there are currently [{}] pending tasks, the oldest of which has age [{}/{}ms]", TimeValue.timeValueMillis(nonemptyDurationMillis), nonemptyDurationMillis, - warnThreshold, - threadPoolExecutor.getNumberOfPendingTasks(), + starvationLoggingThreshold.millis(), + numberOfPendingTasks(), maxTaskWaitTime, maxTaskWaitTime.millis() ); @@ -1236,4 +1251,457 @@ synchronized ClusterStateUpdateStats getStatistics() { public static boolean isPublishFailureException(Exception e) { return e instanceof NotMasterException || e instanceof FailedToCommitClusterStateException; } + + private final Runnable queuesProcessor = new Runnable() { + @Override + public void run() { + assert threadPool.getThreadContext().isSystemContext(); + assert totalQueueSize.get() > 0; + assert currentlyExecutingBatch == null; + try { + final var nextBatch = takeNextItem(); + if (lifecycle.started()) { + nextBatch.run(); + } else { + nextBatch.onRejection(new FailedToCommitClusterStateException("node closed", getRejectionException())); + } + } catch (Exception e) { + logger.error("unexpected exception executing queue entry", e); + assert false : e; + } finally { + currentlyExecutingBatch = null; + if (totalQueueSize.decrementAndGet() > 0) { + starvationWatcher.onNonemptyQueue(); + forkQueueProcessor(); + } else { + starvationWatcher.onEmptyQueue(); + } + } + } + + @Override + public String toString() { + return "master service queue processor"; + } + }; + + private Batch takeNextItem() { + assert totalQueueSize.get() > 0; + assert currentlyExecutingBatch == null; + for (final var queue : queues) { + var item = queue.queue.poll(); + if (item != null) { + currentlyExecutingBatch = item; + return item; + } + } + logger.error("queue processor found no items"); + assert false : "queue processor found no items"; + throw new IllegalStateException("queue processor found no items"); + } + + private void forkQueueProcessor() { + if (lifecycle.started() == false) { + drainQueueOnRejection(new FailedToCommitClusterStateException("node closed", getRejectionException())); + return; + } + + try { + assert totalQueueSize.get() > 0; + final var threadContext = threadPool.getThreadContext(); + try (var ignored = threadContext.stashContext()) { + threadContext.markAsSystemContext(); + threadPoolExecutor.execute(queuesProcessor); + } + } catch (Exception e) { + assert e instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown() : e; + drainQueueOnRejection(new FailedToCommitClusterStateException("node closed", e)); + } + } + + private EsRejectedExecutionException getRejectionException() { + assert lifecycle.started() == false; + return new EsRejectedExecutionException("master service is in state [" + lifecycleState() + "]", true); + } + + private void drainQueueOnRejection(FailedToCommitClusterStateException e) { + assert totalQueueSize.get() > 0; + do { + final var nextItem = takeNextItem(); + try { + nextItem.onRejection(e); + } catch (Exception e2) { + e2.addSuppressed(e); + logger.error(() -> format("exception failing item on rejection [%s]", nextItem), e2); + assert false : e2; + } finally { + currentlyExecutingBatch = null; + } + } while (totalQueueSize.decrementAndGet() > 0); + } + + /** + * Queue which tracks the count of items, allowing it to determine (in a threadsafe fashion) the transitions between empty and nonempty, + * so that it can spawn an action to process its elements if and only if it's needed. This allows it to ensure that there is only ever + * at most one active {@link #queuesProcessor}, and that there's always a pending processor if there is work to be done. + * + * There is one of these queues for each priority level. + */ + private class CountedQueue { + private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final Priority priority; + + CountedQueue(Priority priority) { + this.priority = priority; + } + + void execute(Batch runner) { + queue.add(runner); + if (totalQueueSize.getAndIncrement() == 0) { + starvationWatcher.onEmptyQueue(); + forkQueueProcessor(); + } + } + + Priority priority() { + return priority; + } + } + + private interface Batch { + + void run(); + + /** + * Called when the batch is rejected due to the master service shutting down. + * + * @param e is a {@link FailedToCommitClusterStateException} to cause things like {@link TransportMasterNodeAction} to retry after + * submitting a task to a master which shut down. {@code e.getCause()} is the rejection exception, which should be a + * {@link EsRejectedExecutionException} with {@link EsRejectedExecutionException#isExecutorShutdown()} true. + */ + // Should really be a NodeClosedException instead, but this exception type doesn't trigger retries today. + void onRejection(FailedToCommitClusterStateException e); + + /** + * @return number of tasks in this batch if the batch is pending, or {@code 0} if the batch is not pending. + */ + int getPendingCount(); + + /** + * @return the tasks in this batch if the batch is pending, or an empty stream if the batch is not pending. + */ + Stream getPending(long currentTimeMillis); + + /** + * @return the earliest insertion time of the tasks in this batch if the batch is pending, or {@link Long#MAX_VALUE} otherwise. + */ + long getCreationTimeMillis(); + } + + /** + * Create a new task queue which can be used to submit tasks for execution by the master service. Tasks submitted to the same queue + * (while the master service is otherwise busy) will be batched together into a single cluster state update. You should therefore re-use + * each queue as much as possible. + * + * @param name The name of the queue, which is mostly useful for debugging. + * + * @param priority The priority at which tasks submitted to the queue are executed. Avoid priorites other than {@link Priority#NORMAL} + * where possible. A stream of higher-priority tasks can starve lower-priority ones from running. Higher-priority tasks + * should definitely re-use the same {@link MasterServiceTaskQueue} so that they are executed in batches. + * + * @param executor The executor which processes each batch of tasks. + * + * @param The type of the tasks + * + * @return A new batching task queue. + */ + public MasterServiceTaskQueue getTaskQueue( + String name, + Priority priority, + ClusterStateTaskExecutor executor + ) { + return new BatchingTaskQueue<>( + name, + this::executeAndPublishBatch, + insertionIndexSupplier, + queuesByPriority.get(priority), + executor, + threadPool + ); + } + + @FunctionalInterface + private interface BatchConsumer { + void runBatch(ClusterStateTaskExecutor executor, List> tasks, BatchSummary summary); + } + + private static class TaskTimeoutHandler extends AbstractRunnable { + + private final TimeValue timeout; + private final String source; + private final AtomicBoolean executed; + private final ClusterStateTaskListener listener; + + private TaskTimeoutHandler(TimeValue timeout, String source, AtomicBoolean executed, ClusterStateTaskListener listener) { + this.timeout = timeout; + this.source = source; + this.executed = executed; + this.listener = listener; + } + + @Override + public void onRejection(Exception e) { + assert e instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown() : e; + completeTask(e); + } + + @Override + public void onFailure(Exception e) { + logger.error("unexpected failure executing task timeout handler", e); + assert false : e; + completeTask(e); + } + + @Override + public boolean isForceExecution() { + return true; + } + + @Override + protected void doRun() { + completeTask(new ProcessClusterEventTimeoutException(timeout, source)); + } + + private void completeTask(Exception e) { + if (executed.compareAndSet(false, true)) { + listener.onFailure(e); + } + } + } + + /** + * Actual implementation of {@link MasterServiceTaskQueue} exposed to clients. Conceptually, each entry in each {@link CountedQueue} is + * a {@link BatchingTaskQueue} representing a batch of tasks to be executed. Clients may add more tasks to each of these queues prior to + * their execution. + * + * Works similarly to {@link CountedQueue} in that the queue size is tracked in a threadsafe fashion so that we can detect transitions + * between empty and nonempty queues and arrange to process the queue if and only if it's nonempty. There is only ever one active + * processor for each such queue. + * + * Works differently from {@link CountedQueue} in that each time the queue is processed it will drain all the pending items at once and + * process them in a single batch. + * + * Also handles that tasks may time out before being processed. + */ + private static class BatchingTaskQueue implements MasterServiceTaskQueue { + + private final ConcurrentLinkedQueue> queue = new ConcurrentLinkedQueue<>(); + private final ConcurrentLinkedQueue> executing = new ConcurrentLinkedQueue<>(); // executing tasks are also shown in APIs + private final AtomicInteger queueSize = new AtomicInteger(); + private final String name; + private final BatchConsumer batchConsumer; + private final LongSupplier insertionIndexSupplier; + private final CountedQueue countedQueue; + private final ClusterStateTaskExecutor executor; + private final ThreadPool threadPool; + private final Batch processor = new Processor(); + + BatchingTaskQueue( + String name, + BatchConsumer batchConsumer, + LongSupplier insertionIndexSupplier, + CountedQueue countedQueue, + ClusterStateTaskExecutor executor, + ThreadPool threadPool + ) { + this.name = name; + this.batchConsumer = batchConsumer; + this.insertionIndexSupplier = insertionIndexSupplier; + this.countedQueue = countedQueue; + this.executor = executor; + this.threadPool = threadPool; + } + + @Override + public void submitTask(String source, T task, @Nullable TimeValue timeout) { + final var executed = new AtomicBoolean(false); + final Scheduler.Cancellable timeoutCancellable; + if (timeout != null && timeout.millis() > 0) { + timeoutCancellable = threadPool.schedule( + new TaskTimeoutHandler(timeout, source, executed, task), + timeout, + ThreadPool.Names.GENERIC + ); + } else { + timeoutCancellable = null; + } + + queue.add( + new Entry<>( + source, + task, + insertionIndexSupplier.getAsLong(), + threadPool.relativeTimeInMillis(), + executed, + threadPool.getThreadContext().newRestorableContext(true), + timeoutCancellable + ) + ); + + if (queueSize.getAndIncrement() == 0) { + countedQueue.execute(processor); + } + } + + @Override + public String toString() { + return "BatchingTaskQueue[" + name + "]"; + } + + private record Entry ( + String source, + T task, + long insertionIndex, + long insertionTimeMillis, + AtomicBoolean executed, + Supplier storedContextSupplier, + @Nullable Scheduler.Cancellable timeoutCancellable + ) { + boolean acquireForExecution() { + if (executed.compareAndSet(false, true) == false) { + return false; + } + + if (timeoutCancellable != null) { + timeoutCancellable.cancel(); + } + return true; + } + + void onRejection(FailedToCommitClusterStateException e) { + if (acquireForExecution()) { + try (var ignored = storedContextSupplier.get()) { + task.onFailure(e); + } catch (Exception e2) { + e2.addSuppressed(e); + logger.error(() -> format("exception failing task [%s] on rejection", task), e2); + assert false : e2; + } + } + } + } + + private class Processor implements Batch { + @Override + public void onRejection(FailedToCommitClusterStateException e) { + final var items = queueSize.getAndSet(0); + for (int i = 0; i < items; i++) { + final var entry = queue.poll(); + assert entry != null; + entry.onRejection(e); + } + } + + @Override + public void run() { + assert executing.isEmpty() : executing; + final var entryCount = queueSize.getAndSet(0); + var taskCount = 0; + for (int i = 0; i < entryCount; i++) { + final var entry = queue.poll(); + assert entry != null; + if (entry.acquireForExecution()) { + taskCount += 1; + executing.add(entry); + } + } + if (taskCount == 0) { + return; + } + final var finalTaskCount = taskCount; + final var tasks = new ArrayList>(finalTaskCount); + for (final var entry : executing) { + tasks.add( + new ExecutionResult<>(entry.source(), entry.task(), threadPool.getThreadContext(), entry.storedContextSupplier()) + ); + } + try { + batchConsumer.runBatch(executor, tasks, new BatchSummary(() -> buildTasksDescription(tasks))); + } finally { + assert executing.size() == finalTaskCount; + executing.clear(); + } + } + + private String buildTasksDescription(List> tasks) { + final var tasksBySource = new HashMap>(); + for (final var entry : tasks) { + tasksBySource.computeIfAbsent(entry.getSource(), ignored -> new ArrayList<>()).add(entry.getTask()); + } + + final var output = new StringBuilder(); + Strings.collectionToDelimitedStringWithLimit((Iterable) () -> tasksBySource.entrySet().stream().map(entry -> { + var tasksDescription = executor.describeTasks(entry.getValue()); + return tasksDescription.isEmpty() ? entry.getKey() : entry.getKey() + "[" + tasksDescription + "]"; + }).filter(s -> s.isEmpty() == false).iterator(), ", ", "", "", MAX_TASK_DESCRIPTION_CHARS, output); + if (output.length() > MAX_TASK_DESCRIPTION_CHARS) { + output.append(" (").append(tasks.size()).append(" tasks in total)"); + } + return output.toString(); + } + + @Override + public Stream getPending(long currentTimeMillis) { + return Stream.concat( + executing.stream() + .map( + entry -> new PendingClusterTask( + entry.insertionIndex(), + countedQueue.priority(), + new Text(entry.source()), + currentTimeMillis - entry.insertionTimeMillis(), + true + ) + ), + queue.stream() + .filter(entry -> entry.executed().get() == false) + .map( + entry -> new PendingClusterTask( + entry.insertionIndex(), + countedQueue.priority(), + new Text(entry.source()), + currentTimeMillis - entry.insertionTimeMillis(), + false + ) + ) + ); + } + + @Override + public int getPendingCount() { + int count = executing.size(); + for (final var entry : queue) { + if (entry.executed().get() == false) { + count += 1; + } + } + return count; + } + + @Override + public long getCreationTimeMillis() { + return Stream.concat(executing.stream(), queue.stream().filter(entry -> entry.executed().get() == false)) + .mapToLong(Entry::insertionTimeMillis) + .min() + .orElse(Long.MAX_VALUE); + } + + @Override + public String toString() { + return "process queue for [" + name + "]"; + } + } + } + + static final int MAX_TASK_DESCRIPTION_CHARS = 8 * 1024; } diff --git a/server/src/main/java/org/elasticsearch/cluster/service/MasterServiceTaskQueue.java b/server/src/main/java/org/elasticsearch/cluster/service/MasterServiceTaskQueue.java new file mode 100644 index 0000000000000..13573e20fd321 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/service/MasterServiceTaskQueue.java @@ -0,0 +1,40 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.cluster.service; + +import org.elasticsearch.action.support.master.MasterNodeRequest; +import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; + +/** + * A queue of tasks for the master service to execute. Tasks submitted to the same queue can be processed as a batch, resulting in a single + * cluster state update. Queues are typically created during initialization using {@link MasterService#getTaskQueue}. + * + * @param The type of task to process. + */ +public interface MasterServiceTaskQueue { + + /** + * Submit a task to the queue. + * + * @param source A description of the task. + * + * @param task The task to execute. + * + * @param timeout An optional timeout for the task. If the task is not processed before the timeout elapses, it fails with a {@link + * ProcessClusterEventTimeoutException} (which is passed to {@link ClusterStateTaskListener#onFailure}). Tasks that are + * directly associated with user actions conventionally use a timeout which comes from the REST parameter {@code + * ?master_timeout}, which is typically available from {@link MasterNodeRequest#masterNodeTimeout()}. Tasks that + * correspond with internal actions should normally have no timeout since it is usually better to wait patiently in the + * queue until processed rather than to fail, especially if the only reasonable reaction to a failure is to retry. + */ + void submitTask(String source, T task, @Nullable TimeValue timeout); +} diff --git a/server/src/main/java/org/elasticsearch/cluster/service/TaskBatcher.java b/server/src/main/java/org/elasticsearch/cluster/service/TaskBatcher.java deleted file mode 100644 index eaa56f692af90..0000000000000 --- a/server/src/main/java/org/elasticsearch/cluster/service/TaskBatcher.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.cluster.service; - -import org.apache.logging.log4j.Logger; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.core.TimeValue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Batching support for {@link PrioritizedEsThreadPoolExecutor} - * Tasks that share the same batching key are batched (see {@link BatchedTask#batchingKey}) - */ -public abstract class TaskBatcher { - - private final Logger logger; - private final PrioritizedEsThreadPoolExecutor threadExecutor; - // package visible for tests - final Map> tasksPerBatchingKey = new ConcurrentHashMap<>(); - - public TaskBatcher(Logger logger, PrioritizedEsThreadPoolExecutor threadExecutor) { - this.logger = logger; - this.threadExecutor = threadExecutor; - } - - public void submitTask(BatchedTask task, @Nullable TimeValue timeout) throws EsRejectedExecutionException { - tasksPerBatchingKey.compute(task.batchingKey, (k, existingTasks) -> { - if (existingTasks == null) { - existingTasks = Collections.synchronizedSet(new LinkedHashSet<>()); - } else { - assert assertNoDuplicateTasks(task, existingTasks); - } - existingTasks.add(task); - return existingTasks; - }); - - if (timeout != null) { - threadExecutor.execute(task, timeout, () -> onTimeoutInternal(task, timeout)); - } else { - threadExecutor.execute(task); - } - } - - private static boolean assertNoDuplicateTasks(BatchedTask task, Set existingTasks) { - for (final var existingTask : existingTasks) { - assert existingTask.getTask() != task.getTask() - : "task [" + task.describeTasks(List.of(task)) + "] with source [" + task.source + "] is already queued"; - } - return true; - } - - private void onTimeoutInternal(BatchedTask task, TimeValue timeout) { - if (task.processed.getAndSet(true)) { - return; - } - - logger.debug("task [{}] timed out after [{}]", task.source, timeout); - tasksPerBatchingKey.computeIfPresent(task.batchingKey, (key, existingTasks) -> { - existingTasks.remove(task); - return existingTasks.isEmpty() ? null : existingTasks; - }); - onTimeout(task, timeout); - } - - /** - * Action to be implemented by the specific batching implementation. - * All tasks have the same batching key. - */ - protected abstract void onTimeout(BatchedTask task, TimeValue timeout); - - void runIfNotProcessed(BatchedTask updateTask) { - // if this task is already processed, it shouldn't execute other tasks with same batching key that arrived later, - // to give other tasks with different batching key a chance to execute. - if (updateTask.processed.get() == false) { - final List toExecute = new ArrayList<>(); - final Set pending = tasksPerBatchingKey.remove(updateTask.batchingKey); - if (pending != null) { - // pending is a java.util.Collections.SynchronizedSet so we can safely iterate holding its mutex - // noinspection SynchronizationOnLocalVariableOrMethodParameter - synchronized (pending) { - for (BatchedTask task : pending) { - if (task.processed.getAndSet(true) == false) { - logger.trace("will process {}", task); - toExecute.add(task); - } else { - logger.trace("skipping {}, already processed", task); - } - } - } - } - - if (toExecute.isEmpty() == false) { - run(updateTask.batchingKey, toExecute, new BatchSummary(updateTask, toExecute)); - } - } - } - - /** - * Action to be implemented by the specific batching implementation - * All tasks have the given batching key. - */ - protected abstract void run(Object batchingKey, List tasks, BatchSummary tasksSummarySupplier); - - /** - * Represents a runnable task that supports batching. - * Implementors of TaskBatcher can subclass this to add a payload to the task. - */ - protected abstract class BatchedTask extends SourcePrioritizedRunnable { - /** - * whether the task has been processed already - */ - protected final AtomicBoolean processed = new AtomicBoolean(); - - /** - * the object that is used as batching key - */ - protected final Object batchingKey; - /** - * the task object that is wrapped - */ - protected final Object task; - - protected BatchedTask(Priority priority, String source, Object batchingKey, Object task) { - super(priority, source); - this.batchingKey = batchingKey; - this.task = task; - } - - @Override - public void run() { - runIfNotProcessed(this); - } - - @Override - public String toString() { - String taskDescription = describeTasks(Collections.singletonList(this)); - if (taskDescription.isEmpty()) { - return "[" + source + "]"; - } else { - return "[" + source + "[" + taskDescription + "]]"; - } - } - - public abstract String describeTasks(List tasks); - - public Object getTask() { - return task; - } - } -} diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index 266f698e904af..43e6e90548b7f 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -85,20 +85,9 @@ public static PrioritizedEsThreadPoolExecutor newSinglePrioritizing( String name, ThreadFactory threadFactory, ThreadContext contextHolder, - ScheduledExecutorService timer, - PrioritizedEsThreadPoolExecutor.StarvationWatcher starvationWatcher + ScheduledExecutorService timer ) { - return new PrioritizedEsThreadPoolExecutor( - name, - 1, - 1, - 0L, - TimeUnit.MILLISECONDS, - threadFactory, - contextHolder, - timer, - starvationWatcher - ); + return new PrioritizedEsThreadPoolExecutor(name, 1, 1, 0L, TimeUnit.MILLISECONDS, threadFactory, contextHolder, timer); } public static EsThreadPoolExecutor newScaling( diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java index d4a2c6ac8be32..7c5bc0468c30b 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java @@ -36,7 +36,6 @@ public class PrioritizedEsThreadPoolExecutor extends EsThreadPoolExecutor { private final AtomicLong insertionOrder = new AtomicLong(); private final Queue current = ConcurrentCollections.newQueue(); private final ScheduledExecutorService timer; - private final StarvationWatcher starvationWatcher; public PrioritizedEsThreadPoolExecutor( String name, @@ -46,12 +45,10 @@ public PrioritizedEsThreadPoolExecutor( TimeUnit unit, ThreadFactory threadFactory, ThreadContext contextHolder, - ScheduledExecutorService timer, - StarvationWatcher starvationWatcher + ScheduledExecutorService timer ) { super(name, corePoolSize, maximumPoolSize, keepAliveTime, unit, new PriorityBlockingQueue<>(), threadFactory, contextHolder); this.timer = timer; - this.starvationWatcher = starvationWatcher; } public Pending[] getPending() { @@ -112,20 +109,12 @@ private void addPending(List runnables, List pending, boolean @Override protected void beforeExecute(Thread t, Runnable r) { current.add(r); - if (getQueue().isEmpty()) { - starvationWatcher.onEmptyQueue(); - } } @Override protected void afterExecute(Runnable r, Throwable t) { super.afterExecute(r, t); current.remove(r); - if (getQueue().isEmpty()) { - starvationWatcher.onEmptyQueue(); - } else { - starvationWatcher.onNonemptyQueue(); - } } public void execute(Runnable command, final TimeValue timeout, final Runnable timeoutCallback) { @@ -298,31 +287,4 @@ public int compareTo(PrioritizedFutureTask pft) { } } - /** - * We expect the work queue to be empty fairly frequently; if the queue remains nonempty for sufficiently long then there's a risk that - * some lower-priority tasks are being starved of access to the executor. Implementations of this interface are notified whether the - * work queue is empty or not before and after execution of each task, so that we can warn the user of this possible starvation. - */ - public interface StarvationWatcher { - - /** - * Called before and after the execution of each task if the queue is empty (excluding the task being executed) - */ - void onEmptyQueue(); - - /** - * Called after the execution of each task if the queue is nonempty (excluding the task being executed) - */ - void onNonemptyQueue(); - - StarvationWatcher NOOP_STARVATION_WATCHER = new StarvationWatcher() { - @Override - public void onEmptyQueue() {} - - @Override - public void onNonemptyQueue() {} - }; - - } - } diff --git a/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadataService.java b/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadataService.java index 8f87a5826b9c6..970575393f23f 100644 --- a/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadataService.java +++ b/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadataService.java @@ -15,13 +15,12 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.NamedDiff; import org.elasticsearch.cluster.SimpleBatchedExecutor; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.ClusterSettings; @@ -49,7 +48,7 @@ public class HealthMetadataService { private final ClusterService clusterService; private final ClusterStateListener clusterStateListener; private final Settings settings; - private final ClusterStateTaskExecutor executor = new UpsertHealthMetadataTask.Executor(); + private final MasterServiceTaskQueue taskQueue; private volatile boolean enabled; // Signifies that a node has been elected as master, but it was not able yet to publish its health metadata for @@ -64,6 +63,7 @@ private HealthMetadataService(ClusterService clusterService, Settings settings) this.settings = settings; this.clusterStateListener = this::updateOnClusterStateChange; this.enabled = ENABLED_SETTING.get(settings); + this.taskQueue = clusterService.getTaskQueue("health metadata service", Priority.NORMAL, new UpsertHealthMetadataTask.Executor()); } public static HealthMetadataService create(ClusterService clusterService, Settings settings) { @@ -148,16 +148,13 @@ private void updateOnSettingsUpdated(String setting, String value) { ClusterState clusterState = clusterService.state(); if (clusterState.nodesIfRecovered().getMinNodeVersion().onOrAfter(Version.V_8_5_0)) { var task = new UpdateHealthMetadata(setting, value); - var config = ClusterStateTaskConfig.build(Priority.NORMAL); - clusterService.submitStateUpdateTask("health-metadata-update", task, config, executor); + taskQueue.submitTask("health-metadata-update", task, null); } } } private void resetHealthMetadata(String source) { - var task = new InsertHealthMetadata(settings); - var config = ClusterStateTaskConfig.build(Priority.NORMAL); - clusterService.submitStateUpdateTask(source, task, config, executor); + taskQueue.submitTask(source, new InsertHealthMetadata(settings), null); } public static List getNamedWriteables() { diff --git a/server/src/main/java/org/elasticsearch/ingest/IngestService.java b/server/src/main/java/org/elasticsearch/ingest/IngestService.java index 3362aae1b513a..3a23ed0e98ddd 100644 --- a/server/src/main/java/org/elasticsearch/ingest/IngestService.java +++ b/server/src/main/java/org/elasticsearch/ingest/IngestService.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateApplier; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.metadata.DataStream.TimestampField; @@ -39,6 +38,7 @@ import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.regex.Regex; @@ -93,6 +93,7 @@ public class IngestService implements ClusterStateApplier, ReportingService taskQueue; private final ClusterService clusterService; private final ScriptService scriptService; private final Map processorFactories; @@ -176,6 +177,7 @@ public IngestService( ); this.threadPool = threadPool; + this.taskQueue = clusterService.getTaskQueue("ingest-pipelines", Priority.NORMAL, PIPELINE_TASK_EXECUTOR); } private static Map processorFactories(List ingestPlugins, Processor.Parameters parameters) { @@ -321,11 +323,10 @@ public ScriptService getScriptService() { * Deletes the pipeline specified by id in the request. */ public void delete(DeletePipelineRequest request, ActionListener listener) { - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "delete-pipeline-" + request.getId(), new DeletePipelineClusterStateUpdateTask(listener, request), - ClusterStateTaskConfig.build(Priority.NORMAL, request.masterNodeTimeout()), - PIPELINE_TASK_EXECUTOR + request.masterNodeTimeout() ); } @@ -472,11 +473,10 @@ public void putPipeline( nodeInfoListener.accept(ActionListener.wrap(nodeInfos -> { validatePipelineRequest(request, nodeInfos); - clusterService.submitStateUpdateTask( + taskQueue.submitTask( "put-pipeline-" + request.getId(), new PutPipelineClusterStateUpdateTask(listener, request), - ClusterStateTaskConfig.build(Priority.NORMAL, request.masterNodeTimeout()), - PIPELINE_TASK_EXECUTOR + request.masterNodeTimeout() ); }, listener::onFailure)); } diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java index 02c5f626cb9a1..d7a8c6cccd247 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksClusterService.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractAsyncTask; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.core.TimeValue; import org.elasticsearch.persistent.PersistentTasksCustomMetadata.Assignment; @@ -395,7 +396,12 @@ public void onFailure(Exception e) { // There must be a task that's worth rechecking because there was one // that caused this method to be called and the method failed to assign it, // but only do this if the node is still the master - periodicRechecker.rescheduleIfNecessary(); + try { + periodicRechecker.rescheduleIfNecessary(); + } catch (Exception e2) { + assert e2 instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown() : e2; + logger.warn("failed to reschedule persistent tasks rechecker", e2); + } } } diff --git a/server/src/main/java/org/elasticsearch/reservedstate/service/ReservedClusterStateService.java b/server/src/main/java/org/elasticsearch/reservedstate/service/ReservedClusterStateService.java index 0a3a9f447d0e9..845cdbfe7c0d1 100644 --- a/server/src/main/java/org/elasticsearch/reservedstate/service/ReservedClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/reservedstate/service/ReservedClusterStateService.java @@ -14,10 +14,10 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.metadata.ReservedStateErrorMetadata; import org.elasticsearch.cluster.metadata.ReservedStateMetadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.core.Tuple; import org.elasticsearch.reservedstate.NonStateTransformResult; @@ -60,8 +60,8 @@ public class ReservedClusterStateService { final Map> handlers; final ClusterService clusterService; - private final ReservedStateUpdateTaskExecutor updateStateTaskExecutor; - private final ReservedStateErrorTaskExecutor errorStateTaskExecutor; + private final MasterServiceTaskQueue updateTaskQueue; + private final MasterServiceTaskQueue errorTaskQueue; @SuppressWarnings("unchecked") private final ConstructingObjectParser stateChunkParser = new ConstructingObjectParser<>( @@ -84,8 +84,12 @@ public class ReservedClusterStateService { */ public ReservedClusterStateService(ClusterService clusterService, List> handlerList) { this.clusterService = clusterService; - this.updateStateTaskExecutor = new ReservedStateUpdateTaskExecutor(clusterService.getRerouteService()); - this.errorStateTaskExecutor = new ReservedStateErrorTaskExecutor(); + this.updateTaskQueue = clusterService.getTaskQueue( + "reserved state update", + Priority.URGENT, + new ReservedStateUpdateTaskExecutor(clusterService.getRerouteService()) + ); + this.errorTaskQueue = clusterService.getTaskQueue("reserved state error", Priority.URGENT, new ReservedStateErrorTaskExecutor()); this.handlers = handlerList.stream().collect(Collectors.toMap(ReservedClusterStateHandler::name, Function.identity())); stateChunkParser.declareNamedObjects(ConstructingObjectParser.constructorArg(), (p, c, name) -> { if (handlers.containsKey(name) == false) { @@ -199,7 +203,7 @@ public void onResponse(Collection nonStateTransformResu // Once all of the non-state transformation results complete, we can proceed to // do the final save of the cluster state. The non-state transformation reserved keys are applied // to the reserved state after all other key handlers. - clusterService.submitStateUpdateTask( + updateTaskQueue.submitTask( "reserved cluster state [" + namespace + "]", new ReservedStateUpdateTask( namespace, @@ -227,8 +231,7 @@ public void onFailure(Exception e) { } } ), - ClusterStateTaskConfig.build(Priority.URGENT), - updateStateTaskExecutor + null ); } @@ -273,7 +276,7 @@ void updateErrorState(ErrorState errorState) { } private void submitErrorUpdateTask(ErrorState errorState) { - clusterService.submitStateUpdateTask( + errorTaskQueue.submitTask( "reserved cluster state update error for [ " + errorState.namespace() + "]", new ReservedStateErrorTask(errorState, new ActionListener<>() { @Override @@ -286,8 +289,7 @@ public void onFailure(Exception e) { logger.error("Failed to apply reserved error cluster state", e); } }), - ClusterStateTaskConfig.build(Priority.URGENT), - errorStateTaskExecutor + null ); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 9d8ddd451290e..21bf0d196f333 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateApplier; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -55,6 +54,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; @@ -177,6 +177,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private final SystemIndices systemIndices; + private final MasterServiceTaskQueue masterServiceTaskQueue; + /** * Setting that specifies the maximum number of allowed concurrent snapshot create and delete operations in the * cluster state. The number of concurrent operations in a cluster state is defined as the sum of @@ -223,6 +225,8 @@ public SnapshotsService( .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); } this.systemIndices = systemIndices; + + this.masterServiceTaskQueue = clusterService.getTaskQueue("snapshots-service", Priority.NORMAL, SHARD_STATE_EXECUTOR); } /** @@ -3450,12 +3454,7 @@ private void innerUpdateSnapshotState( } })); logger.trace("received updated snapshot restore state [{}]", update); - clusterService.submitStateUpdateTask( - "update snapshot state", - update, - ClusterStateTaskConfig.build(Priority.NORMAL), - SHARD_STATE_EXECUTOR - ); + masterServiceTaskQueue.submitTask("update snapshot state", update, null); } private void startExecutableClones(SnapshotsInProgress snapshotsInProgress, @Nullable String repoName) { diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 5bf0b302991b5..e940c32cbba0b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -1899,7 +1899,6 @@ public void testSingleNodeDiscoveryWithQuorum() { Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), DiscoveryModule.SINGLE_NODE_DISCOVERY_TYPE).build() ) ) { - cluster.runRandomly(); cluster.stabilise(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java index 435fbc46f94b8..1bd82c2660e8c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java @@ -13,6 +13,8 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.core.TimeValue; @@ -31,7 +33,6 @@ import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Optional; @@ -53,21 +54,23 @@ public void testJoinDeduplication() { DeterministicTaskQueue deterministicTaskQueue = new DeterministicTaskQueue(); CapturingTransport capturingTransport = new HandshakingCapturingTransport(); DiscoveryNode localNode = new DiscoveryNode("node0", buildNewFakeTransportAddress(), Version.CURRENT); - final ThreadPool threadPool = deterministicTaskQueue.getThreadPool(); + final var threadPool = deterministicTaskQueue.getThreadPool(); + final var clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + final var taskManger = new TaskManager(Settings.EMPTY, threadPool, Set.of()); TransportService transportService = new TransportService( Settings.EMPTY, capturingTransport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> localNode, - null, + clusterSettings, new ClusterConnectionManager(Settings.EMPTY, capturingTransport, threadPool.getThreadContext()), - new TaskManager(Settings.EMPTY, threadPool, Set.of()), + taskManger, Tracer.NOOP ); JoinHelper joinHelper = new JoinHelper( null, - null, + new MasterService(Settings.EMPTY, clusterSettings, threadPool, taskManger), new NoOpClusterApplier(), transportService, () -> 0L, @@ -212,18 +215,24 @@ public void testJoinFailureOnUnhealthyNodes() { DeterministicTaskQueue deterministicTaskQueue = new DeterministicTaskQueue(); CapturingTransport capturingTransport = new HandshakingCapturingTransport(); DiscoveryNode localNode = new DiscoveryNode("node0", buildNewFakeTransportAddress(), Version.CURRENT); - TransportService transportService = capturingTransport.createTransportService( + ThreadPool threadPool = deterministicTaskQueue.getThreadPool(); + final var clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + final var taskManger = new TaskManager(Settings.EMPTY, threadPool, Set.of()); + TransportService transportService = new TransportService( Settings.EMPTY, - deterministicTaskQueue.getThreadPool(), + capturingTransport, + threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> localNode, - null, - Collections.emptySet() + clusterSettings, + new ClusterConnectionManager(Settings.EMPTY, capturingTransport, threadPool.getThreadContext()), + taskManger, + Tracer.NOOP ); AtomicReference nodeHealthServiceStatus = new AtomicReference<>(new StatusInfo(UNHEALTHY, "unhealthy-info")); JoinHelper joinHelper = new JoinHelper( null, - null, + new MasterService(Settings.EMPTY, clusterSettings, threadPool, taskManger), new NoOpClusterApplier(), transportService, () -> 0L, diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTransportHandlerTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTransportHandlerTests.java index 52cc92f73c749..de5692f62dc86 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTransportHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTransportHandlerTests.java @@ -106,7 +106,7 @@ public void writeTo(StreamOutput out) throws IOException { ElasticsearchException.class, () -> handler.newPublicationContext( new ClusterStatePublicationEvent( - new BatchSummary("test"), + new BatchSummary(() -> "test"), clusterState, unserializableClusterState, new Task(randomNonNegativeLong(), "test", STATE_UPDATE_ACTION_NAME, "", TaskId.EMPTY_TASK_ID, emptyMap()), @@ -288,7 +288,7 @@ public void writeTo(StreamOutput out) throws IOException { try { context = handler.newPublicationContext( new ClusterStatePublicationEvent( - new BatchSummary("test"), + new BatchSummary(() -> "test"), prevClusterState, nextClusterState, new Task(randomNonNegativeLong(), "test", STATE_UPDATE_ACTION_NAME, "", TaskId.EMPTY_TASK_ID, emptyMap()), diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java index e26a970d3e1a7..8720ef05122fa 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterStateTaskExecutorUtils; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; @@ -25,6 +26,7 @@ import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInProgressException; import org.elasticsearch.snapshots.SnapshotInfoTestUtils; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; import org.junit.Before; @@ -60,7 +62,11 @@ public void setUp() throws Exception { when(allocationService.reroute(any(ClusterState.class), any(String.class), any())).thenAnswer( mockInvocation -> mockInvocation.getArguments()[0] ); - service = new MetadataDeleteIndexService(Settings.EMPTY, null, allocationService); + service = new MetadataDeleteIndexService( + Settings.EMPTY, + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), + allocationService + ); } public void testDeleteMissing() { diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexStateServiceBatchingTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexStateServiceBatchingTests.java index bd2eb6c6202ed..f395c87846bfd 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexStateServiceBatchingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexStateServiceBatchingTests.java @@ -9,7 +9,6 @@ package org.elasticsearch.cluster.metadata; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock; import org.elasticsearch.cluster.metadata.IndexMetadata.State; @@ -199,20 +198,14 @@ public void testBatchBlockIndices() throws Exception { private static CheckedRunnable blockMasterService(MasterService masterService) { final var executionBarrier = new CyclicBarrier(2); - masterService.submitStateUpdateTask( - "block", - new ExpectSuccessTask(), - ClusterStateTaskConfig.build(Priority.URGENT), - batchExecutionContext -> { - executionBarrier.await(10, TimeUnit.SECONDS); // notify test thread that the master service is blocked - executionBarrier.await(10, TimeUnit.SECONDS); // wait for test thread to release us - for (final var taskContext : batchExecutionContext.taskContexts()) { - taskContext.success(() -> {}); - } - return batchExecutionContext.initialState(); + masterService.getTaskQueue("block", Priority.URGENT, batchExecutionContext -> { + executionBarrier.await(10, TimeUnit.SECONDS); // notify test thread that the master service is blocked + executionBarrier.await(10, TimeUnit.SECONDS); // wait for test thread to release us + for (final var taskContext : batchExecutionContext.taskContexts()) { + taskContext.success(() -> {}); } - ); - + return batchExecutionContext.initialState(); + }).submitTask("block", new ExpectSuccessTask(), null); return () -> executionBarrier.await(10, TimeUnit.SECONDS); } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java index 9702810e3afc6..4395163311ddd 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java @@ -2126,7 +2126,7 @@ private static List putTemplate(NamedXContentRegistry xContentRegistr new IndexSettingProviders(Set.of()) ); MetadataIndexTemplateService service = new MetadataIndexTemplateService( - null, + clusterService, createIndexService, null, new IndexScopedSettings(Settings.EMPTY, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS), diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataMappingServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataMappingServiceTests.java index ec7004e6faa22..f8947131dcb24 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataMappingServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataMappingServiceTests.java @@ -38,6 +38,7 @@ public void testMappingClusterStateUpdateDoesntChangeExistingIndices() throws Ex final CompressedXContent currentMapping = indexService.mapperService().documentMapper().mappingSource(); final MetadataMappingService mappingService = getInstanceFromNode(MetadataMappingService.class); + final MetadataMappingService.PutMappingExecutor putMappingExecutor = mappingService.new PutMappingExecutor(); final ClusterService clusterService = getInstanceFromNode(ClusterService.class); // TODO - it will be nice to get a random mapping generator final PutMappingClusterStateUpdateRequest request = new PutMappingClusterStateUpdateRequest(""" @@ -45,7 +46,7 @@ public void testMappingClusterStateUpdateDoesntChangeExistingIndices() throws Ex request.indices(new Index[] { indexService.index() }); final var resultingState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( clusterService.state(), - mappingService.putMappingExecutor, + putMappingExecutor, singleTask(request) ); // the task really was a mapping update @@ -61,17 +62,18 @@ public void testClusterStateIsNotChangedWithIdenticalMappings() throws Exception final IndexService indexService = createIndex("test", client().admin().indices().prepareCreate("test")); final MetadataMappingService mappingService = getInstanceFromNode(MetadataMappingService.class); + final MetadataMappingService.PutMappingExecutor putMappingExecutor = mappingService.new PutMappingExecutor(); final ClusterService clusterService = getInstanceFromNode(ClusterService.class); final PutMappingClusterStateUpdateRequest request = new PutMappingClusterStateUpdateRequest(""" { "properties": { "field": { "type": "text" }}}""").indices(new Index[] { indexService.index() }); final var resultingState1 = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( clusterService.state(), - mappingService.putMappingExecutor, + putMappingExecutor, singleTask(request) ); final var resultingState2 = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( resultingState1, - mappingService.putMappingExecutor, + putMappingExecutor, singleTask(request) ); assertSame(resultingState1, resultingState2); @@ -81,13 +83,14 @@ public void testMappingVersion() throws Exception { final IndexService indexService = createIndex("test", client().admin().indices().prepareCreate("test")); final long previousVersion = indexService.getMetadata().getMappingVersion(); final MetadataMappingService mappingService = getInstanceFromNode(MetadataMappingService.class); + final MetadataMappingService.PutMappingExecutor putMappingExecutor = mappingService.new PutMappingExecutor(); final ClusterService clusterService = getInstanceFromNode(ClusterService.class); final PutMappingClusterStateUpdateRequest request = new PutMappingClusterStateUpdateRequest(""" { "properties": { "field": { "type": "text" }}}"""); request.indices(new Index[] { indexService.index() }); final var resultingState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( clusterService.state(), - mappingService.putMappingExecutor, + putMappingExecutor, singleTask(request) ); assertThat(resultingState.metadata().index("test").getMappingVersion(), equalTo(1 + previousVersion)); @@ -97,12 +100,13 @@ public void testMappingVersionUnchanged() throws Exception { final IndexService indexService = createIndex("test", client().admin().indices().prepareCreate("test").setMapping()); final long previousVersion = indexService.getMetadata().getMappingVersion(); final MetadataMappingService mappingService = getInstanceFromNode(MetadataMappingService.class); + final MetadataMappingService.PutMappingExecutor putMappingExecutor = mappingService.new PutMappingExecutor(); final ClusterService clusterService = getInstanceFromNode(ClusterService.class); final PutMappingClusterStateUpdateRequest request = new PutMappingClusterStateUpdateRequest("{ \"properties\": {}}"); request.indices(new Index[] { indexService.index() }); final var resultingState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( clusterService.state(), - mappingService.putMappingExecutor, + putMappingExecutor, singleTask(request) ); assertThat(resultingState.metadata().index("test").getMappingVersion(), equalTo(previousVersion)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterAllocationSimulationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterAllocationSimulationTests.java index 98c068d874d9f..772a3c6b7ac3a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterAllocationSimulationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterAllocationSimulationTests.java @@ -222,8 +222,7 @@ public void testBalanceQuality() throws IOException { TimeUnit.SECONDS, r -> { throw new AssertionError("should not create new threads"); }, null, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ) { @Override @@ -266,6 +265,7 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { applierService.setInitialState(unassignedClusterState); final var clusterService = new ClusterService(settings, clusterSettings, masterService, applierService); + clusterService.start(); final var clusterInfoService = new TestClusterInfoService(clusterService); diff --git a/server/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java index 6ed1436327dd5..7f0e1fbbc5179 100644 --- a/server/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/service/MasterServiceTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -31,13 +30,19 @@ import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.coordination.ClusterStatePublisher; import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; +import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.BaseFuture; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.core.TimeValue; @@ -74,12 +79,16 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.elasticsearch.cluster.service.MasterService.MAX_TASK_DESCRIPTION_CHARS; +import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.lessThanOrEqualTo; public class MasterServiceTests extends ESTestCase { @@ -119,6 +128,15 @@ private MasterService createMasterService(boolean makeMaster) { } private MasterService createMasterService(boolean makeMaster, TaskManager taskManager) { + return createMasterService(makeMaster, taskManager, threadPool, null); + } + + private MasterService createMasterService( + boolean makeMaster, + TaskManager taskManager, + ThreadPool threadPool, + PrioritizedEsThreadPoolExecutor threadPoolExecutor + ) { final DiscoveryNode localNode = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); final Settings settings = Settings.builder() .put(ClusterName.CLUSTER_NAME_SETTING.getKey(), MasterServiceTests.class.getSimpleName()) @@ -134,7 +152,16 @@ private MasterService createMasterService(boolean makeMaster, TaskManager taskMa new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool, taskManager - ); + ) { + @Override + protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { + if (threadPoolExecutor == null) { + return super.createThreadPoolExecutor(); + } else { + return threadPoolExecutor; + } + } + }; final ClusterState initialClusterState = ClusterState.builder(new ClusterName(MasterServiceTests.class.getSimpleName())) .nodes( DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(makeMaster ? localNode.getId() : null) @@ -208,25 +235,20 @@ public void waitForTaskCompletion(Task task) {} final CountDownLatch latch = new CountDownLatch(1); try (MasterService masterService = createMasterService(true, taskManager)) { - masterService.submitStateUpdateTask( - "testCreatesChildTaskForPublishingClusterState", - new ExpectSuccessTask(), - ClusterStateTaskConfig.build(Priority.NORMAL), - new ClusterStateTaskExecutor<>() { - @Override - public ClusterState execute(BatchExecutionContext batchExecutionContext) { - for (final var taskContext : batchExecutionContext.taskContexts()) { - taskContext.success(() -> {}); - } - return ClusterState.builder(batchExecutionContext.initialState()).build(); + masterService.getTaskQueue("test", Priority.NORMAL, new ClusterStateTaskExecutor<>() { + @Override + public ClusterState execute(BatchExecutionContext batchExecutionContext) { + for (final var taskContext : batchExecutionContext.taskContexts()) { + taskContext.success(() -> {}); } + return ClusterState.builder(batchExecutionContext.initialState()).build(); + } - @Override - public void clusterStatePublished(ClusterState newClusterState) { - latch.countDown(); - } + @Override + public void clusterStatePublished(ClusterState newClusterState) { + latch.countDown(); } - ); + }).submitTask("testCreatesChildTaskForPublishingClusterState", new ExpectSuccessTask(), null); assertTrue(latch.await(10, TimeUnit.SECONDS)); } @@ -235,87 +257,89 @@ public void clusterStatePublished(ClusterState newClusterState) { } public void testThreadContext() throws InterruptedException { - final MasterService master = createMasterService(true); - final CountDownLatch latch = new CountDownLatch(1); - - try (ThreadContext.StoredContext ignored = threadPool.getThreadContext().stashContext()) { - final Map expectedHeaders = Collections.singletonMap("test", "test"); - final Map> expectedResponseHeaders = Collections.singletonMap( - "testResponse", - Collections.singletonList("testResponse") - ); - threadPool.getThreadContext().putHeader(expectedHeaders); + try (var master = createMasterService(true)) { + final CountDownLatch latch = new CountDownLatch(1); + + try (ThreadContext.StoredContext ignored = threadPool.getThreadContext().stashContext()) { + final Map expectedHeaders = Collections.singletonMap("test", "test"); + final Map> expectedResponseHeaders = Collections.singletonMap( + "testResponse", + Collections.singletonList("testResponse") + ); + threadPool.getThreadContext().putHeader(expectedHeaders); - final TimeValue ackTimeout = randomBoolean() ? TimeValue.ZERO : TimeValue.timeValueMillis(randomInt(10000)); - final TimeValue masterTimeout = randomBoolean() ? TimeValue.ZERO : TimeValue.timeValueMillis(randomInt(10000)); + final TimeValue ackTimeout = randomBoolean() ? TimeValue.ZERO : TimeValue.timeValueMillis(randomInt(10000)); + final TimeValue masterTimeout = randomBoolean() ? TimeValue.ZERO : TimeValue.timeValueMillis(randomInt(10000)); - master.submitUnbatchedStateUpdateTask("test", new AckedClusterStateUpdateTask(ackedRequest(ackTimeout, masterTimeout), null) { - @Override - public ClusterState execute(ClusterState currentState) { - assertTrue(threadPool.getThreadContext().isSystemContext()); - assertEquals(Collections.emptyMap(), threadPool.getThreadContext().getHeaders()); - threadPool.getThreadContext().addResponseHeader("testResponse", "testResponse"); - assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + master.submitUnbatchedStateUpdateTask( + "test", + new AckedClusterStateUpdateTask(ackedRequest(ackTimeout, masterTimeout), null) { + @Override + public ClusterState execute(ClusterState currentState) { + assertTrue(threadPool.getThreadContext().isSystemContext()); + assertEquals(Collections.emptyMap(), threadPool.getThreadContext().getHeaders()); + threadPool.getThreadContext().addResponseHeader("testResponse", "testResponse"); + assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + + if (randomBoolean()) { + return ClusterState.builder(currentState).build(); + } else if (randomBoolean()) { + return currentState; + } else { + throw new IllegalArgumentException("mock failure"); + } + } - if (randomBoolean()) { - return ClusterState.builder(currentState).build(); - } else if (randomBoolean()) { - return currentState; - } else { - throw new IllegalArgumentException("mock failure"); - } - } + @Override + public void onFailure(Exception e) { + assertFalse(threadPool.getThreadContext().isSystemContext()); + assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); + assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + latch.countDown(); + } - @Override - public void onFailure(Exception e) { - assertFalse(threadPool.getThreadContext().isSystemContext()); - assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); - assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); - latch.countDown(); - } + @Override + public void clusterStateProcessed(ClusterState oldState, ClusterState newState) { + assertFalse(threadPool.getThreadContext().isSystemContext()); + assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); + assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + latch.countDown(); + } - @Override - public void clusterStateProcessed(ClusterState oldState, ClusterState newState) { - assertFalse(threadPool.getThreadContext().isSystemContext()); - assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); - assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); - latch.countDown(); - } + @Override + public void onAllNodesAcked() { + assertFalse(threadPool.getThreadContext().isSystemContext()); + assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); + assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + latch.countDown(); + } - @Override - public void onAllNodesAcked() { - assertFalse(threadPool.getThreadContext().isSystemContext()); - assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); - assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); - latch.countDown(); - } + @Override + public void onAckFailure(Exception e) { + assertFalse(threadPool.getThreadContext().isSystemContext()); + assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); + assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + latch.countDown(); + } - @Override - public void onAckFailure(Exception e) { - assertFalse(threadPool.getThreadContext().isSystemContext()); - assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); - assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); - latch.countDown(); - } + @Override + public void onAckTimeout() { + assertFalse(threadPool.getThreadContext().isSystemContext()); + assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); + assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); + latch.countDown(); + } - @Override - public void onAckTimeout() { - assertFalse(threadPool.getThreadContext().isSystemContext()); - assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); - assertEquals(expectedResponseHeaders, threadPool.getThreadContext().getResponseHeaders()); - latch.countDown(); - } + } + ); - }); + assertFalse(threadPool.getThreadContext().isSystemContext()); + assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); + assertEquals(Collections.emptyMap(), threadPool.getThreadContext().getResponseHeaders()); + } - assertFalse(threadPool.getThreadContext().isSystemContext()); - assertEquals(expectedHeaders, threadPool.getThreadContext().getHeaders()); - assertEquals(Collections.emptyMap(), threadPool.getThreadContext().getResponseHeaders()); + assertTrue(latch.await(10, TimeUnit.SECONDS)); } - - latch.await(); - - master.close(); } /* @@ -327,11 +351,10 @@ public void testClusterStateTaskListenerThrowingExceptionIsOkay() throws Interru final CountDownLatch latch = new CountDownLatch(1); try (MasterService masterService = createMasterService(true)) { - masterService.submitStateUpdateTask( + masterService.getTaskQueue( "testClusterStateTaskListenerThrowingExceptionIsOkay", - new ExpectSuccessTask(), - ClusterStateTaskConfig.build(Priority.NORMAL), - new ClusterStateTaskExecutor<>() { + Priority.NORMAL, + new ClusterStateTaskExecutor() { @Override public ClusterState execute(BatchExecutionContext batchExecutionContext) { for (final var taskContext : batchExecutionContext.taskContexts()) { @@ -345,8 +368,7 @@ public void clusterStatePublished(ClusterState newClusterState) { latch.countDown(); } } - ); - + ).submitTask("testClusterStateTaskListenerThrowingExceptionIsOkay", new ExpectSuccessTask(), null); assertTrue(latch.await(10, TimeUnit.SECONDS)); } } @@ -545,28 +567,29 @@ public ClusterState execute(BatchExecutionContext batchExecut } } - final var executors = new Executor[executorCount]; - for (int i = 0; i < executors.length; i++) { - executors[i] = new Executor(); - } + record QueueAndExecutor(MasterServiceTaskQueue queue, Executor executor) {} try (var masterService = createMasterService(true)) { + final var executors = new QueueAndExecutor[executorCount]; + for (int i = 0; i < executors.length; i++) { + final var executor = new Executor(); + executors[i] = new QueueAndExecutor( + masterService.getTaskQueue("executor-" + i, randomFrom(Priority.values()), executor), + executor + ); + } + final var executionBarrier = new CyclicBarrier(2); - masterService.submitStateUpdateTask( - "block", - new ExpectSuccessTask(), - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - executionBarrier.await(10, TimeUnit.SECONDS); // notify test thread that the master service is blocked - executionBarrier.await(10, TimeUnit.SECONDS); // wait for test thread to release us - for (final var taskContext : batchExecutionContext.taskContexts()) { - taskContext.success(() -> {}); - } - return batchExecutionContext.initialState(); + masterService.getTaskQueue("block", Priority.NORMAL, batchExecutionContext -> { + executionBarrier.await(10, TimeUnit.SECONDS); // notify test thread that the master service is blocked + executionBarrier.await(10, TimeUnit.SECONDS); // wait for test thread to release us + for (final var taskContext : batchExecutionContext.taskContexts()) { + taskContext.success(() -> {}); } - ); + return batchExecutionContext.initialState(); + }).submitTask("block", new ExpectSuccessTask(), null); executionBarrier.await(10, TimeUnit.SECONDS); // wait for the master service to be blocked @@ -576,25 +599,19 @@ public ClusterState execute(BatchExecutionContext batchExecut for (int i = 0; i < submitThreads.length; i++) { final var executor = randomFrom(executors); final var task = new ExpectSuccessTask(); - executor.addExpectedTaskCount(1); + executor.executor().addExpectedTaskCount(1); submitThreads[i] = new Thread(() -> { try { assertTrue(submissionLatch.await(10, TimeUnit.SECONDS)); - masterService.submitStateUpdateTask( - Thread.currentThread().getName(), - task, - ClusterStateTaskConfig.build(randomFrom(Priority.values())), - executor - ); + executor.queue().submitTask(Thread.currentThread().getName(), task, null); } catch (InterruptedException e) { throw new AssertionError(e); } - }, "submit-thread-" + i); } for (var executor : executors) { - if (executor.expectedTaskCount == 0) { + if (executor.executor().expectedTaskCount == 0) { executionCountDown.countDown(); } } @@ -610,7 +627,7 @@ public ClusterState execute(BatchExecutionContext batchExecut } for (var executor : executors) { - assertFalse(executor.executed.get()); + assertFalse(executor.executor().executed.get()); } assertThat(masterService.numberOfPendingTasks(), equalTo(submitThreads.length + 1)); @@ -624,7 +641,7 @@ public ClusterState execute(BatchExecutionContext batchExecut executionBarrier.await(10, TimeUnit.SECONDS); // release block on master service assertTrue(executionCountDown.await(10, TimeUnit.SECONDS)); for (var executor : executors) { - assertTrue(executor.executed.get() != (executor.expectedTaskCount == 0)); + assertTrue(executor.executor().executed.get() != (executor.executor().expectedTaskCount == 0)); } } } @@ -750,28 +767,33 @@ public void clusterStatePublished(ClusterState newClusterState) { } } - List executors = new ArrayList<>(); - for (int i = 0; i < numberOfExecutors; i++) { - executors.add(new TaskExecutor()); - } + record QueueAndExecutor(MasterServiceTaskQueue queue, TaskExecutor executor) {} + + try (var masterService = createMasterService(true)) { + final var executors = new ArrayList(); + for (int i = 0; i < numberOfExecutors; i++) { + final var executor = new TaskExecutor(); + executors.add( + new QueueAndExecutor(masterService.getTaskQueue("queue-" + i, randomFrom(Priority.values()), executor), executor) + ); + } - // randomly assign tasks to executors - List> assignments = new ArrayList<>(); - AtomicInteger totalTasks = new AtomicInteger(); - for (int i = 0; i < numberOfThreads; i++) { - for (int j = 0; j < taskSubmissionsPerThread; j++) { - var executor = randomFrom(executors); - var task = new Task(totalTasks.getAndIncrement()); + // randomly assign tasks to queues + List, Task>> assignments = new ArrayList<>(); + AtomicInteger totalTasks = new AtomicInteger(); + for (int i = 0; i < numberOfThreads; i++) { + for (int j = 0; j < taskSubmissionsPerThread; j++) { + var executor = randomFrom(executors); + var task = new Task(totalTasks.getAndIncrement()); - assignments.add(Tuple.tuple(executor, task)); - executor.assigned.incrementAndGet(); - executor.assignments.add(task); + assignments.add(Tuple.tuple(executor.queue(), task)); + executor.executor().assigned.incrementAndGet(); + executor.executor().assignments.add(task); + } } - } - processedStatesLatch.set(new CountDownLatch(totalTasks.get())); + processedStatesLatch.set(new CountDownLatch(totalTasks.get())); - try (MasterService masterService = createMasterService(true)) { - CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); + final var barrier = new CyclicBarrier(1 + numberOfThreads); for (int i = 0; i < numberOfThreads; i++) { final int index = i; Thread thread = new Thread(() -> { @@ -783,12 +805,7 @@ public void clusterStatePublished(ClusterState newClusterState) { var task = assignment.v2(); var executor = assignment.v1(); submittedTasks.incrementAndGet(); - masterService.submitStateUpdateTask( - threadName, - task, - ClusterStateTaskConfig.build(randomFrom(Priority.values())), - executor - ); + executor.submitTask(threadName, task, null); } barrier.await(); } catch (BrokenBarrierException | InterruptedException e) { @@ -804,9 +821,9 @@ public void clusterStatePublished(ClusterState newClusterState) { barrier.await(); // wait until all the cluster state updates have been processed - processedStatesLatch.get().await(); + assertTrue(processedStatesLatch.get().await(10, TimeUnit.SECONDS)); // and until all the publication callbacks have completed - semaphore.acquire(); + assertTrue(semaphore.tryAcquire(10, TimeUnit.SECONDS)); // assert the number of executed tasks is correct assertThat(submittedTasks.get(), equalTo(totalTasks.get())); @@ -814,9 +831,9 @@ public void clusterStatePublished(ClusterState newClusterState) { assertThat(processedStates.get(), equalTo(totalTasks.get())); // assert each executor executed the correct number of tasks - for (TaskExecutor executor : executors) { - assertEquals(executor.assigned.get(), executor.executed.get()); - assertEquals(executor.batches.get(), executor.published.get()); + for (var executor : executors) { + assertEquals(executor.executor().assigned.get(), executor.executor().executed.get()); + assertEquals(executor.executor().batches.get(), executor.executor().published.get()); } } } @@ -872,6 +889,7 @@ public void onFailure(Exception e) { }; try (var masterService = createMasterService(true)) { + final var queue = masterService.getTaskQueue("test", Priority.NORMAL, executor); masterService.submitUnbatchedStateUpdateTask("block", blockMasterTask); executionBarrier.await(10, TimeUnit.SECONDS); // wait for the master service to be blocked @@ -886,9 +904,7 @@ public void onFailure(Exception e) { try (ThreadContext.StoredContext ignored = threadContext.newStoredContext()) { final String testContextHeaderValue = randomAlphaOfLength(10); threadContext.putHeader(testContextHeaderName, testContextHeaderValue); - final var task = new Task(testContextHeaderValue); - final var clusterStateTaskConfig = ClusterStateTaskConfig.build(Priority.NORMAL); - masterService.submitStateUpdateTask("test", task, clusterStateTaskConfig, executor); + queue.submitTask("test", new Task(testContextHeaderValue), null); } } @@ -951,6 +967,8 @@ public void onFailure(Exception e) { try (var masterService = createMasterService(true)) { + final var queue = masterService.getTaskQueue("test", Priority.NORMAL, executor); + // success case: submit some tasks, possibly in different contexts, and verify that the expected listener is completed masterService.submitUnbatchedStateUpdateTask("block", blockMasterTask); @@ -986,8 +1004,7 @@ public void onFailure(Exception e) { } }); - final ClusterStateTaskConfig clusterStateTaskConfig = ClusterStateTaskConfig.build(Priority.NORMAL); - masterService.submitStateUpdateTask("test", task, clusterStateTaskConfig, executor); + queue.submitTask("test", task, null); } } @@ -1029,8 +1046,7 @@ public void onFailure(Exception e) { } }); - final ClusterStateTaskConfig clusterStateTaskConfig = ClusterStateTaskConfig.build(Priority.NORMAL); - masterService.submitStateUpdateTask("test", task, clusterStateTaskConfig, executor); + queue.submitTask("test", task, null); } } @@ -1045,32 +1061,27 @@ public void testBlockingCallInClusterStateTaskListenerFails() throws Interrupted final AtomicReference assertionRef = new AtomicReference<>(); try (MasterService masterService = createMasterService(true)) { - masterService.submitStateUpdateTask( - "testBlockingCallInClusterStateTaskListenerFails", - new ExpectSuccessTask(), - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - for (final var taskContext : batchExecutionContext.taskContexts()) { - taskContext.success(() -> { - BaseFuture future = new BaseFuture() { - }; - try { - if (randomBoolean()) { - future.get(1L, TimeUnit.SECONDS); - } else { - future.get(); - } - } catch (Exception e) { - throw new RuntimeException(e); - } catch (AssertionError e) { - assertionRef.set(e); - latch.countDown(); + masterService.getTaskQueue("testBlockingCallInClusterStateTaskListenerFails", Priority.NORMAL, batchExecutionContext -> { + for (final var taskContext : batchExecutionContext.taskContexts()) { + taskContext.success(() -> { + BaseFuture future = new BaseFuture() { + }; + try { + if (randomBoolean()) { + future.get(1L, TimeUnit.SECONDS); + } else { + future.get(); } - }); - } - return ClusterState.builder(batchExecutionContext.initialState()).build(); + } catch (Exception e) { + throw new RuntimeException(e); + } catch (AssertionError e) { + assertionRef.set(e); + latch.countDown(); + } + }); } - ); + return ClusterState.builder(batchExecutionContext.initialState()).build(); + }).submitTask("testBlockingCallInClusterStateTaskListenerFails", new ExpectSuccessTask(), null); latch.await(); assertNotNull(assertionRef.get()); @@ -1143,7 +1154,13 @@ public void testLongClusterStateUpdateLogging() throws Exception { new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool, new TaskManager(settings, threadPool, emptySet()) - ) + ) { + @Override + protected boolean publicationMayFail() { + // checking logging even during unexpected failures + return true; + } + } ) { final DiscoveryNode localNode = new DiscoveryNode( "node1", @@ -1398,29 +1415,24 @@ public void onFailure(Exception e) { } } - masterService.submitStateUpdateTask( - "success-test", - new Task(), - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - for (final var taskContext : batchExecutionContext.taskContexts()) { - final var responseHeaderValue = randomAlphaOfLength(10); - try (var ignored = taskContext.captureResponseHeaders()) { - threadPool.getThreadContext().addResponseHeader(responseHeaderName, responseHeaderValue); - } - taskContext.success(() -> { - assertThat( - threadPool.getThreadContext().getResponseHeaders().get(responseHeaderName), - equalTo(List.of(responseHeaderValue)) - ); - latch.countDown(); - }, taskContext.getTask()); + masterService.getTaskQueue("success-test", Priority.NORMAL, batchExecutionContext -> { + for (final var taskContext : batchExecutionContext.taskContexts()) { + final var responseHeaderValue = randomAlphaOfLength(10); + try (var ignored = taskContext.captureResponseHeaders()) { + threadPool.getThreadContext().addResponseHeader(responseHeaderName, responseHeaderValue); } - return randomBoolean() - ? batchExecutionContext.initialState() - : ClusterState.builder(batchExecutionContext.initialState()).build(); + taskContext.success(() -> { + assertThat( + threadPool.getThreadContext().getResponseHeaders().get(responseHeaderName), + equalTo(List.of(responseHeaderValue)) + ); + latch.countDown(); + }, taskContext.getTask()); } - ); + return randomBoolean() + ? batchExecutionContext.initialState() + : ClusterState.builder(batchExecutionContext.initialState()).build(); + }).submitTask("success-test", new Task(), null); assertTrue(latch.await(10, TimeUnit.SECONDS)); } @@ -1444,19 +1456,14 @@ public void onFailure(Exception e) { } } - masterService.submitStateUpdateTask( - "success-test", - new Task(), - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - for (final var taskContext : batchExecutionContext.taskContexts()) { - taskContext.success(latch::countDown, new LatchAckListener(latch)); - } - return randomBoolean() - ? batchExecutionContext.initialState() - : ClusterState.builder(batchExecutionContext.initialState()).build(); + masterService.getTaskQueue("success-test", Priority.NORMAL, batchExecutionContext -> { + for (final var taskContext : batchExecutionContext.taskContexts()) { + taskContext.success(latch::countDown, new LatchAckListener(latch)); } - ); + return randomBoolean() + ? batchExecutionContext.initialState() + : ClusterState.builder(batchExecutionContext.initialState()).build(); + }).submitTask("success-test", new Task(), null); assertTrue(latch.await(10, TimeUnit.SECONDS)); } @@ -1480,19 +1487,14 @@ public void onFailure(Exception e) { } } - masterService.submitStateUpdateTask( - "success-test", - new Task(), - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - for (final var taskContext : batchExecutionContext.taskContexts()) { - taskContext.success(new LatchAckListener(latch)); - } - return randomBoolean() - ? batchExecutionContext.initialState() - : ClusterState.builder(batchExecutionContext.initialState()).build(); + masterService.getTaskQueue("success-test", Priority.NORMAL, batchExecutionContext -> { + for (final var taskContext : batchExecutionContext.taskContexts()) { + taskContext.success(new LatchAckListener(latch)); } - ); + return randomBoolean() + ? batchExecutionContext.initialState() + : ClusterState.builder(batchExecutionContext.initialState()).build(); + }).submitTask("success-test", new Task(), null); assertTrue(latch.await(10, TimeUnit.SECONDS)); } @@ -1516,37 +1518,32 @@ public void onFailure(Exception e) { } } - masterService.submitStateUpdateTask( - "node-ack-fail-test", - new Task(), - ClusterStateTaskConfig.build(Priority.NORMAL), - batchExecutionContext -> { - for (final var taskContext : batchExecutionContext.taskContexts()) { - final var responseHeaderValue = randomAlphaOfLength(10); - try (var ignored = taskContext.captureResponseHeaders()) { - threadPool.getThreadContext().addResponseHeader(responseHeaderName, responseHeaderValue); - } - taskContext.success(new LatchAckListener(latch) { - @Override - public void onAllNodesAcked() { - fail(); - } - - @Override - public void onAckFailure(Exception e) { - assertThat( - threadPool.getThreadContext().getResponseHeaders().get(responseHeaderName), - equalTo(List.of(responseHeaderValue)) - ); - assertThat(e, instanceOf(ElasticsearchException.class)); - assertThat(e.getMessage(), equalTo("simulated")); - latch.countDown(); - } - }); + masterService.getTaskQueue("node-ack-fail-test", Priority.NORMAL, batchExecutionContext -> { + for (final var taskContext : batchExecutionContext.taskContexts()) { + final var responseHeaderValue = randomAlphaOfLength(10); + try (var ignored = taskContext.captureResponseHeaders()) { + threadPool.getThreadContext().addResponseHeader(responseHeaderName, responseHeaderValue); } - return ClusterState.builder(batchExecutionContext.initialState()).build(); + taskContext.success(new LatchAckListener(latch) { + @Override + public void onAllNodesAcked() { + fail(); + } + + @Override + public void onAckFailure(Exception e) { + assertThat( + threadPool.getThreadContext().getResponseHeaders().get(responseHeaderName), + equalTo(List.of(responseHeaderValue)) + ); + assertThat(e, instanceOf(ElasticsearchException.class)); + assertThat(e.getMessage(), equalTo("simulated")); + latch.countDown(); + } + }); } - ); + return ClusterState.builder(batchExecutionContext.initialState()).build(); + }).submitTask("node-ack-fail-test", new Task(), null); assertTrue(latch.await(10, TimeUnit.SECONDS)); } @@ -1698,7 +1695,7 @@ public ClusterState execute(ClusterState currentState) { @Override public void onFailure(Exception e) { - fail(); + throw new AssertionError(e); } }; masterService.submitUnbatchedStateUpdateTask("starvation-causing task", starvationCausingTask); @@ -1714,7 +1711,7 @@ public ClusterState execute(ClusterState currentState) { @Override public void onFailure(Exception e) { - fail(); + throw new AssertionError(e); } }); @@ -1835,34 +1832,26 @@ public ClusterState execute(BatchExecutionContext batchExecutionContext) { barrier.await(10, TimeUnit.SECONDS); final var smallBatchExecutor = new Executor(); + final var smallBatchQueue = masterService.getTaskQueue("small-batch", Priority.NORMAL, smallBatchExecutor); for (int source = 0; source < 2; source++) { for (int task = 0; task < 2; task++) { - masterService.submitStateUpdateTask( - "source-" + source, - new Task("task-" + task), - ClusterStateTaskConfig.build(Priority.NORMAL), - smallBatchExecutor - ); + smallBatchQueue.submitTask("source-" + source, new Task("task-" + source + "-" + task), null); } mockAppender.addExpectation( new MockLogAppender.SeenEventExpectation( "mention of tasks source-" + source, MasterService.class.getCanonicalName(), Level.DEBUG, - "executing cluster state update for [*source-" + source + "[task-0, task-1]*" + "executing cluster state update for [*source-" + source + "[task-" + source + "-0, task-" + source + "-1]*" ) ); } final var manySourceExecutor = new Executor(); + final var manySourceQueue = masterService.getTaskQueue("many-source", Priority.NORMAL, manySourceExecutor); for (int source = 0; source < 1024; source++) { for (int task = 0; task < 2; task++) { - masterService.submitStateUpdateTask( - "source-" + source, - new Task("task-" + task), - ClusterStateTaskConfig.build(Priority.NORMAL), - manySourceExecutor - ); + manySourceQueue.submitTask("source-" + source, new Task("task-" + task), null); } } mockAppender.addExpectation( @@ -1874,19 +1863,19 @@ public ClusterState execute(BatchExecutionContext batchExecutionContext) { ) { @Override public boolean innerMatch(LogEvent event) { - return event.getMessage().getFormattedMessage().length() < BatchSummary.MAX_TASK_DESCRIPTION_CHARS + 200; + return event.getMessage().getFormattedMessage().length() < MAX_TASK_DESCRIPTION_CHARS + 200; } } ); final var manyTasksPerSourceExecutor = new Executor(); + final var manyTasksPerSourceQueue = masterService.getTaskQueue( + "many-tasks-per-source", + Priority.NORMAL, + manyTasksPerSourceExecutor + ); for (int task = 0; task < 2048; task++) { - masterService.submitStateUpdateTask( - "unique-source", - new Task("task-" + task), - ClusterStateTaskConfig.build(Priority.NORMAL), - manyTasksPerSourceExecutor - ); + manyTasksPerSourceQueue.submitTask("unique-source", new Task("task-" + task), null); } mockAppender.addExpectation( new MockLogAppender.SeenEventExpectation( @@ -1913,6 +1902,437 @@ public boolean innerMatch(LogEvent event) { } } + public void testPendingTasksReporting() { + + final var deterministicTaskQueue = new DeterministicTaskQueue(); + + final var threadPool = deterministicTaskQueue.getThreadPool(); + final var threadPoolExecutor = deterministicTaskQueue.getPrioritizedEsThreadPoolExecutor(); + + try (var masterService = createMasterService(true, null, threadPool, threadPoolExecutor)) { + + final var actionCount = new AtomicInteger(); + + class BatchedTask implements ClusterStateTaskListener { + final int queueIndex; + final int taskIndex; + final Priority priority; + final long insertionTimeMillis; + final TimeValue timeout; + boolean isComplete; + + BatchedTask(int queueIndex, int taskIndex, Priority priority, long insertionTimeMillis, TimeValue timeout) { + this.queueIndex = queueIndex; + this.taskIndex = taskIndex; + this.priority = priority; + this.insertionTimeMillis = insertionTimeMillis; + this.timeout = timeout; + } + + void assertPendingTaskEntry(boolean expectExecuting) { + assertFalse(isComplete); + final var pendingTaskEntry = getPendingTasks().stream() + .filter(t -> t.getInsertOrder() == taskIndex) + .findFirst() + .orElseThrow(() -> new AssertionError("task not found")); + + assertEquals(getSource(), pendingTaskEntry.getSource().string()); + assertEquals(expectExecuting, pendingTaskEntry.isExecuting()); + assertEquals(priority, pendingTaskEntry.getPriority()); + assertEquals( + deterministicTaskQueue.getCurrentTimeMillis() - insertionTimeMillis, + pendingTaskEntry.getTimeInQueueInMillis() + ); + assertThat(pendingTaskEntry.getTimeInQueueInMillis(), lessThanOrEqualTo(masterService.getMaxTaskWaitTime().millis())); + } + + private List getPendingTasks() { + final var pendingTasks = masterService.pendingTasks(); + assertEquals(pendingTasks.size(), masterService.numberOfPendingTasks()); + return pendingTasks; + } + + void assertNoPendingTaskEntry() { + assertTrue(isComplete); + assertTrue(getPendingTasks().stream().noneMatch(t -> t.getInsertOrder() == taskIndex)); + } + + void onExecute() { + assertPendingTaskEntry(true); + actionCount.incrementAndGet(); + } + + void onSuccess() { + assertPendingTaskEntry(true); + actionCount.incrementAndGet(); + isComplete = true; + } + + String getSource() { + return "task-" + (queueIndex < 0 ? "unbatched" : Integer.toString(queueIndex)) + "-" + taskIndex; + } + + @Override + public void onFailure(Exception e) { + assertThat(e, instanceOf(ProcessClusterEventTimeoutException.class)); + assertThat(e.getMessage(), equalTo("failed to process cluster event (" + getSource() + ") within " + timeout)); + assertFalse(isComplete); + isComplete = true; + assertNoPendingTaskEntry(); + actionCount.incrementAndGet(); + actionCount.incrementAndGet(); + } + + } + + final var batchingPriorities = new ArrayList(); + final var taskQueues = new ArrayList>(); + for (int i = 0; i < 3; i++) { + final var batchingPriority = randomFrom(Priority.values()); + batchingPriorities.add(batchingPriority); + taskQueues.add(masterService.getTaskQueue("queue-" + i, batchingPriority, batchExecutionContext -> { + for (final var taskContext : batchExecutionContext.taskContexts()) { + final var task = taskContext.getTask(); + task.onExecute(); + taskContext.success(() -> { + deterministicTaskQueue.scheduleNow(task::assertNoPendingTaskEntry); + task.onSuccess(); + }); + } + return batchExecutionContext.initialState(); + })); + } + + final var taskCount = between(1, 10); + final var tasks = new ArrayList(taskCount); + long firstTaskInsertTimeMillis = 0L; + for (int i = 1; i <= taskCount; i++) { + + if (randomBoolean()) { + var targetTime = deterministicTaskQueue.getCurrentTimeMillis() + between(1, 30000); + deterministicTaskQueue.scheduleAt(targetTime, () -> {}); + + while (deterministicTaskQueue.getCurrentTimeMillis() < targetTime) { + deterministicTaskQueue.advanceTime(); + } + } + if (i == 1) { + firstTaskInsertTimeMillis = deterministicTaskQueue.getCurrentTimeMillis(); + } + + final var queueIndex = between(-1, taskQueues.size() - 1); + final var priority = queueIndex == -1 ? randomFrom(Priority.values()) : batchingPriorities.get(queueIndex); + + final var task = new BatchedTask( + queueIndex, + i, + priority, + deterministicTaskQueue.getCurrentTimeMillis(), + TimeValue.timeValueMillis(between(0, 30000)) + ); + tasks.add(task); + + if (queueIndex == -1) { + masterService.submitUnbatchedStateUpdateTask(task.getSource(), new ClusterStateUpdateTask(priority, task.timeout) { + @Override + public ClusterState execute(ClusterState currentState) { + task.onExecute(); + return currentState; + } + + @Override + public void clusterStateProcessed(ClusterState initialState, ClusterState newState) { + task.onSuccess(); + } + + @Override + public void onFailure(Exception e) { + task.onFailure(e); + } + }); + } else { + taskQueues.get(queueIndex).submitTask(task.getSource(), task, task.timeout); + } + + assertThat( + masterService.getMaxTaskWaitTime().millis(), + equalTo(deterministicTaskQueue.getCurrentTimeMillis() - firstTaskInsertTimeMillis) + ); + } + + for (final var task : tasks) { + task.assertPendingTaskEntry(false); + } + + while (deterministicTaskQueue.hasDeferredTasks()) { + deterministicTaskQueue.advanceTime(); + } + + for (final var task : tasks) { + task.assertPendingTaskEntry(false); + } + + threadPool.getThreadContext().markAsSystemContext(); + deterministicTaskQueue.runAllTasks(); + assertThat(actionCount.get(), equalTo(taskCount * 2)); + for (final var task : tasks) { + task.assertNoPendingTaskEntry(); + } + assertThat(masterService.getMaxTaskWaitTime(), equalTo(TimeValue.ZERO)); + } + } + + public void testRejectionBehaviour() { + + final var deterministicTaskQueue = new DeterministicTaskQueue(); + + final var threadPool = deterministicTaskQueue.getThreadPool(); + final var threadPoolExecutor = new PrioritizedEsThreadPoolExecutor( + "Rejecting", + 1, + 1, + 1, + TimeUnit.SECONDS, + r -> { throw new AssertionError("should not create new threads"); }, + null, + null + ) { + @Override + public void execute(Runnable command, final TimeValue timeout, final Runnable timeoutCallback) { + throw new AssertionError("not implemented"); + } + + @Override + public void execute(Runnable command) { + if (command instanceof AbstractRunnable) { + throw new AssertionError("unexpected abstract runnable: " + command); + } else { + throw new EsRejectedExecutionException("test", true); + } + } + }; + + try (var masterService = createMasterService(true, null, threadPool, threadPoolExecutor)) { + + final var actionCount = new AtomicInteger(); + final var testHeader = "test-header"; + + class TestTask implements ClusterStateTaskListener { + private final String expectedHeader = threadPool.getThreadContext().getHeader(testHeader); + + @Override + public void onFailure(Exception e) { + assertEquals(expectedHeader, threadPool.getThreadContext().getHeader(testHeader)); + if ((e instanceof FailedToCommitClusterStateException + && e.getCause()instanceof EsRejectedExecutionException esre + && esre.isExecutorShutdown()) == false) { + throw new AssertionError("unexpected exception", e); + } + actionCount.incrementAndGet(); + } + } + + final var queue = masterService.getTaskQueue( + "queue", + randomFrom(Priority.values()), + batchExecutionContext -> { throw new AssertionError("should not execute batch"); } + ); + + try (var ignored = threadPool.getThreadContext().stashContext()) { + threadPool.getThreadContext().putHeader(testHeader, randomAlphaOfLength(10)); + queue.submitTask("batched", new TestTask(), null); + } + try (var ignored = threadPool.getThreadContext().stashContext()) { + threadPool.getThreadContext().putHeader(testHeader, randomAlphaOfLength(10)); + masterService.submitUnbatchedStateUpdateTask("unbatched", new ClusterStateUpdateTask() { + private final TestTask innerTask = new TestTask(); + + @Override + public ClusterState execute(ClusterState currentState) { + throw new AssertionError("should not execute task"); + } + + @Override + public void onFailure(Exception e) { + innerTask.onFailure(e); + } + }); + } + threadPool.getThreadContext().markAsSystemContext(); + deterministicTaskQueue.runAllTasks(); + + assertEquals(2, actionCount.get()); + } + } + + public void testLifecycleBehaviour() { + + final var deterministicTaskQueue = new DeterministicTaskQueue(); + + final var threadPool = deterministicTaskQueue.getThreadPool(); + final var threadPoolExecutor = deterministicTaskQueue.getPrioritizedEsThreadPoolExecutor(); + + try (var masterService = createMasterService(true, null, threadPool, threadPoolExecutor)) { + + final var actionCount = new AtomicInteger(); + final var testHeader = "test-header"; + + class TestTask implements ClusterStateTaskListener { + private final String expectedHeader = threadPool.getThreadContext().getHeader(testHeader); + + @Override + public void onFailure(Exception e) { + assertEquals(expectedHeader, threadPool.getThreadContext().getHeader(testHeader)); + if ((e instanceof FailedToCommitClusterStateException + && e.getCause()instanceof EsRejectedExecutionException esre + && esre.isExecutorShutdown()) == false) { + throw new AssertionError("unexpected exception", e); + } + actionCount.incrementAndGet(); + } + } + + final var queue = masterService.getTaskQueue( + "queue", + randomFrom(Priority.values()), + batchExecutionContext -> { throw new AssertionError("should not execute batch"); } + ); + + while (true) { + try (var ignored = threadPool.getThreadContext().stashContext()) { + threadPool.getThreadContext().putHeader(testHeader, randomAlphaOfLength(10)); + queue.submitTask("batched", new TestTask(), null); + } + try (var ignored = threadPool.getThreadContext().stashContext()) { + threadPool.getThreadContext().putHeader(testHeader, randomAlphaOfLength(10)); + masterService.submitUnbatchedStateUpdateTask("unbatched", new ClusterStateUpdateTask() { + private final TestTask innerTask = new TestTask(); + + @Override + public ClusterState execute(ClusterState currentState) { + throw new AssertionError("should not execute task"); + } + + @Override + public void onFailure(Exception e) { + innerTask.onFailure(e); + } + }); + } + + if (masterService.lifecycleState() == Lifecycle.State.STARTED) { + masterService.close(); + } else { + break; + } + } + + threadPool.getThreadContext().markAsSystemContext(); + deterministicTaskQueue.runAllTasks(); + assertEquals(4, actionCount.get()); + } + } + + public void testTimeoutBehaviour() { + + final var deterministicTaskQueue = new DeterministicTaskQueue(); + + final var threadPool = deterministicTaskQueue.getThreadPool(); + final var threadPoolExecutor = deterministicTaskQueue.getPrioritizedEsThreadPoolExecutor(); + + try (var masterService = createMasterService(true, null, threadPool, threadPoolExecutor)) { + + final var actionCount = new AtomicInteger(); + final var testHeader = "test-header"; + + class BlockingTask extends ClusterStateUpdateTask { + BlockingTask() { + super(Priority.IMMEDIATE); + } + + @Override + public ClusterState execute(ClusterState currentState) { + var targetTime = deterministicTaskQueue.getCurrentTimeMillis() + between(1, 1000); + deterministicTaskQueue.scheduleAt(targetTime, () -> {}); + + while (deterministicTaskQueue.getCurrentTimeMillis() < targetTime) { + deterministicTaskQueue.advanceTime(); + } + + return currentState; + } + + @Override + public void clusterStateProcessed(ClusterState initialState, ClusterState newState) { + if (actionCount.get() < 2) { + masterService.submitUnbatchedStateUpdateTask("blocker", BlockingTask.this); + } + } + + @Override + public void onFailure(Exception e) { + throw new AssertionError("unexpected", e); + } + } + + masterService.submitUnbatchedStateUpdateTask("blocker", new BlockingTask()); + + class TestTask implements ClusterStateTaskListener { + private final String expectedHeader = threadPool.getThreadContext().getHeader(testHeader); + private final TimeValue timeout; + + TestTask(TimeValue timeout) { + this.timeout = timeout; + } + + @Override + public void onFailure(Exception e) { + assertEquals(expectedHeader, threadPool.getThreadContext().getHeader(testHeader)); + assertThat(deterministicTaskQueue.getCurrentTimeMillis(), greaterThanOrEqualTo(timeout.millis())); + assertThat(e, instanceOf(ProcessClusterEventTimeoutException.class)); + assertThat( + e.getMessage(), + allOf(containsString("failed to process cluster event"), containsString(timeout.toString())) + ); + actionCount.incrementAndGet(); + } + } + + final var queue = masterService.getTaskQueue( + "queue", + Priority.NORMAL, + batchExecutionContext -> { throw new AssertionError("should not execute batch"); } + ); + + try (var ignored = threadPool.getThreadContext().stashContext()) { + threadPool.getThreadContext().putHeader(testHeader, randomAlphaOfLength(10)); + final var testTask = new TestTask(TimeValue.timeValueMillis(between(1, 30000))); + queue.submitTask("batched", testTask, testTask.timeout); + } + + try (var ignored = threadPool.getThreadContext().stashContext()) { + threadPool.getThreadContext().putHeader(testHeader, randomAlphaOfLength(10)); + final var innerTask = new TestTask(TimeValue.timeValueMillis(between(1, 30000))); + masterService.submitUnbatchedStateUpdateTask("unbatched", new ClusterStateUpdateTask(innerTask.timeout) { + + @Override + public ClusterState execute(ClusterState currentState) { + throw new AssertionError("should not execute task"); + } + + @Override + public void onFailure(Exception e) { + innerTask.onFailure(e); + } + }); + } + + threadPool.getThreadContext().markAsSystemContext(); + deterministicTaskQueue.runAllTasks(); + assertEquals(2, actionCount.get()); + } + } + /** * Returns the cluster state that the master service uses (and that is provided by the discovery layer) */ diff --git a/server/src/test/java/org/elasticsearch/cluster/service/TaskBatcherTests.java b/server/src/test/java/org/elasticsearch/cluster/service/TaskBatcherTests.java deleted file mode 100644 index 3941cbb51ac63..0000000000000 --- a/server/src/test/java/org/elasticsearch/cluster/service/TaskBatcherTests.java +++ /dev/null @@ -1,301 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.cluster.service; - -import org.apache.logging.log4j.Logger; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; -import org.elasticsearch.core.TimeValue; -import org.junit.Before; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasToString; - -public class TaskBatcherTests extends TaskExecutorTests { - - protected TestTaskBatcher taskBatcher; - - @Before - public void setUpBatchingTaskExecutor() throws Exception { - taskBatcher = new TestTaskBatcher(logger, threadExecutor); - } - - static class TestTaskBatcher extends TaskBatcher { - - TestTaskBatcher(Logger logger, PrioritizedEsThreadPoolExecutor threadExecutor) { - super(logger, threadExecutor); - } - - @SuppressWarnings("unchecked") - @Override - protected void run(Object batchingKey, List tasks, BatchSummary tasksSummary) { - List updateTasks = (List) tasks; - ((TestExecutor) batchingKey).execute(updateTasks.stream().map(t -> t.task).toList()); - updateTasks.forEach(updateTask -> updateTask.listener.processed()); - } - - @Override - protected void onTimeout(BatchedTask task, TimeValue timeout) { - threadPool.generic() - .execute(() -> ((UpdateTask) task).listener.onFailure(new ProcessClusterEventTimeoutException(timeout, task.source))); - } - - class UpdateTask extends BatchedTask { - final TestListener listener; - - UpdateTask(Priority priority, String source, Object task, TestListener listener, TestExecutor executor) { - super(priority, source, executor, task); - this.listener = listener; - } - - @Override - @SuppressWarnings("unchecked") - public String describeTasks(List tasks) { - return ((TestExecutor) batchingKey).describeTasks(tasks.stream().map(BatchedTask::getTask).toList()); - } - } - - } - - @Override - protected void submitTask(String source, TestTask testTask) { - submitTask(source, testTask, testTask, testTask, testTask); - } - - private void submitTask(String source, T task, ClusterStateTaskConfig config, TestExecutor executor, TestListener listener) { - taskBatcher.submitTask(taskBatcher.new UpdateTask(config.priority(), source, task, listener, executor), config.timeout()); - } - - @Override - public void testTimedOutTaskCleanedUp() throws Exception { - super.testTimedOutTaskCleanedUp(); - assertTrue("expected empty map but was " + taskBatcher.tasksPerBatchingKey, taskBatcher.tasksPerBatchingKey.isEmpty()); - } - - public void testOneExecutorDoesntStarveAnother() throws InterruptedException { - final List executionOrder = Collections.synchronizedList(new ArrayList<>()); - final Semaphore allowProcessing = new Semaphore(0); - final Semaphore startedProcessing = new Semaphore(0); - - class TaskExecutor implements TestExecutor { - - @Override - public void execute(List tasks) { - executionOrder.addAll(tasks); // do this first, so startedProcessing can be used as a notification that this is done. - startedProcessing.release(tasks.size()); - try { - allowProcessing.acquire(tasks.size()); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - } - - TaskExecutor executorA = new TaskExecutor(); - TaskExecutor executorB = new TaskExecutor(); - - final ClusterStateTaskConfig config = ClusterStateTaskConfig.build(Priority.NORMAL); - final TestListener noopListener = e -> { throw new AssertionError(e); }; - // this blocks the cluster state queue, so we can set it up right - submitTask("0", "A0", config, executorA, noopListener); - // wait to be processed - startedProcessing.acquire(1); - assertThat(executionOrder, equalTo(Arrays.asList("A0"))); - - // these will be the first batch - submitTask("1", "A1", config, executorA, noopListener); - submitTask("2", "A2", config, executorA, noopListener); - - // release the first 0 task, but not the second - allowProcessing.release(1); - startedProcessing.acquire(2); - assertThat(executionOrder, equalTo(Arrays.asList("A0", "A1", "A2"))); - - // setup the queue with pending tasks for another executor same priority - submitTask("3", "B3", config, executorB, noopListener); - submitTask("4", "B4", config, executorB, noopListener); - - submitTask("5", "A5", config, executorA, noopListener); - submitTask("6", "A6", config, executorA, noopListener); - - // now release the processing - allowProcessing.release(6); - - // wait for last task to be processed - startedProcessing.acquire(4); - - assertThat(executionOrder, equalTo(Arrays.asList("A0", "A1", "A2", "B3", "B4", "A5", "A6"))); - } - - static class TaskExecutor implements TestExecutor { - List tasks = new ArrayList<>(); - - @Override - public void execute(List tasks) { - this.tasks.addAll(tasks); - } - } - - // test that for a single thread, tasks are executed in the order - // that they are submitted - public void testTasksAreExecutedInOrder() throws BrokenBarrierException, InterruptedException { - int numberOfThreads = randomIntBetween(2, 8); - TaskExecutor[] executors = new TaskExecutor[numberOfThreads]; - for (int i = 0; i < numberOfThreads; i++) { - executors[i] = new TaskExecutor(); - } - - int tasksSubmittedPerThread = randomIntBetween(2, 1024); - - CountDownLatch updateLatch = new CountDownLatch(numberOfThreads * tasksSubmittedPerThread); - - final TestListener listener = new TestListener() { - @Override - public void onFailure(Exception e) { - throw new AssertionError(e); - } - - @Override - public void processed() { - updateLatch.countDown(); - } - }; - - CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); - - for (int i = 0; i < numberOfThreads; i++) { - final int index = i; - Thread thread = new Thread(() -> { - try { - barrier.await(); - for (int j = 0; j < tasksSubmittedPerThread; j++) { - submitTask( - "[" + index + "][" + j + "]", - j, - ClusterStateTaskConfig.build(randomFrom(Priority.values())), - executors[index], - listener - ); - } - barrier.await(); - } catch (InterruptedException | BrokenBarrierException e) { - throw new AssertionError(e); - } - }); - thread.start(); - } - - // wait for all threads to be ready - barrier.await(); - // wait for all threads to finish - barrier.await(); - - assertTrue(updateLatch.await(10, TimeUnit.SECONDS)); - - for (int i = 0; i < numberOfThreads; i++) { - assertEquals(tasksSubmittedPerThread, executors[i].tasks.size()); - for (int j = 0; j < tasksSubmittedPerThread; j++) { - assertNotNull(executors[i].tasks.get(j)); - assertEquals("cluster state update task executed out of order", j, (int) executors[i].tasks.get(j)); - } - } - } - - public void testSingleTaskSubmission() throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(1); - final Integer task = randomInt(1024); - TestExecutor executor = taskList -> { - assertThat(taskList.size(), equalTo(1)); - assertThat(taskList.get(0), equalTo(task)); - }; - submitTask("test", task, ClusterStateTaskConfig.build(randomFrom(Priority.values())), executor, new TestListener() { - @Override - public void processed() { - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - throw new AssertionError(e); - } - }); - latch.await(); - } - - public void testDuplicateSubmission() throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(2); - try (BlockingTask blockingTask = new BlockingTask(Priority.IMMEDIATE)) { - submitTask("blocking", blockingTask); - - TestExecutor executor = tasks -> {}; - SimpleTask task = new SimpleTask(1); - TestListener listener = new TestListener() { - @Override - public void processed() { - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - throw new AssertionError(e); - } - }; - - submitTask("first time", task, ClusterStateTaskConfig.build(Priority.NORMAL), executor, listener); - - final AssertionError e = expectThrows( - AssertionError.class, - () -> submitTask("second time", task, ClusterStateTaskConfig.build(Priority.NORMAL), executor, listener) - ); - assertThat(e, hasToString(containsString("task [1] with source [second time] is already queued"))); - - submitTask("third time a charm", new SimpleTask(1), ClusterStateTaskConfig.build(Priority.NORMAL), executor, listener); - - assertThat(latch.getCount(), equalTo(2L)); - } - latch.await(); - } - - private static class SimpleTask { - private final int id; - - private SimpleTask(int id) { - this.id = id; - } - - @Override - public int hashCode() { - return super.hashCode(); - } - - @Override - public boolean equals(Object obj) { - return super.equals(obj); - } - - @Override - public String toString() { - return Integer.toString(id); - } - } - -} diff --git a/server/src/test/java/org/elasticsearch/cluster/service/TaskExecutorTests.java b/server/src/test/java/org/elasticsearch/cluster/service/TaskExecutorTests.java index 8872bab724020..71ce3023212c8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/service/TaskExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/service/TaskExecutorTests.java @@ -7,7 +7,6 @@ */ package org.elasticsearch.cluster.service; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException; import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; @@ -59,8 +58,7 @@ public void setUpExecutor() { getClass().getName() + "/" + getTestName(), daemonThreadFactory(Settings.EMPTY, "test_thread"), threadPool.getThreadContext(), - threadPool.scheduler(), - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + threadPool.scheduler() ); } @@ -93,10 +91,7 @@ default String describeTasks(List tasks) { } } - /** - * Task class that works for single tasks as well as batching (see {@link TaskBatcherTests}) - */ - protected abstract static class TestTask implements TestExecutor, TestListener, ClusterStateTaskConfig { + protected abstract static class TestTask implements TestExecutor, TestListener { @Override public void execute(List tasks) { @@ -104,12 +99,10 @@ public void execute(List tasks) { } @Nullable - @Override public TimeValue timeout() { return null; } - @Override public Priority priority() { return Priority.NORMAL; } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java index f43e4a91d7fee..e639407aa01a9 100644 --- a/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java @@ -18,9 +18,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.PriorityBlockingQueue; @@ -28,7 +26,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -65,8 +62,7 @@ public void testSubmitPrioritizedExecutorWithRunnables() throws Exception { getName(), EsExecutors.daemonThreadFactory(getTestName()), holder, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ); List results = new ArrayList<>(8); CountDownLatch awaitingLatch = new CountDownLatch(1); @@ -100,8 +96,7 @@ public void testExecutePrioritizedExecutorWithRunnables() throws Exception { getName(), EsExecutors.daemonThreadFactory(getTestName()), holder, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ); List results = new ArrayList<>(8); CountDownLatch awaitingLatch = new CountDownLatch(1); @@ -135,8 +130,7 @@ public void testSubmitPrioritizedExecutorWithCallables() throws Exception { getName(), EsExecutors.daemonThreadFactory(getTestName()), holder, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ); List results = new ArrayList<>(8); CountDownLatch awaitingLatch = new CountDownLatch(1); @@ -170,8 +164,7 @@ public void testSubmitPrioritizedExecutorWithMixed() throws Exception { getTestName(), EsExecutors.daemonThreadFactory(getTestName()), holder, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ); List results = new ArrayList<>(8); CountDownLatch awaitingLatch = new CountDownLatch(1); @@ -206,8 +199,7 @@ public void testTimeout() throws Exception { getName(), EsExecutors.daemonThreadFactory(getTestName()), holder, - timer, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + timer ); final CountDownLatch invoked = new CountDownLatch(1); final CountDownLatch block = new CountDownLatch(1); @@ -274,8 +266,7 @@ public void testTimeoutCleanup() throws Exception { getName(), EsExecutors.daemonThreadFactory(getTestName()), holder, - timer, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + timer ); final CountDownLatch invoked = new CountDownLatch(1); executor.execute(new Runnable() { @@ -300,61 +291,6 @@ public void run() { assertTrue(terminate(threadPool)); } - public void testStarvationWatcherInteraction() throws Exception { - final AtomicInteger emptyQueueCount = new AtomicInteger(); - final AtomicInteger nonemptyQueueCount = new AtomicInteger(); - - final ExecutorService executor = EsExecutors.newSinglePrioritizing( - getName(), - EsExecutors.daemonThreadFactory(getTestName()), - holder, - null, - new PrioritizedEsThreadPoolExecutor.StarvationWatcher() { - @Override - public void onEmptyQueue() { - emptyQueueCount.incrementAndGet(); - } - - @Override - public void onNonemptyQueue() { - nonemptyQueueCount.incrementAndGet(); - } - } - ); - final int jobCount = between(1, 10); - final List results = new ArrayList<>(jobCount); - final CyclicBarrier awaitingBarrier = new CyclicBarrier(2); - final CountDownLatch finishedLatch = new CountDownLatch(jobCount); - executor.submit(() -> { - try { - awaitingBarrier.await(); - awaitingBarrier.await(); - } catch (InterruptedException | BrokenBarrierException e) { - throw new AssertionError("unexpected", e); - } - }); - awaitingBarrier.await(); // ensure blocking job started and observed an empty queue first - for (int i = 0; i < jobCount; i++) { - executor.submit(new Job(i, Priority.NORMAL, results, finishedLatch)); - } - awaitingBarrier.await(); // allow blocking job to complete - finishedLatch.await(); - - assertThat(results.size(), equalTo(jobCount)); - for (int i = 0; i < jobCount; i++) { - assertThat(results.get(i), equalTo(i)); - } - - terminate(executor); - - // queue was observed empty when the blocking job started and before and after the last numbered Job - assertThat(emptyQueueCount.get(), equalTo(3)); - - // queue was observed nonempty after the blocking job and all but the last numbered Job - // NB it was also nonempty before each Job but the last, but this doesn't result in notifications - assertThat(nonemptyQueueCount.get(), equalTo(jobCount)); - } - static class AwaitingJob extends PrioritizedRunnable { private final CountDownLatch latch; diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index 2954cdcbbdcda..d6e8309919e96 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -181,8 +181,7 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { TimeUnit.SECONDS, r -> { throw new AssertionError("should not create new threads"); }, null, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ) { @Override public void execute(Runnable command, final TimeValue timeout, final Runnable timeoutCallback) { diff --git a/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java b/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java index 4c5b2ae78203c..8485f28d087ac 100644 --- a/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java @@ -18,16 +18,20 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.env.Environment; import org.elasticsearch.ingest.IngestInfo; import org.elasticsearch.ingest.ProcessorInfo; import org.elasticsearch.reservedstate.action.ReservedClusterSettingsAction; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -49,6 +53,7 @@ import java.time.ZoneOffset; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -56,6 +61,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.elasticsearch.node.Node.NODE_NAME_SETTING; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; @@ -88,10 +94,10 @@ public void setUp() throws Exception { clusterService = spy( new ClusterService( - Settings.EMPTY, + Settings.builder().put(NODE_NAME_SETTING.getKey(), "test").build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadpool, - null + new TaskManager(Settings.EMPTY, threadpool, Set.of()) ) ); @@ -102,6 +108,17 @@ public void setUp() throws Exception { doAnswer((Answer) invocation -> clusterState).when(clusterService).state(); clusterService.setRerouteService(mock(RerouteService.class)); + clusterService.setNodeConnectionsService(mock(NodeConnectionsService.class)); + clusterService.getClusterApplierService().setInitialState(clusterState); + clusterService.getMasterService().setClusterStatePublisher((e, pl, al) -> { + ClusterServiceUtils.setAllElapsedMillis(e); + al.onCommit(TimeValue.ZERO); + for (DiscoveryNode node : e.getNewState().nodes()) { + al.onNodeAck(node, null); + } + pl.onResponse(null); + }); + clusterService.getMasterService().setClusterStateSupplier(() -> clusterState); env = newEnvironment(Settings.EMPTY); Files.createDirectories(env.configFile()); @@ -145,11 +162,14 @@ public void setUp() throws Exception { nodeClient = mock(NodeClient.class); fileSettingsService = spy(new FileSettingsService(clusterService, controller, env, nodeClient)); doAnswer(i -> clusterAdminClient).when(fileSettingsService).clusterAdminClient(); + + clusterService.start(); } @After public void tearDown() throws Exception { super.tearDown(); + clusterService.close(); threadpool.shutdownNow(); } diff --git a/server/src/test/java/org/elasticsearch/reservedstate/service/ReservedClusterStateServiceTests.java b/server/src/test/java/org/elasticsearch/reservedstate/service/ReservedClusterStateServiceTests.java index 1af56ed9079ab..d863163c5b2a1 100644 --- a/server/src/test/java/org/elasticsearch/reservedstate/service/ReservedClusterStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/reservedstate/service/ReservedClusterStateServiceTests.java @@ -15,12 +15,14 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; import org.elasticsearch.cluster.ClusterStateTaskExecutor; +import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.ReservedStateErrorMetadata; import org.elasticsearch.cluster.metadata.ReservedStateHandlerMetadata; import org.elasticsearch.cluster.metadata.ReservedStateMetadata; import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Releasable; @@ -67,9 +69,15 @@ public class ReservedClusterStateServiceTests extends ESTestCase { + @SuppressWarnings("unchecked") + private static MasterServiceTaskQueue mockTaskQueue() { + return (MasterServiceTaskQueue) mock(MasterServiceTaskQueue.class); + } + public void testOperatorController() throws IOException { ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); ClusterService clusterService = mock(ClusterService.class); + when(clusterService.getTaskQueue(any(), any(), any())).thenReturn(mockTaskQueue()); final ClusterName clusterName = new ClusterName("elasticsearch"); ClusterState state = ClusterState.builder(clusterName).build(); @@ -486,6 +494,7 @@ public void testCheckAndReportError() { ClusterService clusterService = mock(ClusterService.class); var state = ClusterState.builder(new ClusterName("elasticsearch")).build(); when(clusterService.state()).thenReturn(state); + when(clusterService.getTaskQueue(any(), any(), any())).thenReturn(mockTaskQueue()); final var controller = spy(new ReservedClusterStateService(clusterService, List.of())); diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index 38ac3dfb64224..52285d7dfa0bd 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; @@ -55,6 +56,7 @@ import static org.elasticsearch.cluster.ClusterModule.BALANCED_ALLOCATOR; import static org.elasticsearch.cluster.ClusterModule.DESIRED_BALANCE_ALLOCATOR; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; +import static org.mockito.Mockito.mock; public abstract class ESAllocationTestCase extends ESTestCase { private static final ClusterSettings EMPTY_CLUSTER_SETTINGS = new ClusterSettings( @@ -116,7 +118,12 @@ private static ShardsAllocator createShardsAllocator(Settings settings) { private static DesiredBalanceShardsAllocator createDesiredBalanceShardsAllocator(Settings settings) { var queue = new DeterministicTaskQueue(); - return new DesiredBalanceShardsAllocator(new BalancedShardsAllocator(settings), queue.getThreadPool(), null, null) { + return new DesiredBalanceShardsAllocator( + new BalancedShardsAllocator(settings), + queue.getThreadPool(), + mock(ClusterService.class), + null + ) { private RoutingAllocation lastAllocation; @Override diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java index 92200734b646e..e5c66ae88bcac 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java @@ -1604,7 +1604,7 @@ boolean deliverBlackholedRequests() { } int getPendingTaskCount() { - return masterService.getFakeMasterServicePendingTaskCount(); + return masterService.numberOfPendingTasks(); } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/service/FakeThreadPoolMasterService.java b/test/framework/src/main/java/org/elasticsearch/cluster/service/FakeThreadPoolMasterService.java index 7b4a2283dbd41..53711fdb54d28 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/service/FakeThreadPoolMasterService.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/service/FakeThreadPoolMasterService.java @@ -13,6 +13,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStatePublicationEvent; import org.elasticsearch.cluster.coordination.ClusterStatePublisher.AckListener; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; @@ -79,8 +80,7 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { TimeUnit.SECONDS, r -> { throw new AssertionError("should not create new threads"); }, null, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ) { @Override @@ -90,14 +90,18 @@ public void execute(Runnable command, final TimeValue timeout, final Runnable ti @Override public void execute(Runnable command) { - pendingTasks.add(command); - scheduleNextTaskIfNecessary(); + if (command.toString().equals("awaitsfix thread keepalive") == false) { + // TODO remove this temporary fix + pendingTasks.add(command); + scheduleNextTaskIfNecessary(); + } } - }; - } - public int getFakeMasterServicePendingTaskCount() { - return pendingTasks.size(); + @Override + public Pending[] getPending() { + return pendingTasks.stream().map(r -> new Pending(r, Priority.NORMAL, 0L, false)).toArray(Pending[]::new); + } + }; } private void scheduleNextTaskIfNecessary() { diff --git a/test/framework/src/main/java/org/elasticsearch/common/util/concurrent/DeterministicTaskQueue.java b/test/framework/src/main/java/org/elasticsearch/common/util/concurrent/DeterministicTaskQueue.java index b27ab2899c671..98c6642d91574 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/util/concurrent/DeterministicTaskQueue.java +++ b/test/framework/src/main/java/org/elasticsearch/common/util/concurrent/DeterministicTaskQueue.java @@ -213,8 +213,7 @@ public PrioritizedEsThreadPoolExecutor getPrioritizedEsThreadPoolExecutor(Functi TimeUnit.SECONDS, r -> { throw new AssertionError("should not create new threads"); }, null, - null, - PrioritizedEsThreadPoolExecutor.StarvationWatcher.NOOP_STARVATION_WATCHER + null ) { @Override public void execute(Runnable command, final TimeValue timeout, final Runnable timeoutCallback) { @@ -357,12 +356,13 @@ public ScheduledCancellable schedule(Runnable command, TimeValue delay, String e final int STARTED = 1; final int CANCELLED = 2; final AtomicInteger taskState = new AtomicInteger(NOT_STARTED); + final Runnable contextPreservingRunnable = getThreadContext().preserveContext(command); scheduleAt(currentTimeMillis + delay.millis(), runnableWrapper.apply(new Runnable() { @Override public void run() { if (taskState.compareAndSet(NOT_STARTED, STARTED)) { - command.run(); + contextPreservingRunnable.run(); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/LicenseService.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/LicenseService.java index 41cd4dbf59851..5f2c3840864d3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/LicenseService.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/LicenseService.java @@ -15,10 +15,10 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.Lifecycle; @@ -136,8 +136,8 @@ public class LicenseService extends AbstractLifecycleComponent implements Cluste */ private final List allowedLicenseTypes; - private final StartTrialClusterTask.Executor startTrialExecutor = new StartTrialClusterTask.Executor(); - private final StartBasicClusterTask.Executor startBasicExecutor = new StartBasicClusterTask.Executor(); + private final MasterServiceTaskQueue startTrialTaskQueue; + private final MasterServiceTaskQueue startBasicTaskQueue; /** * Max number of nodes licensed by generated trial license @@ -163,6 +163,16 @@ public LicenseService( ) { this.settings = settings; this.clusterService = clusterService; + this.startTrialTaskQueue = clusterService.getTaskQueue( + "license-service-start-trial", + Priority.NORMAL, + new StartTrialClusterTask.Executor() + ); + this.startBasicTaskQueue = clusterService.getTaskQueue( + "license-service-start-basic", + Priority.NORMAL, + new StartBasicClusterTask.Executor() + ); this.clock = clock; this.scheduler = new SchedulerEngine(settings, clock); this.licenseState = licenseState; @@ -403,12 +413,7 @@ public void removeLicense(final ActionListener listener) "delete license", listener ); - clusterService.submitStateUpdateTask( - task.getDescription(), - task, - ClusterStateTaskConfig.build(Priority.NORMAL), // TODO should pass in request.masterNodeTimeout() here - startBasicExecutor - ); + startBasicTaskQueue.submitTask(task.getDescription(), task, null); // TODO should pass in request.masterNodeTimeout() here } public License getLicense() { @@ -431,11 +436,10 @@ void startTrialLicense(PostStartTrialRequest request, final ActionListener) mock(MasterServiceTaskQueue.class); + Mockito.when(clusterService.getTaskQueue(eq("license-service-start-basic"), any(), any())).thenReturn(taskQueue); + final Clock clock = randomBoolean() ? Clock.systemUTC() : Clock.systemDefaultZone(); + final var taskExecutorCaptor = ArgumentCaptor.forClass(StartBasicClusterTask.Executor.class); final LicenseService service = new LicenseService( settings, mock(ThreadPool.class), @@ -182,6 +190,7 @@ public void testStartBasicStartsNewLicenseIfFieldsDifferent() throws Exception { mock(ResourceWatcherService.class), mock(XPackLicenseState.class) ); + verify(clusterService).getTaskQueue(eq("license-service-start-basic"), any(), taskExecutorCaptor.capture()); final Consumer> assertion = future -> { PostStartBasicResponse response = future.actionGet(); @@ -195,11 +204,10 @@ public void testStartBasicStartsNewLicenseIfFieldsDifferent() throws Exception { assertion.accept(future); } else { final var taskCaptor = ArgumentCaptor.forClass(StartBasicClusterTask.class); - final var taskExecutorCaptor = ArgumentCaptor.forClass(StartBasicClusterTask.Executor.class); @SuppressWarnings("unchecked") final ArgumentCaptor listenerCaptor = ArgumentCaptor.forClass(Runnable.class); + verify(taskQueue).submitTask(any(), taskCaptor.capture(), any()); doNothing().when(taskContext).success(listenerCaptor.capture()); - verify(clusterService).submitStateUpdateTask(any(), taskCaptor.capture(), any(), taskExecutorCaptor.capture()); when(taskContext.getTask()).thenReturn(taskCaptor.getValue()); int maxNodes = randomValueOtherThan( @@ -230,6 +238,11 @@ private ClusterService mockDefaultClusterService() { return clusterService; } + @SuppressWarnings("unchecked") + private static MasterServiceTaskQueue newMockTaskQueue() { + return mock(MasterServiceTaskQueue.class); + } + private void assertRegisterValidLicense(Settings baseSettings, License.LicenseType licenseType) throws IOException { tryRegisterLicense(baseSettings, licenseType, future -> assertThat(future.actionGet().status(), equalTo(LicensesStatus.VALID))); } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunner.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunner.java index 9c69395050f7b..12864bb1dc7df 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunner.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunner.java @@ -11,12 +11,12 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.LifecycleExecutionState; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.SuppressForbidden; @@ -54,6 +54,7 @@ class IndexLifecycleRunner { private final PolicyStepsRegistry stepRegistry; private final ILMHistoryStore ilmHistoryStore; private final LongSupplier nowSupplier; + private final MasterServiceTaskQueue masterServiceTaskQueue; @SuppressWarnings("Convert2Lambda") // can't SuppressForbidden on a lambda private static final ClusterStateTaskExecutor ILM_TASK_EXECUTOR = @@ -91,6 +92,7 @@ public ClusterState execute(BatchExecutionContext> busyIndices = Collections.synchronizedSet(new HashSet<>()); - static final ClusterStateTaskConfig ILM_TASK_CONFIG = ClusterStateTaskConfig.build(Priority.NORMAL); - /** * Tracks already executing {@link IndexLifecycleClusterStateUpdateTask} tasks in {@link #executingTasks} to prevent queueing up * duplicate cluster state updates. @@ -640,7 +640,7 @@ private void submitUnlessAlreadyQueued(String source, IndexLifecycleClusterState busyIndices.remove(dedupKey); assert removed : "tried to unregister unknown task [" + task + "]"; })); - clusterService.submitStateUpdateTask(source, task, ILM_TASK_CONFIG, ILM_TASK_EXECUTOR); + masterServiceTaskQueue.submitTask(source, task, null); } else { logger.trace("skipped redundant execution of [{}]", source); } diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunnerTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunnerTests.java index 83f58493304fd..1a03541dcf16c 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunnerTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunnerTests.java @@ -20,6 +20,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.ClusterSettings; @@ -146,6 +148,7 @@ public void testRunPolicyTerminalPolicyStep() { runner.runPolicyAfterStateChange(policyName, indexMetadata); + Mockito.verify(clusterService, times(1)).getTaskQueue(anyString(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } @@ -164,15 +167,26 @@ public void testRunPolicyPhaseCompletePolicyStep() { runner.runPolicyAfterStateChange(policyName, indexMetadata); runner.runPeriodicStep(policyName, Metadata.builder().put(indexMetadata, true).build(), indexMetadata); + Mockito.verify(clusterService, times(1)).getTaskQueue(anyString(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } + @SuppressWarnings("unchecked") + private static MasterServiceTaskQueue newMockTaskQueue(ClusterService clusterService) { + final var masterServiceTaskQueue = mock(MasterServiceTaskQueue.class); + when(clusterService.getTaskQueue(eq("ilm-runner"), eq(Priority.NORMAL), any())).thenReturn( + masterServiceTaskQueue + ); + return masterServiceTaskQueue; + } + public void testRunPolicyPhaseCompleteWithMoreStepsPolicyStep() { String policyName = "async_action_policy"; TerminalPolicyStep stop = TerminalPolicyStep.INSTANCE; PhaseCompleteStep step = new PhaseCompleteStep(new StepKey("cold", "complete", "complete"), stop.getKey()); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); + MasterServiceTaskQueue taskQueue = newMockTaskQueue(clusterService); IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, historyStore, clusterService, threadPool, () -> 0L); IndexMetadata indexMetadata = IndexMetadata.builder("my_index") .settings(settings(Version.CURRENT)) @@ -183,7 +197,7 @@ public void testRunPolicyPhaseCompleteWithMoreStepsPolicyStep() { runner.runPolicyAfterStateChange(policyName, indexMetadata); runner.runPeriodicStep(policyName, Metadata.builder().put(indexMetadata, true).build(), indexMetadata); - Mockito.verify(clusterService, times(1)).submitStateUpdateTask(anyString(), any(), any(), any()); + Mockito.verify(taskQueue, times(1)).submitTask(anyString(), any(), any()); } public void testRunPolicyErrorStep() { @@ -216,6 +230,7 @@ public void testRunPolicyErrorStep() { runner.runPolicyAfterStateChange(policyName, indexMetadata); + Mockito.verify(clusterService).getTaskQueue(anyString(), any(Priority.class), any()); Mockito.verifyNoMoreInteractions(clusterService); } @@ -240,6 +255,7 @@ public void testRunPolicyErrorStepOnRetryableFailedStep() { PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, waitForRolloverStep); ClusterService clusterService = mock(ClusterService.class); + MasterServiceTaskQueue taskQueue = newMockTaskQueue(clusterService); when(clusterService.state()).thenReturn(ClusterState.EMPTY_STATE); IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, historyStore, clusterService, threadPool, () -> 0L); LifecycleExecutionState.Builder newState = LifecycleExecutionState.builder(); @@ -258,7 +274,7 @@ public void testRunPolicyErrorStepOnRetryableFailedStep() { runner.runPeriodicStep(policyName, Metadata.builder().put(indexMetadata, true).build(), indexMetadata); - Mockito.verify(clusterService, times(1)).submitStateUpdateTask(any(), any(), eq(IndexLifecycleRunner.ILM_TASK_CONFIG), any()); + Mockito.verify(taskQueue, times(1)).submitTask(anyString(), any(), any()); } public void testRunStateChangePolicyWithNoNextStep() throws Exception { @@ -648,6 +664,7 @@ public void testRunPolicyClusterStateActionStep() { MockClusterStateActionStep step = new MockClusterStateActionStep(stepKey, null); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); + MasterServiceTaskQueue taskQueue = newMockTaskQueue(clusterService); IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, historyStore, clusterService, threadPool, () -> 0L); IndexMetadata indexMetadata = IndexMetadata.builder("my_index") .settings(settings(Version.CURRENT)) @@ -658,14 +675,15 @@ public void testRunPolicyClusterStateActionStep() { runner.runPolicyAfterStateChange(policyName, indexMetadata); final ExecuteStepsUpdateTaskMatcher taskMatcher = new ExecuteStepsUpdateTaskMatcher(indexMetadata.getIndex(), policyName, step); - Mockito.verify(clusterService, Mockito.times(1)) - .submitStateUpdateTask( + Mockito.verify(taskQueue, Mockito.times(1)) + .submitTask( Mockito.eq(""" ilm-execute-cluster-state-steps [{"phase":"phase","action":"action","name":"cluster_state_action_step"} => null]"""), Mockito.argThat(taskMatcher), - eq(IndexLifecycleRunner.ILM_TASK_CONFIG), - any() + Mockito.eq(null) ); + Mockito.verifyNoMoreInteractions(taskQueue); + Mockito.verify(clusterService, Mockito.times(1)).getTaskQueue(any(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } @@ -675,6 +693,7 @@ public void testRunPolicyClusterStateWaitStep() { MockClusterStateWaitStep step = new MockClusterStateWaitStep(stepKey, null); PolicyStepsRegistry stepRegistry = createOneStepPolicyStepRegistry(policyName, step); ClusterService clusterService = mock(ClusterService.class); + MasterServiceTaskQueue taskQueue = newMockTaskQueue(clusterService); IndexLifecycleRunner runner = new IndexLifecycleRunner(stepRegistry, historyStore, clusterService, threadPool, () -> 0L); IndexMetadata indexMetadata = IndexMetadata.builder("my_index") .settings(settings(Version.CURRENT)) @@ -685,14 +704,15 @@ public void testRunPolicyClusterStateWaitStep() { runner.runPolicyAfterStateChange(policyName, indexMetadata); final ExecuteStepsUpdateTaskMatcher taskMatcher = new ExecuteStepsUpdateTaskMatcher(indexMetadata.getIndex(), policyName, step); - Mockito.verify(clusterService, Mockito.times(1)) - .submitStateUpdateTask( + Mockito.verify(taskQueue, Mockito.times(1)) + .submitTask( Mockito.eq(""" ilm-execute-cluster-state-steps [{"phase":"phase","action":"action","name":"cluster_state_action_step"} => null]"""), Mockito.argThat(taskMatcher), - eq(IndexLifecycleRunner.ILM_TASK_CONFIG), - any() + Mockito.eq(null) ); + Mockito.verifyNoMoreInteractions(taskQueue); + Mockito.verify(clusterService, Mockito.times(1)).getTaskQueue(any(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } @@ -714,6 +734,7 @@ public void testRunPolicyAsyncActionStepClusterStateChangeIgnored() { runner.runPolicyAfterStateChange(policyName, indexMetadata); assertEquals(0, step.getExecuteCount()); + Mockito.verify(clusterService, Mockito.times(1)).getTaskQueue(any(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } @@ -735,12 +756,14 @@ public void testRunPolicyAsyncWaitStepClusterStateChangeIgnored() { runner.runPolicyAfterStateChange(policyName, indexMetadata); assertEquals(0, step.getExecuteCount()); + Mockito.verify(clusterService, Mockito.times(1)).getTaskQueue(any(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } public void testRunPolicyThatDoesntExist() { String policyName = "cluster_state_action_policy"; ClusterService clusterService = mock(ClusterService.class); + MasterServiceTaskQueue taskQueue = newMockTaskQueue(clusterService); IndexLifecycleRunner runner = new IndexLifecycleRunner( new PolicyStepsRegistry(NamedXContentRegistry.EMPTY, null, null), historyStore, @@ -767,13 +790,14 @@ public void testRunPolicyThatDoesntExist() { return builder; } ); - Mockito.verify(clusterService, Mockito.times(1)) - .submitStateUpdateTask( + Mockito.verify(taskQueue, Mockito.times(1)) + .submitTask( Mockito.eq("ilm-set-step-info {policy [cluster_state_action_policy], index [my_index], currentStep [null]}"), Mockito.argThat(taskMatcher), - eq(IndexLifecycleRunner.ILM_TASK_CONFIG), - any() + Mockito.eq(null) ); + Mockito.verifyNoMoreInteractions(taskQueue); + Mockito.verify(clusterService, Mockito.times(1)).getTaskQueue(any(), any(), any()); Mockito.verifyNoMoreInteractions(clusterService); } diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java index ab983332730e7..e922eaf0f0ae5 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java @@ -63,6 +63,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static java.time.Clock.systemUTC; import static org.elasticsearch.cluster.metadata.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY; @@ -75,6 +76,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; public class IndexLifecycleServiceTests extends ESTestCase { @@ -328,25 +330,23 @@ public void testRequestedStopOnSafeAction() { ClusterChangedEvent event = new ClusterChangedEvent("_source", currentState, ClusterState.EMPTY_STATE); - SetOnce ranPolicy = new SetOnce<>(); - SetOnce moveToMaintenance = new SetOnce<>(); - doAnswer(invocationOnMock -> { - ranPolicy.set(true); - throw new AssertionError("invalid invocation"); - }).when(clusterService).submitStateUpdateTask(anyString(), any(), eq(IndexLifecycleRunner.ILM_TASK_CONFIG), any()); - + AtomicBoolean moveToMaintenance = new AtomicBoolean(); doAnswer(invocationOnMock -> { OperationModeUpdateTask task = (OperationModeUpdateTask) invocationOnMock.getArguments()[1]; assertThat(task.getILMOperationMode(), equalTo(OperationMode.STOPPED)); - moveToMaintenance.set(true); + assertTrue(moveToMaintenance.compareAndSet(false, true)); return null; }).when(clusterService) .submitUnbatchedStateUpdateTask(eq("ilm_operation_mode_update[stopped]"), any(OperationModeUpdateTask.class)); indexLifecycleService.applyClusterState(event); indexLifecycleService.triggerPolicies(currentState, randomBoolean()); - assertNull(ranPolicy.get()); assertTrue(moveToMaintenance.get()); + + Mockito.verify(clusterService, Mockito.atLeastOnce()).getClusterSettings(); + Mockito.verify(clusterService, Mockito.atLeastOnce()).submitUnbatchedStateUpdateTask(anyString(), any()); + Mockito.verify(clusterService, times(1)).getTaskQueue(anyString(), any(), any()); + Mockito.verifyNoMoreInteractions(clusterService); } public void testExceptionStillProcessesOtherIndices() { diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/ReservedLifecycleStateServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/ReservedLifecycleStateServiceTests.java index 359380754e05e..90ffa60de6ec9 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/ReservedLifecycleStateServiceTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/ReservedLifecycleStateServiceTests.java @@ -15,7 +15,9 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; import org.elasticsearch.cluster.ClusterStateTaskExecutor; +import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Releasable; @@ -54,7 +56,6 @@ import org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType; import org.elasticsearch.xpack.core.ilm.UnfollowAction; import org.elasticsearch.xpack.core.ilm.WaitForSnapshotAction; -import org.mockito.stubbing.Answer; import java.io.IOException; import java.util.ArrayList; @@ -241,58 +242,60 @@ public void testActionAddRemove() throws Exception { assertThat(ilmMetadata.getPolicyMetadatas().keySet(), containsInAnyOrder("my_timeseries_lifecycle2")); } - private void setupTaskMock(ClusterService clusterService, ClusterState state) { - doAnswer((Answer) invocation -> { - Object[] args = invocation.getArguments(); - - if ((args[3] instanceof ReservedStateUpdateTaskExecutor) == false) { - fail("Should have gotten a state update task to execute, instead got: " + args[3].getClass().getName()); - } - - ReservedStateUpdateTaskExecutor task = (ReservedStateUpdateTaskExecutor) args[3]; - - ClusterStateTaskExecutor.TaskContext context = new ClusterStateTaskExecutor.TaskContext<>() { - @Override - public ReservedStateUpdateTask getTask() { - return (ReservedStateUpdateTask) args[1]; - } + private void setupTaskMock(ClusterService clusterService) { + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).build(); + when(clusterService.state()).thenReturn(state); + when(clusterService.getTaskQueue(anyString(), any(), any())).thenAnswer(getQueueInvocation -> { + Object[] getQueueArgs = getQueueInvocation.getArguments(); + @SuppressWarnings("unchecked") + final MasterServiceTaskQueue taskQueue = mock(MasterServiceTaskQueue.class); + + if ((getQueueArgs[2]instanceof ReservedStateUpdateTaskExecutor executor)) { + doAnswer(submitTaskInvocation -> { + Object[] submitTaskArgs = submitTaskInvocation.getArguments(); + ClusterStateTaskExecutor.TaskContext context = new ClusterStateTaskExecutor.TaskContext<>() { + @Override + public ReservedStateUpdateTask getTask() { + return (ReservedStateUpdateTask) submitTaskArgs[1]; + } - @Override - public void success(Runnable onPublicationSuccess) {} + @Override + public void success(Runnable onPublicationSuccess) {} - @Override - public void success(Consumer publishedStateConsumer) {} + @Override + public void success(Consumer publishedStateConsumer) {} - @Override - public void success(Runnable onPublicationSuccess, ClusterStateAckListener clusterStateAckListener) {} + @Override + public void success(Runnable onPublicationSuccess, ClusterStateAckListener clusterStateAckListener) {} - @Override - public void success(Consumer publishedStateConsumer, ClusterStateAckListener clusterStateAckListener) {} + @Override + public void success( + Consumer publishedStateConsumer, + ClusterStateAckListener clusterStateAckListener + ) {} - @Override - public void onFailure(Exception failure) { - fail("Shouldn't fail here"); - } + @Override + public void onFailure(Exception failure) { + fail("Shouldn't fail here"); + } - @Override - public Releasable captureResponseHeaders() { + @Override + public Releasable captureResponseHeaders() { + return null; + } + }; + executor.execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(state, List.of(context), () -> null)); return null; - } - }; - - task.execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(state, List.of(context), () -> null)); - - return null; - }).when(clusterService).submitStateUpdateTask(anyString(), any(), any(), any()); + }).when(taskQueue).submitTask(anyString(), any(), any()); + } + return taskQueue; + }); } public void testOperatorControllerFromJSONContent() throws IOException { ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); ClusterService clusterService = mock(ClusterService.class); - final ClusterName clusterName = new ClusterName("elasticsearch"); - - ClusterState state = ClusterState.builder(clusterName).build(); - when(clusterService.state()).thenReturn(state); + setupTaskMock(clusterService); ReservedClusterStateService controller = new ReservedClusterStateService( clusterService, @@ -374,8 +377,6 @@ public void testOperatorControllerFromJSONContent() throws IOException { ) ); - setupTaskMock(clusterService, state); - try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, testJSON)) { controller.process("operator", parser, (e) -> { if (e != null) { @@ -388,10 +389,7 @@ public void testOperatorControllerFromJSONContent() throws IOException { public void testOperatorControllerWithPluginPackage() { ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); ClusterService clusterService = mock(ClusterService.class); - final ClusterName clusterName = new ClusterName("elasticsearch"); - - ClusterState state = ClusterState.builder(clusterName).build(); - when(clusterService.state()).thenReturn(state); + setupTaskMock(clusterService); ReservedClusterStateService controller = new ReservedClusterStateService( clusterService, @@ -438,8 +436,6 @@ public void testOperatorControllerWithPluginPackage() { ) ); - setupTaskMock(clusterService, state); - controller.process("operator", pack, (e) -> { if (e != null) { fail("Should not fail"); diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/action/ReservedSnapshotLifecycleStateServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/action/ReservedSnapshotLifecycleStateServiceTests.java index e180c2971a9be..a266974df70c9 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/action/ReservedSnapshotLifecycleStateServiceTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/action/ReservedSnapshotLifecycleStateServiceTests.java @@ -14,11 +14,13 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateAckListener; import org.elasticsearch.cluster.ClusterStateTaskExecutor; +import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.RepositoriesMetadata; import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Releasable; @@ -37,7 +39,6 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata; import org.elasticsearch.xpack.core.slm.action.DeleteSnapshotLifecycleAction; import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction; -import org.mockito.stubbing.Answer; import java.io.IOException; import java.util.Collections; @@ -216,58 +217,60 @@ public void testActionAddRemove() throws Exception { assertThat(slmMetadata.getSnapshotConfigurations().keySet(), containsInAnyOrder("daily-snapshots-2")); } - private void setupTaskMock(ClusterService clusterService, ClusterState state) { - doAnswer((Answer) invocation -> { - Object[] args = invocation.getArguments(); - - if ((args[3] instanceof ReservedStateUpdateTaskExecutor) == false) { - fail("Should have gotten a state update task to execute, instead got: " + args[3].getClass().getName()); - } - - ReservedStateUpdateTaskExecutor task = (ReservedStateUpdateTaskExecutor) args[3]; - - ClusterStateTaskExecutor.TaskContext context = new ClusterStateTaskExecutor.TaskContext<>() { - @Override - public ReservedStateUpdateTask getTask() { - return (ReservedStateUpdateTask) args[1]; - } + private void setupTaskMock(ClusterService clusterService) { + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).build(); + when(clusterService.state()).thenReturn(state); + when(clusterService.getTaskQueue(anyString(), any(), any())).thenAnswer(getQueueInvocation -> { + Object[] getQueueArgs = getQueueInvocation.getArguments(); + @SuppressWarnings("unchecked") + final MasterServiceTaskQueue taskQueue = mock(MasterServiceTaskQueue.class); + + if ((getQueueArgs[2]instanceof ReservedStateUpdateTaskExecutor executor)) { + doAnswer(submitTaskInvocation -> { + Object[] submitTaskArgs = submitTaskInvocation.getArguments(); + ClusterStateTaskExecutor.TaskContext context = new ClusterStateTaskExecutor.TaskContext<>() { + @Override + public ReservedStateUpdateTask getTask() { + return (ReservedStateUpdateTask) submitTaskArgs[1]; + } - @Override - public void success(Runnable onPublicationSuccess) {} + @Override + public void success(Runnable onPublicationSuccess) {} - @Override - public void success(Consumer publishedStateConsumer) {} + @Override + public void success(Consumer publishedStateConsumer) {} - @Override - public void success(Runnable onPublicationSuccess, ClusterStateAckListener clusterStateAckListener) {} + @Override + public void success(Runnable onPublicationSuccess, ClusterStateAckListener clusterStateAckListener) {} - @Override - public void success(Consumer publishedStateConsumer, ClusterStateAckListener clusterStateAckListener) {} + @Override + public void success( + Consumer publishedStateConsumer, + ClusterStateAckListener clusterStateAckListener + ) {} - @Override - public void onFailure(Exception failure) { - fail("Shouldn't fail here"); - } + @Override + public void onFailure(Exception failure) { + fail("Shouldn't fail here"); + } - @Override - public Releasable captureResponseHeaders() { + @Override + public Releasable captureResponseHeaders() { + return null; + } + }; + executor.execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(state, List.of(context), () -> null)); return null; - } - }; - - task.execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(state, List.of(context), () -> null)); - - return null; - }).when(clusterService).submitStateUpdateTask(anyString(), any(), any(), any()); + }).when(taskQueue).submitTask(anyString(), any(), any()); + } + return taskQueue; + }); } public void testOperatorControllerFromJSONContent() throws IOException { ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); ClusterService clusterService = mock(ClusterService.class); - final ClusterName clusterName = new ClusterName("elasticsearch"); - - ClusterState state = ClusterState.builder(clusterName).build(); - when(clusterService.state()).thenReturn(state); + setupTaskMock(clusterService); var repositoriesService = mock(RepositoriesService.class); @@ -350,8 +353,6 @@ public void testOperatorControllerFromJSONContent() throws IOException { ) ); - setupTaskMock(clusterService, state); - try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, testJSON)) { controller.process("operator", parser, (e) -> { if (e != null) { diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportRollupAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportRollupAction.java index a793cddc8e421..118395202ed5e 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportRollupAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/downsample/TransportRollupAction.java @@ -26,7 +26,6 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.SimpleBatchedExecutor; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -37,6 +36,7 @@ import org.elasticsearch.cluster.metadata.MetadataCreateIndexService; import org.elasticsearch.cluster.routing.allocation.allocator.AllocationActionListener; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; @@ -92,6 +92,7 @@ public class TransportRollupAction extends AcknowledgedTransportMasterNodeAction private final Client client; private final IndicesService indicesService; private final ClusterService clusterService; + private final MasterServiceTaskQueue taskQueue; private final MetadataCreateIndexService metadataCreateIndexService; private final IndexScopedSettings indexScopedSettings; private final ThreadContext threadContext; @@ -140,6 +141,7 @@ public TransportRollupAction( this.metadataCreateIndexService = metadataCreateIndexService; this.indexScopedSettings = indexScopedSettings; this.threadContext = threadPool.getThreadContext(); + this.taskQueue = clusterService.getTaskQueue("rollup", Priority.URGENT, STATE_UPDATE_TASK_EXECUTOR); } @Override @@ -642,7 +644,7 @@ private void createRollupIndex( rollupIndexName ).settings(builder.build()).mappings(mapping); var delegate = new AllocationActionListener<>(listener, threadPool.getThreadContext()); - clusterService.submitStateUpdateTask("create-rollup-index [" + rollupIndexName + "]", new RollupClusterStateUpdateTask(listener) { + taskQueue.submitTask("create-rollup-index [" + rollupIndexName + "]", new RollupClusterStateUpdateTask(listener) { @Override public ClusterState execute(ClusterState currentState) throws Exception { return metadataCreateIndexService.applyCreateIndexRequest( @@ -654,7 +656,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { delegate.reroute() ); } - }, ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), STATE_UPDATE_TASK_EXECUTOR); + }, request.masterNodeTimeout()); } private void updateRollupMetadata( @@ -663,30 +665,25 @@ private void updateRollupMetadata( ActionListener listener ) { // 6. Mark rollup index as "completed successfully" ("index.rollup.status": "success") - clusterService.submitStateUpdateTask( - "update-rollup-metadata [" + rollupIndexName + "]", - new RollupClusterStateUpdateTask(listener) { - - @Override - public ClusterState execute(ClusterState currentState) { - Metadata metadata = currentState.metadata(); - Metadata.Builder metadataBuilder = Metadata.builder(metadata); - Index rollupIndex = metadata.index(rollupIndexName).getIndex(); - IndexMetadata rollupIndexMetadata = metadata.index(rollupIndex); - - metadataBuilder.updateSettings( - Settings.builder() - .put(rollupIndexMetadata.getSettings()) - .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), IndexMetadata.DownsampleTaskStatus.SUCCESS) - .build(), - rollupIndexName - ); - return ClusterState.builder(currentState).metadata(metadataBuilder.build()).build(); - } - }, - ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()), - STATE_UPDATE_TASK_EXECUTOR - ); + taskQueue.submitTask("update-rollup-metadata [" + rollupIndexName + "]", new RollupClusterStateUpdateTask(listener) { + + @Override + public ClusterState execute(ClusterState currentState) { + Metadata metadata = currentState.metadata(); + Metadata.Builder metadataBuilder = Metadata.builder(metadata); + Index rollupIndex = metadata.index(rollupIndexName).getIndex(); + IndexMetadata rollupIndexMetadata = metadata.index(rollupIndex); + + metadataBuilder.updateSettings( + Settings.builder() + .put(rollupIndexMetadata.getSettings()) + .put(IndexMetadata.INDEX_DOWNSAMPLE_STATUS.getKey(), IndexMetadata.DownsampleTaskStatus.SUCCESS) + .build(), + rollupIndexName + ); + return ClusterState.builder(currentState).metadata(metadataBuilder.build()).build(); + } + }, request.masterNodeTimeout()); } private void refreshIndex(String index, TaskId parentTask, ActionListener listener) { diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeAction.java index 40c18618cab9b..086fdd949294e 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeAction.java @@ -15,7 +15,6 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -26,6 +25,7 @@ import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.tasks.Task; @@ -41,7 +41,7 @@ public class TransportDeleteShutdownNodeAction extends AcknowledgedTransportMasterNodeAction { private static final Logger logger = LogManager.getLogger(TransportDeleteShutdownNodeAction.class); - private final DeleteShutdownNodeExecutor executor = new DeleteShutdownNodeExecutor(); + private final MasterServiceTaskQueue taskQueue; private static boolean deleteShutdownNodeState(Map shutdownMetadata, Request request) { if (shutdownMetadata.containsKey(request.getNodeId()) == false) { @@ -124,6 +124,7 @@ public TransportDeleteShutdownNodeAction( indexNameExpressionResolver, ThreadPool.Names.SAME ); + taskQueue = clusterService.getTaskQueue("delete-node-shutdown", Priority.URGENT, new DeleteShutdownNodeExecutor()); } @Override @@ -135,10 +136,11 @@ protected void masterOperation(Task task, Request request, ClusterState state, A throw new ResourceNotFoundException("node [" + request.getNodeId() + "] is not currently shutting down"); } } - - var deleteTask = new DeleteShutdownNodeTask(request, listener); - var taskConfig = ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()); - clusterService.submitStateUpdateTask("delete-node-shutdown-" + request.getNodeId(), deleteTask, taskConfig, executor); + taskQueue.submitTask( + "delete-node-shutdown-" + request.getNodeId(), + new DeleteShutdownNodeTask(request, listener), + request.masterNodeTimeout() + ); } @Override diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java index 590447459fd79..860ba408b75c0 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -25,6 +24,7 @@ import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.common.Priority; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.tasks.Task; @@ -42,6 +42,8 @@ public class TransportPutShutdownNodeAction extends AcknowledgedTransportMasterNodeAction { private static final Logger logger = LogManager.getLogger(TransportPutShutdownNodeAction.class); + private final MasterServiceTaskQueue taskQueue; + private final PutShutdownNodeExecutor executor = new PutShutdownNodeExecutor(); private static boolean putShutdownNodeState( @@ -163,6 +165,7 @@ public TransportPutShutdownNodeAction( indexNameExpressionResolver, ThreadPool.Names.SAME ); + taskQueue = clusterService.getTaskQueue("put-shutdown", Priority.URGENT, new PutShutdownNodeExecutor()); } @Override @@ -171,9 +174,11 @@ protected void masterOperation(Task task, Request request, ClusterState state, A listener.onResponse(AcknowledgedResponse.TRUE); return; } - var updateTask = new PutShutdownNodeTask(request, listener); - var taskConfig = ClusterStateTaskConfig.build(Priority.URGENT, request.masterNodeTimeout()); - clusterService.submitStateUpdateTask("put-node-shutdown-" + request.getNodeId(), updateTask, taskConfig, executor); + taskQueue.submitTask( + "put-node-shutdown-" + request.getNodeId(), + new PutShutdownNodeTask(request, listener), + request.masterNodeTimeout() + ); } private static boolean isNoop(Map shutdownMetadata, Request request) { diff --git a/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeActionTests.java b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeActionTests.java index a29b5784f618d..a6e1123dd8552 100644 --- a/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeActionTests.java +++ b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportDeleteShutdownNodeActionTests.java @@ -10,7 +10,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskExecutor.TaskContext; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -18,6 +17,7 @@ import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -26,6 +26,7 @@ import org.junit.Before; import org.mockito.ArgumentCaptor; import org.mockito.Mock; +import org.mockito.Mockito; import org.mockito.MockitoAnnotations; import java.util.List; @@ -46,6 +47,9 @@ public class TransportDeleteShutdownNodeActionTests extends ESTestCase { @Mock private TaskContext taskContext; + @Mock + private MasterServiceTaskQueue taskQueue; + @Before public void init() { MockitoAnnotations.openMocks(this); @@ -55,6 +59,9 @@ public void init() { clusterService = mock(ClusterService.class); var actionFilters = mock(ActionFilters.class); var indexNameExpressionResolver = mock(IndexNameExpressionResolver.class); + when(clusterService.getTaskQueue(any(), any(), Mockito.>any())).thenReturn( + taskQueue + ); action = new TransportDeleteShutdownNodeAction( transportService, clusterService, @@ -73,9 +80,9 @@ public void testNoop() throws Exception { var request = new DeleteShutdownNodeAction.Request("node1"); action.masterOperation(null, request, clusterStateWithShutdown, ActionListener.noop()); var updateTask = ArgumentCaptor.forClass(DeleteShutdownNodeTask.class); - var taskConfig = ArgumentCaptor.forClass(ClusterStateTaskConfig.class); var taskExecutor = ArgumentCaptor.forClass(DeleteShutdownNodeExecutor.class); - verify(clusterService).submitStateUpdateTask(any(), updateTask.capture(), taskConfig.capture(), taskExecutor.capture()); + verify(clusterService).getTaskQueue(any(), any(), taskExecutor.capture()); + verify(taskQueue).submitTask(any(), updateTask.capture(), any()); when(taskContext.getTask()).thenReturn(updateTask.getValue()); ClusterState gotState = taskExecutor.getValue() .execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(ClusterState.EMPTY_STATE, List.of(taskContext), () -> null)); diff --git a/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeActionTests.java b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeActionTests.java index dd4258d37840f..1afeba5a9362c 100644 --- a/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeActionTests.java +++ b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeActionTests.java @@ -10,12 +10,12 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateTaskExecutor.TaskContext; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata.Type; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -25,6 +25,7 @@ import org.junit.Before; import org.mockito.ArgumentCaptor; import org.mockito.Mock; +import org.mockito.Mockito; import org.mockito.MockitoAnnotations; import java.util.List; @@ -46,6 +47,9 @@ public class TransportPutShutdownNodeActionTests extends ESTestCase { @Mock private TaskContext taskContext; + @Mock + private MasterServiceTaskQueue taskQueue; + @Before public void init() { MockitoAnnotations.openMocks(this); @@ -55,6 +59,7 @@ public void init() { clusterService = mock(ClusterService.class); var actionFilters = mock(ActionFilters.class); var indexNameExpressionResolver = mock(IndexNameExpressionResolver.class); + when(clusterService.getTaskQueue(any(), any(), Mockito.>any())).thenReturn(taskQueue); action = new TransportPutShutdownNodeAction( transportService, clusterService, @@ -71,24 +76,29 @@ public void testNoop() throws Exception { var request = new PutShutdownNodeAction.Request("node1", type, "sunsetting", allocationDelay, targetNodeName); action.masterOperation(null, request, ClusterState.EMPTY_STATE, ActionListener.noop()); var updateTask = ArgumentCaptor.forClass(PutShutdownNodeTask.class); - var taskConfig = ArgumentCaptor.forClass(ClusterStateTaskConfig.class); var taskExecutor = ArgumentCaptor.forClass(PutShutdownNodeExecutor.class); - verify(clusterService).submitStateUpdateTask(any(), updateTask.capture(), taskConfig.capture(), taskExecutor.capture()); + verify(clusterService).getTaskQueue(any(), any(), taskExecutor.capture()); + verify(taskQueue).submitTask(any(), updateTask.capture(), any()); when(taskContext.getTask()).thenReturn(updateTask.getValue()); ClusterState stableState = taskExecutor.getValue() .execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(ClusterState.EMPTY_STATE, List.of(taskContext), () -> null)); // run the request again, there should be no call to submit an update task - clearInvocations(clusterService); + clearTaskQueueInvocations(); action.masterOperation(null, request, stableState, ActionListener.noop()); - verifyNoInteractions(clusterService); + verifyNoInteractions(taskQueue); // run the request again with empty state, the update task should return the same state action.masterOperation(null, request, ClusterState.EMPTY_STATE, ActionListener.noop()); - verify(clusterService).submitStateUpdateTask(any(), updateTask.capture(), taskConfig.capture(), taskExecutor.capture()); + verify(taskQueue).submitTask(any(), updateTask.capture(), any()); when(taskContext.getTask()).thenReturn(updateTask.getValue()); ClusterState gotState = taskExecutor.getValue() .execute(new ClusterStateTaskExecutor.BatchExecutionContext<>(stableState, List.of(taskContext), () -> null)); assertThat(gotState, sameInstance(stableState)); } + + @SuppressWarnings("unchecked") + private void clearTaskQueueInvocations() { + clearInvocations(taskQueue); + } }