-
Notifications
You must be signed in to change notification settings - Fork 25.6k
Zen2: Deterministic MasterService #32493
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 8 commits
29e6bc8
2efb267
82b366c
2f8b153
80de420
8cc6d9e
dfce0a8
4b41773
506608c
05c9455
40d7c95
d3c5a3d
526511d
c84ddf7
b9d407f
e4bd482
6bc69f6
a0030c0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |
| import org.apache.logging.log4j.Logger; | ||
| import org.apache.logging.log4j.message.ParameterizedMessage; | ||
| import org.elasticsearch.Assertions; | ||
| import org.elasticsearch.action.ActionListener; | ||
| import org.elasticsearch.cluster.AckedClusterStateTaskListener; | ||
| import org.elasticsearch.cluster.ClusterChangedEvent; | ||
| import org.elasticsearch.cluster.ClusterState; | ||
|
|
@@ -48,7 +49,9 @@ | |
| import org.elasticsearch.common.util.concurrent.FutureUtils; | ||
| import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; | ||
| import org.elasticsearch.common.util.concurrent.ThreadContext; | ||
| import org.elasticsearch.discovery.ClusterStatePublisher; | ||
| import org.elasticsearch.discovery.Discovery; | ||
| import org.elasticsearch.discovery.FailedToCommitClusterStateException; | ||
| import org.elasticsearch.threadpool.ThreadPool; | ||
|
|
||
| import java.util.Arrays; | ||
|
|
@@ -57,9 +60,9 @@ | |
| import java.util.Locale; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.function.BiConsumer; | ||
| import java.util.function.Supplier; | ||
| import java.util.stream.Collectors; | ||
|
|
||
|
|
@@ -70,7 +73,7 @@ public class MasterService extends AbstractLifecycleComponent { | |
|
|
||
| public static final String MASTER_UPDATE_THREAD_NAME = "masterService#updateTask"; | ||
|
|
||
| private BiConsumer<ClusterChangedEvent, Discovery.AckListener> clusterStatePublisher; | ||
| protected ClusterStatePublisher clusterStatePublisher; | ||
|
|
||
| private java.util.function.Supplier<ClusterState> clusterStateSupplier; | ||
|
|
||
|
|
@@ -92,7 +95,7 @@ public void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) { | |
| this.slowTaskLoggingThreshold = slowTaskLoggingThreshold; | ||
| } | ||
|
|
||
| public synchronized void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) { | ||
| public synchronized void setClusterStatePublisher(ClusterStatePublisher publisher) { | ||
| clusterStatePublisher = publisher; | ||
| } | ||
|
|
||
|
|
@@ -104,12 +107,16 @@ public synchronized void setClusterStateSupplier(java.util.function.Supplier<Clu | |
| 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 = EsExecutors.newSinglePrioritizing( | ||
| threadPoolExecutor = createThreadPoolExecutor(); | ||
| taskBatcher = new Batcher(logger, threadPoolExecutor); | ||
| } | ||
|
|
||
| protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { | ||
| return EsExecutors.newSinglePrioritizing( | ||
| nodeName() + "/" + MASTER_UPDATE_THREAD_NAME, | ||
| daemonThreadFactory(settings, MASTER_UPDATE_THREAD_NAME), | ||
| threadPool.getThreadContext(), | ||
| threadPool.scheduler()); | ||
| taskBatcher = new Batcher(logger, threadPoolExecutor); | ||
| } | ||
|
|
||
| class Batcher extends TaskBatcher { | ||
|
|
@@ -222,46 +229,87 @@ protected void runTasks(TaskInputs taskInputs) { | |
| } | ||
|
|
||
| logger.debug("publishing cluster state version [{}]", newClusterState.version()); | ||
| try { | ||
| clusterStatePublisher.accept(clusterChangedEvent, taskOutputs.createAckListener(threadPool, newClusterState)); | ||
| } catch (Discovery.FailedToCommitClusterStateException t) { | ||
| final long version = newClusterState.version(); | ||
| logger.warn(() -> new ParameterizedMessage( | ||
| "failing [{}]: failed to commit cluster state version [{}]", summary, version), t); | ||
| taskOutputs.publishingFailed(t); | ||
| return; | ||
| } | ||
| publish(clusterChangedEvent, taskOutputs, startTimeNS); | ||
| } catch (Exception e) { | ||
| handleException(summary, startTimeNS, newClusterState, e); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| protected void publish(ClusterChangedEvent clusterChangedEvent, TaskOutputs taskOutputs, long startTimeNS) throws Exception { | ||
| CompletableFuture<Void> fut = new CompletableFuture<>(); | ||
| clusterStatePublisher.publish(clusterChangedEvent, new ActionListener<Void>() { | ||
|
||
| @Override | ||
| public void onResponse(Void aVoid) { | ||
| fut.complete(aVoid); | ||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(Exception e) { | ||
| fut.completeExceptionally(e); | ||
| } | ||
| }, taskOutputs.createAckListener(threadPool, clusterChangedEvent.state())); | ||
|
|
||
| final ActionListener<Void> publishListener = getPublishListener(clusterChangedEvent, taskOutputs, startTimeNS); | ||
|
||
| // indefinitely wait for publication to complete | ||
| try { | ||
| FutureUtils.get(fut); | ||
| publishListener.onResponse(null); | ||
| } catch (Exception e) { | ||
| publishListener.onFailure(e); | ||
| } | ||
| } | ||
|
|
||
| taskOutputs.processedDifferentClusterState(previousClusterState, newClusterState); | ||
| protected ActionListener<Void> getPublishListener(ClusterChangedEvent clusterChangedEvent, TaskOutputs taskOutputs, long startTimeNS) { | ||
| return new ActionListener<Void>() { | ||
|
|
||
| @Override | ||
| public void onResponse(Void ignore) { | ||
| taskOutputs.processedDifferentClusterState(clusterChangedEvent.previousState(), clusterChangedEvent.state()); | ||
|
|
||
| try { | ||
| taskOutputs.clusterStatePublished(clusterChangedEvent); | ||
| } catch (Exception e) { | ||
| logger.error(() -> new ParameterizedMessage( | ||
| "exception thrown while notifying executor of new cluster state publication [{}]", | ||
| summary), e); | ||
| "exception thrown while notifying executor of new cluster state publication [{}]", | ||
| clusterChangedEvent.source()), e); | ||
| } | ||
| TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); | ||
| logger.debug("processing [{}]: took [{}] done publishing updated cluster state (version: {}, uuid: {})", summary, | ||
| executionTime, newClusterState.version(), | ||
| newClusterState.stateUUID()); | ||
| warnAboutSlowTaskIfNeeded(executionTime, summary); | ||
| } catch (Exception e) { | ||
| TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); | ||
| final long version = newClusterState.version(); | ||
| final String stateUUID = newClusterState.stateUUID(); | ||
| final String fullState = newClusterState.toString(); | ||
| logger.warn(() -> new ParameterizedMessage( | ||
| "failed to publish updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}", | ||
| executionTime, | ||
| version, | ||
| stateUUID, | ||
| summary, | ||
| fullState), | ||
| e); | ||
| // TODO: do we want to call updateTask.onFailure here? | ||
| logger.debug("processing [{}]: took [{}] done publishing updated cluster state (version: {}, uuid: {})", | ||
| clusterChangedEvent.source(), | ||
| executionTime, clusterChangedEvent.state().version(), | ||
| clusterChangedEvent.state().stateUUID()); | ||
| warnAboutSlowTaskIfNeeded(executionTime, clusterChangedEvent.source()); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(Exception exception) { | ||
| if (exception instanceof FailedToCommitClusterStateException) { | ||
| final long version = clusterChangedEvent.state().version(); | ||
| logger.warn(() -> new ParameterizedMessage( | ||
| "failing [{}]: failed to commit cluster state version [{}]", clusterChangedEvent.source(), version), exception); | ||
| taskOutputs.publishingFailed((FailedToCommitClusterStateException) exception); | ||
| } else { | ||
| handleException(clusterChangedEvent.source(), startTimeNS, clusterChangedEvent.state(), exception); | ||
| } | ||
| } | ||
| }; | ||
| } | ||
|
|
||
| private void handleException(String summary, long startTimeNS, ClusterState newClusterState, Exception e) { | ||
| TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); | ||
| final long version = newClusterState.version(); | ||
| final String stateUUID = newClusterState.stateUUID(); | ||
| final String fullState = newClusterState.toString(); | ||
| logger.warn(() -> new ParameterizedMessage( | ||
| "failed to publish updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}", | ||
| executionTime, | ||
| version, | ||
| stateUUID, | ||
| summary, | ||
| fullState), | ||
| e); | ||
| // TODO: do we want to call updateTask.onFailure here? | ||
| } | ||
|
|
||
| public TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, long startTimeNS) { | ||
|
|
@@ -276,7 +324,7 @@ private ClusterState patchVersions(ClusterState previousClusterState, ClusterTas | |
|
|
||
| if (previousClusterState != newClusterState) { | ||
| // only the master controls the version numbers | ||
| Builder builder = ClusterState.builder(newClusterState).incrementVersion(); | ||
| Builder builder = incrementVersion(newClusterState); | ||
| if (previousClusterState.routingTable() != newClusterState.routingTable()) { | ||
| builder.routingTable(RoutingTable.builder(newClusterState.routingTable()) | ||
| .version(newClusterState.routingTable().version() + 1).build()); | ||
|
|
@@ -291,6 +339,10 @@ private ClusterState patchVersions(ClusterState previousClusterState, ClusterTas | |
| return newClusterState; | ||
| } | ||
|
|
||
| public Builder incrementVersion(ClusterState clusterState) { | ||
|
||
| return ClusterState.builder(clusterState).incrementVersion(); | ||
| } | ||
|
|
||
| /** | ||
| * Submits a cluster state update task; unlike {@link #submitStateUpdateTask(String, Object, ClusterStateTaskConfig, | ||
| * ClusterStateTaskExecutor, ClusterStateTaskListener)}, submitted updates will not be batched. | ||
|
|
@@ -335,7 +387,7 @@ public <T> void submitStateUpdateTask(String source, T task, | |
| /** | ||
| * Output created by executing a set of tasks provided as TaskInputs | ||
| */ | ||
| class TaskOutputs { | ||
| protected class TaskOutputs { | ||
| public final TaskInputs taskInputs; | ||
| public final ClusterState previousClusterState; | ||
| public final ClusterState newClusterState; | ||
|
|
@@ -353,7 +405,7 @@ class TaskOutputs { | |
| this.executionResults = executionResults; | ||
| } | ||
|
|
||
| public void publishingFailed(Discovery.FailedToCommitClusterStateException t) { | ||
| public void publishingFailed(FailedToCommitClusterStateException t) { | ||
| nonFailedTasks.forEach(task -> task.listener.onFailure(task.source(), t)); | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Almost all of these registrations use the fully-qualified class name (except
CoordinationStateRejectedException, oops) so it looks like this should too.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
right, also fixed for
CoordinationStateRejectedExceptionin d3c5a3d