-
Notifications
You must be signed in to change notification settings - Fork 24.3k
/
MasterService.java
1652 lines (1471 loc) · 71.5 KB
/
MasterService.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.ThreadedActionListener;
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.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.NotMasterException;
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.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;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.Releasable;
import org.elasticsearch.core.Releasables;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.node.Node;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskAwareRequest;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskManager;
import org.elasticsearch.threadpool.Scheduler;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList;
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.ExecutorService;
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;
public class MasterService extends AbstractLifecycleComponent {
private static final Logger logger = LogManager.getLogger(MasterService.class);
public static final Setting<TimeValue> MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING = Setting.positiveTimeSetting(
"cluster.service.slow_master_task_logging_threshold",
TimeValue.timeValueSeconds(10),
Setting.Property.Dynamic,
Setting.Property.NodeScope
);
public static final Setting<TimeValue> MASTER_SERVICE_STARVATION_LOGGING_THRESHOLD_SETTING = Setting.positiveTimeSetting(
"cluster.service.master_service_starvation_logging_threshold",
TimeValue.timeValueMinutes(5),
Setting.Property.NodeScope
);
static final String MASTER_UPDATE_THREAD_NAME = "masterService#updateTask";
public static final String STATE_UPDATE_ACTION_NAME = "publish_cluster_state_update";
private final ClusterStateTaskExecutor<ClusterStateUpdateTask> unbatchedExecutor;
private ClusterStatePublisher clusterStatePublisher;
private Supplier<ClusterState> clusterStateSupplier;
private final String nodeName;
private volatile TimeValue slowTaskLoggingThreshold;
private final TimeValue starvationLoggingThreshold;
protected final ThreadPool threadPool;
private final TaskManager taskManager;
private volatile ExecutorService threadPoolExecutor;
private final AtomicInteger totalQueueSize = new AtomicInteger();
private volatile Batch currentlyExecutingBatch;
private final Map<Priority, PerPriorityQueue> 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));
this.slowTaskLoggingThreshold = MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, this::setSlowTaskLoggingThreshold);
this.starvationLoggingThreshold = MASTER_SERVICE_STARVATION_LOGGING_THRESHOLD_SETTING.get(settings);
this.threadPool = threadPool;
this.taskManager = taskManager;
final var queuesByPriorityBuilder = new EnumMap<Priority, PerPriorityQueue>(Priority.class);
for (final var priority : Priority.values()) {
queuesByPriorityBuilder.put(priority, new PerPriorityQueue(priority));
}
this.queuesByPriority = Collections.unmodifiableMap(queuesByPriorityBuilder);
this.unbatchedExecutor = new UnbatchedExecutor();
}
private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
}
public synchronized void setClusterStatePublisher(ClusterStatePublisher publisher) {
clusterStatePublisher = publisher;
}
public synchronized void setClusterStateSupplier(Supplier<ClusterState> clusterStateSupplier) {
this.clusterStateSupplier = clusterStateSupplier;
}
@Override
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();
}
protected ExecutorService createThreadPoolExecutor() {
return EsExecutors.newScaling(
nodeName + "/" + MASTER_UPDATE_THREAD_NAME,
0,
1,
60,
TimeUnit.SECONDS,
true,
daemonThreadFactory(nodeName, MASTER_UPDATE_THREAD_NAME),
threadPool.getThreadContext()
);
}
public ClusterStateUpdateStats getClusterStateUpdateStats() {
return clusterStateUpdateStatsTracker.getStatistics();
}
@Override
protected synchronized void doStop() {
ThreadPool.terminate(threadPoolExecutor, 10, TimeUnit.SECONDS);
}
@Override
protected synchronized void doClose() {}
/**
* The current cluster state exposed by the discovery layer. Package-visible for tests.
*/
ClusterState state() {
return clusterStateSupplier.get();
}
public static boolean isMasterUpdateThread() {
return Thread.currentThread().getName().contains('[' + MASTER_UPDATE_THREAD_NAME + ']');
}
public static boolean assertMasterUpdateOrTestThread() {
return ThreadPool.assertCurrentThreadPool(MASTER_UPDATE_THREAD_NAME);
}
public static boolean assertNotMasterUpdateThread(String reason) {
assert isMasterUpdateThread() == false
: "Expected current thread [" + Thread.currentThread() + "] to not be the master service thread. Reason: [" + reason + "]";
return true;
}
private <T extends ClusterStateTaskListener> void executeAndPublishBatch(
final ClusterStateTaskExecutor<T> executor,
final List<ExecutionResult<T>> executionResults,
final BatchSummary summary,
final ActionListener<Void> listener
) {
if (lifecycle.started() == false) {
logger.debug("processing [{}]: ignoring, master service not started", summary);
listener.onResponse(null);
return;
}
logger.debug("executing cluster state update for [{}]", summary);
final ClusterState previousClusterState = state();
if (previousClusterState.nodes().isLocalNodeElectedMaster() == false && executor.runOnlyOnMaster()) {
logger.debug("failing [{}]: local node is no longer master", summary);
for (ExecutionResult<T> executionResult : executionResults) {
executionResult.onBatchFailure(new NotMasterException("no longer master"));
executionResult.notifyFailure();
}
listener.onResponse(null);
return;
}
final long computationStartTime = threadPool.rawRelativeTimeInMillis();
final var newClusterState = patchVersions(
previousClusterState,
executeTasks(previousClusterState, executionResults, executor, summary, threadPool.getThreadContext())
);
final TimeValue computationTime = getTimeSince(computationStartTime);
logExecutionTime(computationTime, "compute cluster state update", summary);
if (previousClusterState == newClusterState) {
final long notificationStartTime = threadPool.rawRelativeTimeInMillis();
for (final var executionResult : executionResults) {
final var contextPreservingAckListener = executionResult.getContextPreservingAckListener();
if (contextPreservingAckListener != null) {
// no need to wait for ack if nothing changed, the update can be counted as acknowledged
contextPreservingAckListener.onAckSuccess();
}
executionResult.onClusterStateUnchanged(newClusterState);
}
final TimeValue executionTime = getTimeSince(notificationStartTime);
logExecutionTime(executionTime, "notify listeners on unchanged cluster state", summary);
clusterStateUpdateStatsTracker.onUnchangedClusterState(computationTime.millis(), executionTime.millis());
listener.onResponse(null);
} else {
final long publicationStartTime = threadPool.rawRelativeTimeInMillis();
try (var ignored = threadPool.getThreadContext().newTraceContext()) {
final Task task = taskManager.register("master", STATE_UPDATE_ACTION_NAME, new TaskAwareRequest() {
@Override
public void setParentTask(TaskId taskId) {}
@Override
public TaskId getParentTask() {
return TaskId.EMPTY_TASK_ID;
}
@Override
public String getDescription() {
return "publication of cluster state [" + newClusterState.getVersion() + "]";
}
});
ActionListener.run(ActionListener.runAfter(listener, () -> taskManager.unregister(task)).delegateResponse((l, e) -> {
assert publicationMayFail() : e;
handleException(summary, publicationStartTime, newClusterState, e);
l.onResponse(null);
}),
l -> publishClusterStateUpdate(
executor,
summary,
previousClusterState,
executionResults,
newClusterState,
computationTime,
publicationStartTime,
task,
l
)
);
}
}
}
private <T extends ClusterStateTaskListener> void publishClusterStateUpdate(
ClusterStateTaskExecutor<T> executor,
BatchSummary summary,
ClusterState previousClusterState,
List<ExecutionResult<T>> executionResults,
ClusterState newClusterState,
TimeValue computationTime,
long publicationStartTime,
Task task,
ActionListener<Void> listener
) {
if (logger.isTraceEnabled()) {
logger.trace("cluster state updated, source [{}]\n{}", summary, newClusterState);
} else {
logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), summary);
}
final ClusterStatePublicationEvent clusterStatePublicationEvent = new ClusterStatePublicationEvent(
summary,
previousClusterState,
newClusterState,
task,
computationTime.millis(),
publicationStartTime
);
// new cluster state, notify all listeners
final DiscoveryNodes.Delta nodesDelta = newClusterState.nodes().delta(previousClusterState.nodes());
if (nodesDelta.hasChanges() && logger.isInfoEnabled()) {
String nodesDeltaSummary = nodesDelta.shortSummary();
if (nodesDeltaSummary.length() > 0) {
logger.info(
"{}, term: {}, version: {}, delta: {}",
summary,
newClusterState.term(),
newClusterState.version(),
nodesDeltaSummary
);
}
}
logger.debug("publishing cluster state version [{}]", newClusterState.version());
// initialize routing nodes and the indices lookup concurrently, we will need both of them for the cluster state
// application and can compute them while we wait for the other nodes during publication
newClusterState.initializeAsync(threadPool.generic());
publish(
clusterStatePublicationEvent,
new CompositeTaskAckListener(
executionResults.stream()
.map(ExecutionResult::getContextPreservingAckListener)
.filter(Objects::nonNull)
.map(
contextPreservingAckListener -> new TaskAckListener(
contextPreservingAckListener,
newClusterState.version(),
newClusterState.nodes(),
threadPool
)
)
.toList()
),
ActionListener.runAfter(new ActionListener<>() {
@Override
public void onResponse(Void unused) {
final long notificationStartTime = threadPool.rawRelativeTimeInMillis();
for (final var executionResult : executionResults) {
executionResult.onPublishSuccess(newClusterState);
}
try {
executor.clusterStatePublished(newClusterState);
} catch (Exception e) {
logger.error(
() -> format("exception thrown while notifying executor of new cluster state publication [%s]", summary),
e
);
}
final TimeValue executionTime = getTimeSince(notificationStartTime);
logExecutionTime(
executionTime,
"notify listeners on successful publication of cluster state (version: "
+ newClusterState.version()
+ ", uuid: "
+ newClusterState.stateUUID()
+ ')',
summary
);
clusterStateUpdateStatsTracker.onPublicationSuccess(
threadPool.rawRelativeTimeInMillis(),
clusterStatePublicationEvent,
executionTime.millis()
);
}
@Override
public void onFailure(Exception exception) {
if (exception instanceof FailedToCommitClusterStateException failedToCommitClusterStateException) {
final long notificationStartTime = threadPool.rawRelativeTimeInMillis();
final long version = newClusterState.version();
logger.warn(() -> format("failing [%s]: failed to commit cluster state version [%s]", summary, version), exception);
for (final var executionResult : executionResults) {
executionResult.onPublishFailure(failedToCommitClusterStateException);
}
final long notificationMillis = threadPool.rawRelativeTimeInMillis() - notificationStartTime;
clusterStateUpdateStatsTracker.onPublicationFailure(
threadPool.rawRelativeTimeInMillis(),
clusterStatePublicationEvent,
notificationMillis
);
} else {
assert publicationMayFail() || (exception instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown())
: exception;
clusterStateUpdateStatsTracker.onPublicationFailure(
threadPool.rawRelativeTimeInMillis(),
clusterStatePublicationEvent,
0L
);
handleException(summary, publicationStartTime, newClusterState, exception);
}
}
@Override
public String toString() {
return Strings.format(
"publication completion listener for version [%d]",
clusterStatePublicationEvent.getNewState().version()
);
}
}, new Runnable() {
@Override
public void run() {
listener.onResponse(null);
}
@Override
public String toString() {
return listener + "/onResponse";
}
})
);
}
protected boolean publicationMayFail() {
return false;
}
private TimeValue getTimeSince(long startTimeMillis) {
return TimeValue.timeValueMillis(Math.max(0, threadPool.rawRelativeTimeInMillis() - startTimeMillis));
}
protected void publish(
ClusterStatePublicationEvent clusterStatePublicationEvent,
ClusterStatePublisher.AckListener ackListener,
ActionListener<Void> publicationListener
) {
clusterStatePublisher.publish(
clusterStatePublicationEvent,
// Fork the completion of publicationListener back onto the master service thread, mainly for legacy reasons; note that this
// might be rejected if the MasterService shut down mid-publication. The master service thread remains idle until this listener
// is completed at the end of the publication, at which point the publicationListener performs various bits of cleanup and then
// picks up the next waiting task.
new ThreadedActionListener<>(
threadPoolExecutor,
new ContextPreservingActionListener<>(threadPool.getThreadContext().newRestorableContext(false), publicationListener)
),
ackListener
);
}
private void handleException(BatchSummary summary, long startTimeMillis, ClusterState newClusterState, Exception e) {
logger.warn(
() -> format(
"took [%s] and then failed to publish updated cluster state (version: %s, uuid: %s) for [%s]:\n%s",
getTimeSince(startTimeMillis),
newClusterState.version(),
newClusterState.stateUUID(),
summary,
newClusterState
),
e
);
}
private ClusterState patchVersions(ClusterState previousClusterState, ClusterState newClusterState) {
if (previousClusterState != newClusterState) {
// only the master controls the version numbers
Builder builder = incrementVersion(newClusterState);
if (previousClusterState.routingTable() != newClusterState.routingTable()) {
builder.routingTable(newClusterState.routingTable().withIncrementedVersion());
}
if (previousClusterState.metadata() != newClusterState.metadata()) {
builder.metadata(newClusterState.metadata().withIncrementedVersion());
}
final var previousMetadata = newClusterState.metadata();
newClusterState = builder.build();
assert previousMetadata.sameIndicesLookup(newClusterState.metadata());
}
return newClusterState;
}
public Builder incrementVersion(ClusterState clusterState) {
return ClusterState.builder(clusterState).incrementVersion();
}
/**
* Submits an unbatched cluster state update task. This method exists for legacy reasons but is deprecated and forbidden in new
* production code because unbatched tasks are a source of performance and stability bugs. You should instead implement your update
* logic in a dedicated {@link ClusterStateTaskExecutor} which is reused across multiple task instances. The task itself is typically
* just a collection of parameters consumed by the executor, together with any listeners to be notified when execution completes.
*
* @param source the source of the cluster state update task
* @param updateTask the full context for the cluster state update
*/
@Deprecated
public void submitUnbatchedStateUpdateTask(String source, ClusterStateUpdateTask updateTask) {
createTaskQueue("unbatched", updateTask.priority(), unbatchedExecutor).submitTask(source, updateTask, updateTask.timeout());
}
private static class UnbatchedExecutor implements ClusterStateTaskExecutor<ClusterStateUpdateTask> {
@Override
@SuppressForbidden(reason = "consuming published cluster state for legacy reasons")
public ClusterState execute(BatchExecutionContext<ClusterStateUpdateTask> batchExecutionContext) throws Exception {
assert batchExecutionContext.taskContexts().size() == 1
: "this only supports a single task but received " + batchExecutionContext.taskContexts();
final var taskContext = batchExecutionContext.taskContexts().get(0);
final var task = taskContext.getTask();
final ClusterState newState;
try (var ignored = taskContext.captureResponseHeaders()) {
newState = task.execute(batchExecutionContext.initialState());
}
final Consumer<ClusterState> publishListener = publishedState -> task.clusterStateProcessed(
batchExecutionContext.initialState(),
publishedState
);
if (task instanceof ClusterStateAckListener ackListener) {
taskContext.success(publishListener, ackListener);
} else {
taskContext.success(publishListener);
}
return newState;
}
@Override
public String describeTasks(List<ClusterStateUpdateTask> tasks) {
return ""; // one task, so the source is enough
}
}
/**
* Returns the tasks that are pending.
*/
public List<PendingClusterTask> pendingTasks() {
final var currentTimeMillis = threadPool.relativeTimeInMillis();
return allBatchesStream().flatMap(e -> e.getPending(currentTimeMillis)).toList();
}
/**
* Returns the number of currently pending tasks.
*/
public int numberOfPendingTasks() {
return allBatchesStream().mapToInt(Batch::getPendingCount).sum();
}
/**
* Returns the maximum wait time for tasks in the queue
*
* @return A zero time value if the queue is empty, otherwise the time value oldest task waiting in the queue
*/
public TimeValue getMaxTaskWaitTime() {
final var oldestTaskTimeMillis = allBatchesStream().mapToLong(Batch::getCreationTimeMillis).min().orElse(Long.MAX_VALUE);
if (oldestTaskTimeMillis == Long.MAX_VALUE) {
return TimeValue.ZERO;
}
return TimeValue.timeValueMillis(threadPool.relativeTimeInMillis() - oldestTaskTimeMillis);
}
private Stream<Batch> allBatchesStream() {
return Stream.concat(
Stream.ofNullable(currentlyExecutingBatch),
queuesByPriority.values().stream().filter(Objects::nonNull).flatMap(q -> q.queue.stream())
);
}
private void logExecutionTime(TimeValue executionTime, String activity, BatchSummary summary) {
if (executionTime.getMillis() > slowTaskLoggingThreshold.getMillis()) {
logger.warn(
"took [{}/{}ms] to {} for [{}], which exceeds the warn threshold of [{}]",
executionTime,
executionTime.getMillis(),
activity,
summary,
slowTaskLoggingThreshold
);
} else {
logger.debug("took [{}] to {} for [{}]", executionTime, activity, summary);
}
}
/**
* A wrapper around a {@link ClusterStateAckListener} which restores the given thread context before delegating to the inner listener's
* callbacks, and also logs and swallows any exceptions thrown. One of these is created for each task in the batch that passes a
* {@link ClusterStateAckListener} to {@link ClusterStateTaskExecutor.TaskContext#success}.
*/
private record ContextPreservingAckListener(
ClusterStateAckListener listener,
Supplier<ThreadContext.StoredContext> context,
Runnable restoreResponseHeaders
) {
public boolean mustAck(DiscoveryNode discoveryNode) {
return listener.mustAck(discoveryNode);
}
public void onAckSuccess() {
try (ThreadContext.StoredContext ignore = context.get()) {
restoreResponseHeaders.run();
listener.onAllNodesAcked();
} catch (Exception inner) {
logger.error("exception thrown by listener while notifying on all nodes acked", inner);
}
}
public void onAckFailure(@Nullable Exception e) {
try (ThreadContext.StoredContext ignore = context.get()) {
restoreResponseHeaders.run();
listener.onAckFailure(e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error("exception thrown by listener while notifying on all nodes acked or failed", inner);
}
}
public void onAckTimeout() {
try (ThreadContext.StoredContext ignore = context.get()) {
restoreResponseHeaders.run();
listener.onAckTimeout();
} catch (Exception e) {
logger.error("exception thrown by listener while notifying on ack timeout", e);
}
}
public TimeValue ackTimeout() {
return listener.ackTimeout();
}
}
/**
* A wrapper around a {@link ContextPreservingAckListener} which keeps track of acks received during publication and notifies the inner
* listener when sufficiently many have been received. One of these is created for each {@link ContextPreservingAckListener} once the
* state for publication has been computed.
*/
private static class TaskAckListener {
private final ContextPreservingAckListener contextPreservingAckListener;
private final CountDown countDown;
private final DiscoveryNode masterNode;
private final ThreadPool threadPool;
private final long clusterStateVersion;
private volatile Scheduler.Cancellable ackTimeoutCallback;
private Exception lastFailure;
TaskAckListener(
ContextPreservingAckListener contextPreservingAckListener,
long clusterStateVersion,
DiscoveryNodes nodes,
ThreadPool threadPool
) {
this.contextPreservingAckListener = contextPreservingAckListener;
this.clusterStateVersion = clusterStateVersion;
this.threadPool = threadPool;
this.masterNode = nodes.getMasterNode();
int countDown = 0;
for (DiscoveryNode node : nodes) {
// we always wait for at least the master node
if (node.equals(masterNode) || contextPreservingAckListener.mustAck(node)) {
countDown++;
}
}
logger.trace("expecting {} acknowledgements for cluster_state update (version: {})", countDown, clusterStateVersion);
this.countDown = new CountDown(countDown + 1); // we also wait for onCommit to be called
}
public void onCommit(TimeValue commitTime) {
TimeValue ackTimeout = contextPreservingAckListener.ackTimeout();
if (ackTimeout == null) {
ackTimeout = TimeValue.ZERO;
}
final TimeValue timeLeft = TimeValue.timeValueNanos(Math.max(0, ackTimeout.nanos() - commitTime.nanos()));
if (timeLeft.nanos() == 0L) {
onTimeout();
} else if (countDown.countDown()) {
finish();
} else {
this.ackTimeoutCallback = threadPool.schedule(this::onTimeout, timeLeft, ThreadPool.Names.GENERIC);
// re-check if onNodeAck has not completed while we were scheduling the timeout
if (countDown.isCountedDown()) {
ackTimeoutCallback.cancel();
}
}
}
public void onNodeAck(DiscoveryNode node, @Nullable Exception e) {
if (node.equals(masterNode) == false && contextPreservingAckListener.mustAck(node) == false) {
return;
}
if (e == null) {
logger.trace("ack received from node [{}], cluster_state update (version: {})", node, clusterStateVersion);
} else {
this.lastFailure = e;
logger.debug(() -> format("ack received from node [%s], cluster_state update (version: %s)", node, clusterStateVersion), e);
}
if (countDown.countDown()) {
finish();
}
}
private void finish() {
logger.trace("all expected nodes acknowledged cluster_state update (version: {})", clusterStateVersion);
if (ackTimeoutCallback != null) {
ackTimeoutCallback.cancel();
}
final var failure = lastFailure;
if (failure == null) {
contextPreservingAckListener.onAckSuccess();
} else {
contextPreservingAckListener.onAckFailure(failure);
}
}
public void onTimeout() {
if (countDown.fastForward()) {
logger.trace("timeout waiting for acknowledgement for cluster_state update (version: {})", clusterStateVersion);
contextPreservingAckListener.onAckTimeout();
}
}
}
/**
* A wrapper around the collection of {@link TaskAckListener}s for a publication.
*/
private record CompositeTaskAckListener(List<TaskAckListener> listeners) implements ClusterStatePublisher.AckListener {
@Override
public void onCommit(TimeValue commitTime) {
for (TaskAckListener listener : listeners) {
listener.onCommit(commitTime);
}
}
@Override
public void onNodeAck(DiscoveryNode node, @Nullable Exception e) {
for (TaskAckListener listener : listeners) {
listener.onNodeAck(node, e);
}
}
}
private static class ExecutionResult<T extends ClusterStateTaskListener> implements ClusterStateTaskExecutor.TaskContext<T> {
private final String source;
private final T task;
private final ThreadContext threadContext;
private final Supplier<ThreadContext.StoredContext> threadContextSupplier;
@Nullable // if the task is incomplete or failed or onPublicationSuccess supplied
Consumer<ClusterState> publishedStateConsumer;
@Nullable // if the task is incomplete or failed or publishedStateConsumer supplied
Runnable onPublicationSuccess;
@Nullable // if the task is incomplete or failed or doesn't listen for acks
ClusterStateAckListener clusterStateAckListener;
@Nullable // if the task is incomplete or succeeded
Exception failure;
@Nullable
Map<String, List<String>> responseHeaders;
ExecutionResult(String source, T task, ThreadContext threadContext, Supplier<ThreadContext.StoredContext> threadContextSupplier) {
this.source = source;
this.task = task;
this.threadContext = threadContext;
this.threadContextSupplier = threadContextSupplier;
}
public String getSource() {
return source;
}
@Override
public T getTask() {
return task;
}
private boolean incomplete() {
assert assertMasterUpdateOrTestThread();
return publishedStateConsumer == null && onPublicationSuccess == null && failure == null;
}
// [HISTORICAL NOTE] In the past, tasks executed by the master service would automatically be notified of acks if they implemented
// the ClusterStateAckListener interface (the interface formerly known as AckedClusterStateTaskListener). This implicit behaviour
// was a little troublesome and was removed in favour of having the executor explicitly register an ack listener (where necessary)
// for each task it successfully executes. Making this change carried the risk that someone might implement a new task in the future
// which relied on the old implicit behaviour based on the interfaces that the task implements instead of the explicit behaviour in
// the executor. We protect against this with some weird-looking assertions in the success() methods below which insist that
// ack-listening tasks register themselves as their own ack listener. If you want to supply a different ack listener then you must
// remove the ClusterStateAckListener interface from the task to make it clear that the task itself is not expecting to be notified
// of acks.
//
// Note that the old implicit behaviour lives on in the unbatched() executor so that it can correctly execute either a
// ClusterStateUpdateTask or an AckedClusterStateUpdateTask.
@Override
public void success(Runnable onPublicationSuccess) {
assert getTask() instanceof ClusterStateAckListener == false // see [HISTORICAL NOTE] above
: "tasks that implement ClusterStateAckListener must explicitly supply themselves as the ack listener";
assert incomplete();
this.onPublicationSuccess = Objects.requireNonNull(onPublicationSuccess);
}
@Override
public void success(Consumer<ClusterState> publishListener) {
assert getTask() instanceof ClusterStateAckListener == false // see [HISTORICAL NOTE] above
: "tasks that implement ClusterStateAckListener must explicitly supply themselves as the ack listener";
assert incomplete();
this.publishedStateConsumer = Objects.requireNonNull(publishListener);
}
@Override
public void success(Runnable onPublicationSuccess, ClusterStateAckListener clusterStateAckListener) {
assert getTask() == clusterStateAckListener || getTask() instanceof ClusterStateAckListener == false
// see [HISTORICAL NOTE] above
: "tasks that implement ClusterStateAckListener must not supply a separate clusterStateAckListener";
assert incomplete();
this.onPublicationSuccess = Objects.requireNonNull(onPublicationSuccess);
this.clusterStateAckListener = Objects.requireNonNull(clusterStateAckListener);
}
@Override
public void success(Consumer<ClusterState> publishListener, ClusterStateAckListener clusterStateAckListener) {
assert getTask() == clusterStateAckListener || getTask() instanceof ClusterStateAckListener == false
// see [HISTORICAL NOTE] above
: "tasks that implement ClusterStateAckListener must not supply a separate clusterStateAckListener";
assert incomplete();
this.publishedStateConsumer = Objects.requireNonNull(publishListener);
this.clusterStateAckListener = Objects.requireNonNull(clusterStateAckListener);
}
@Override
public void onFailure(Exception failure) {
assert incomplete();
this.failure = Objects.requireNonNull(failure);
}
@Override
public Releasable captureResponseHeaders() {
final var storedContext = threadContext.newStoredContext();
return Releasables.wrap(() -> {
final var newResponseHeaders = threadContext.getResponseHeaders();
if (newResponseHeaders.isEmpty()) {
return;
}
if (responseHeaders == null) {
responseHeaders = new HashMap<>(newResponseHeaders);
} else {
for (final var newResponseHeader : newResponseHeaders.entrySet()) {
responseHeaders.compute(newResponseHeader.getKey(), (ignored, oldValue) -> {
if (oldValue == null) {
return newResponseHeader.getValue();
}
return CollectionUtils.concatLists(oldValue, newResponseHeader.getValue());
});
}
}
}, storedContext);
}
private void restoreResponseHeaders() {
if (responseHeaders != null) {
for (final var responseHeader : responseHeaders.entrySet()) {
for (final var value : responseHeader.getValue()) {
threadContext.addResponseHeader(responseHeader.getKey(), value);
}
}
}
}
void onBatchFailure(Exception failure) {
// if the whole batch resulted in an exception then this overrides any task-level results whether successful or not
this.failure = Objects.requireNonNull(failure);
this.publishedStateConsumer = null;
this.clusterStateAckListener = null;
}
void onPublishSuccess(ClusterState newClusterState) {
if (publishedStateConsumer == null && onPublicationSuccess == null) {
notifyFailure();
return;
}
try (ThreadContext.StoredContext ignored = threadContextSupplier.get()) {
restoreResponseHeaders();
if (onPublicationSuccess == null) {
publishedStateConsumer.accept(newClusterState);
} else {
onPublicationSuccess.run();
}
} catch (Exception e) {
logger.error("exception thrown by listener while notifying of new cluster state", e);
}
}
void onClusterStateUnchanged(ClusterState clusterState) {
if (publishedStateConsumer == null && onPublicationSuccess == null) {
notifyFailure();
return;
}
try (ThreadContext.StoredContext ignored = threadContextSupplier.get()) {
restoreResponseHeaders();
if (onPublicationSuccess == null) {
publishedStateConsumer.accept(clusterState);
} else {
onPublicationSuccess.run();
}
} catch (Exception e) {
logger.error("exception thrown by listener while notifying of unchanged cluster state", e);
}
}
void onPublishFailure(FailedToCommitClusterStateException e) {
if (publishedStateConsumer == null && onPublicationSuccess == null) {
assert failure != null;
var taskFailure = failure;
failure = new FailedToCommitClusterStateException(e.getMessage(), e);
failure.addSuppressed(taskFailure);
notifyFailure();
return;
}
try (ThreadContext.StoredContext ignored = threadContextSupplier.get()) {
restoreResponseHeaders();
getTask().onFailure(e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.error("exception thrown by listener notifying of failure", inner);
}
}
void notifyFailure() {
assert 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);
}
}
ContextPreservingAckListener getContextPreservingAckListener() {
assert incomplete() == false;
if (clusterStateAckListener == null || failure != null) {
return null;
} else {
return new ContextPreservingAckListener(clusterStateAckListener, threadContextSupplier, this::restoreResponseHeaders);
}
}
@Override
public String toString() {
return "ExecutionResult[" + task + "]";
}
}
private static <T extends ClusterStateTaskListener> ClusterState executeTasks(
ClusterState previousClusterState,
List<ExecutionResult<T>> executionResults,
ClusterStateTaskExecutor<T> executor,
BatchSummary summary,
ThreadContext threadContext
) {
final var resultingState = innerExecuteTasks(previousClusterState, executionResults, executor, summary, threadContext);
if (previousClusterState != resultingState
&& previousClusterState.nodes().isLocalNodeElectedMaster()
&& (resultingState.nodes().isLocalNodeElectedMaster() == false)) {
throw new AssertionError("update task submitted to MasterService cannot remove master");
}
assert assertAllTasksComplete(executionResults);
return resultingState;
}
private static <T extends ClusterStateTaskListener> boolean assertAllTasksComplete(List<ExecutionResult<T>> executionResults) {
for (final var executionResult : executionResults) {
assert executionResult.incomplete() == false : "missing result for " + executionResult;
}
return true;
}
private static <T extends ClusterStateTaskListener> ClusterState innerExecuteTasks(
ClusterState previousClusterState,
List<ExecutionResult<T>> executionResults,
ClusterStateTaskExecutor<T> executor,
BatchSummary summary,
ThreadContext threadContext
) {
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
try {