/
StreamTask.java
1449 lines (1210 loc) · 50.6 KB
/
StreamTask.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
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.streaming.runtime.tasks;
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.core.fs.FileSystemSafetyNet;
import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.SimpleCounter;
import org.apache.flink.runtime.checkpoint.CheckpointException;
import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.CheckpointOptions;
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
import org.apache.flink.runtime.execution.CancelTaskException;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.runtime.state.StateBackendLoader;
import org.apache.flink.runtime.state.TaskStateManager;
import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.runtime.util.FatalExitExceptionHandler;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.graph.StreamEdge;
import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer;
import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl;
import org.apache.flink.streaming.runtime.io.InputStatus;
import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
import org.apache.flink.streaming.runtime.io.StreamInputProcessor;
import org.apache.flink.streaming.runtime.partitioner.ConfigurableStreamPartitioner;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox;
import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext;
import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor;
import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorFactory;
import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor;
import org.apache.flink.streaming.runtime.tasks.mailbox.execution.SuspendedMailboxDefaultAction;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.OptionalLong;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicReference;
/**
* Base class for all streaming tasks. A task is the unit of local processing that is deployed
* and executed by the TaskManagers. Each task runs one or more {@link StreamOperator}s which form
* the Task's operator chain. Operators that are chained together execute synchronously in the
* same thread and hence on the same stream partition. A common case for these chains
* are successive map/flatmap/filter tasks.
*
* <p>The task chain contains one "head" operator and multiple chained operators.
* The StreamTask is specialized for the type of the head operator: one-input and two-input tasks,
* as well as for sources, iteration heads and iteration tails.
*
* <p>The Task class deals with the setup of the streams read by the head operator, and the streams
* produced by the operators at the ends of the operator chain. Note that the chain may fork and
* thus have multiple ends.
*
* <p>The life cycle of the task is set up as follows:
* <pre>{@code
* -- setInitialState -> provides state of all operators in the chain
*
* -- invoke()
* |
* +----> Create basic utils (config, etc) and load the chain of operators
* +----> operators.setup()
* +----> task specific init()
* +----> initialize-operator-states()
* +----> open-operators()
* +----> run()
* +----> close-operators()
* +----> dispose-operators()
* +----> common cleanup
* +----> task specific cleanup()
* }</pre>
*
* <p>The {@code StreamTask} has a lock object called {@code lock}. All calls to methods on a
* {@code StreamOperator} must be synchronized on this lock object to ensure that no methods
* are called concurrently.
*
* @param <OUT>
* @param <OP>
*/
@Internal
public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
extends AbstractInvokable
implements AsyncExceptionHandler {
/** The thread group that holds all trigger timer threads. */
public static final ThreadGroup TRIGGER_THREAD_GROUP = new ThreadGroup("Triggers");
/** The logger used by the StreamTask and its subclasses. */
protected static final Logger LOG = LoggerFactory.getLogger(StreamTask.class);
// ------------------------------------------------------------------------
/**
* All interaction with the {@code StreamOperator} must be synchronized on this lock object to
* ensure that we don't have concurrent method calls that void consistent checkpoints.
*/
private final Object lock = new Object();
/**
* The input processor. Initialized in {@link #init()} method.
*/
@Nullable
protected StreamInputProcessor inputProcessor;
/** the head operator that consumes the input streams of this task. */
protected OP headOperator;
/** The chain of operators executed by this task. */
protected OperatorChain<OUT, OP> operatorChain;
/** The configuration of this streaming task. */
protected final StreamConfig configuration;
/** Our state backend. We use this to create checkpoint streams and a keyed state backend. */
protected StateBackend stateBackend;
/** The external storage where checkpoint data is persisted. */
private CheckpointStorageWorkerView checkpointStorage;
/**
* The internal {@link ProcessingTimeService} used to define the current
* processing time (default = {@code System.currentTimeMillis()}) and
* register timers for tasks to be executed in the future.
*/
protected ProcessingTimeService timerService;
private final Thread.UncaughtExceptionHandler uncaughtExceptionHandler;
/** The map of user-defined accumulators of this task. */
private final Map<String, Accumulator<?, ?>> accumulatorMap;
/** The currently active background materialization threads. */
private final CloseableRegistry cancelables = new CloseableRegistry();
private final StreamTaskAsyncExceptionHandler asyncExceptionHandler;
/**
* Flag to mark the task "in operation", in which case check needs to be initialized to true,
* so that early cancel() before invoke() behaves correctly.
*/
private volatile boolean isRunning;
/** Flag to mark this task as canceled. */
private volatile boolean canceled;
/** Thread pool for async snapshot workers. */
private ExecutorService asyncOperationsThreadPool;
private final List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> recordWriters;
protected final MailboxProcessor mailboxProcessor;
private Long syncSavepointId = null;
// ------------------------------------------------------------------------
/**
* Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
*
* @param env The task environment for this task.
*/
protected StreamTask(Environment env) {
this(env, null);
}
/**
* Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
*
* @param env The task environment for this task.
* @param timeProvider Optionally, a specific time provider to use.
*/
protected StreamTask(Environment env, @Nullable ProcessingTimeService timeProvider) {
this(env, timeProvider, FatalExitExceptionHandler.INSTANCE);
}
/**
* Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
*
* <p>This constructor accepts a special {@link ProcessingTimeService}. By default (and if
* null is passes for the time provider) a {@link SystemProcessingTimeService DefaultTimerService}
* will be used.
*
* @param environment The task environment for this task.
* @param timeProvider Optionally, a specific time provider to use.
* @param uncaughtExceptionHandler to handle uncaught exceptions in the async operations thread pool
*/
protected StreamTask(
Environment environment,
@Nullable ProcessingTimeService timeProvider,
Thread.UncaughtExceptionHandler uncaughtExceptionHandler) {
super(environment);
this.timerService = timeProvider;
this.uncaughtExceptionHandler = Preconditions.checkNotNull(uncaughtExceptionHandler);
this.configuration = new StreamConfig(getTaskConfiguration());
this.accumulatorMap = getEnvironment().getAccumulatorRegistry().getUserMap();
this.recordWriters = createRecordWriters(configuration, environment);
this.mailboxProcessor = new MailboxProcessor(this::processInput);
this.asyncExceptionHandler = new StreamTaskAsyncExceptionHandler(environment);
}
// ------------------------------------------------------------------------
// Life cycle methods for specific implementations
// ------------------------------------------------------------------------
protected abstract void init() throws Exception;
protected void cancelTask() throws Exception {
}
protected void cleanup() throws Exception {
if (inputProcessor != null) {
inputProcessor.close();
}
}
/**
* This method implements the default action of the task (e.g. processing one event from the input). Implementations
* should (in general) be non-blocking.
*
* @param context context object for collaborative interaction between the action and the stream task.
* @throws Exception on any problems in the action.
*/
protected void processInput(DefaultActionContext context) throws Exception {
InputStatus status = inputProcessor.processInput();
if (status == InputStatus.END_OF_INPUT) {
context.allActionsCompleted();
}
else if (status == InputStatus.NOTHING_AVAILABLE) {
SuspendedMailboxDefaultAction suspendedDefaultAction = context.suspendDefaultAction();
inputProcessor.isAvailable().thenRun(suspendedDefaultAction::resume);
}
}
private void resetSynchronousSavepointId() {
syncSavepointId = null;
}
private void setSynchronousSavepointId(long checkpointId) {
Preconditions.checkState(
syncSavepointId == null, "at most one stop-with-savepoint checkpoint at a time is allowed");
syncSavepointId = checkpointId;
}
@VisibleForTesting
OptionalLong getSynchronousSavepointId() {
return syncSavepointId != null ? OptionalLong.of(syncSavepointId) : OptionalLong.empty();
}
private boolean isSynchronousSavepointId(long checkpointId) {
return syncSavepointId != null && syncSavepointId == checkpointId;
}
private void runSynchronousSavepointMailboxLoop() throws Exception {
assert syncSavepointId != null;
MailboxExecutor mailboxExecutor = mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY);
while (!canceled && syncSavepointId != null) {
mailboxExecutor.yield();
}
}
/**
* Emits the {@link org.apache.flink.streaming.api.watermark.Watermark#MAX_WATERMARK MAX_WATERMARK}
* so that all registered timers are fired.
*
* <p>This is used by the source task when the job is {@code TERMINATED}. In the case,
* we want all the timers registered throughout the pipeline to fire and the related
* state (e.g. windows) to be flushed.
*
* <p>For tasks other than the source task, this method does nothing.
*/
protected void advanceToEndOfEventTime() throws Exception {
}
/**
* Instructs the task to go through its normal termination routine, i.e. exit the run-loop
* and call {@link StreamOperator#close()} and {@link StreamOperator#dispose()} on its operators.
*
* <p>This is used by the source task to get out of the run-loop when the job is stopped with a savepoint.
*
* <p>For tasks other than the source task, this method does nothing.
*/
protected void finishTask() throws Exception {
}
// ------------------------------------------------------------------------
// Core work methods of the Stream Task
// ------------------------------------------------------------------------
public StreamTaskStateInitializer createStreamTaskStateInitializer() {
return new StreamTaskStateInitializerImpl(
getEnvironment(),
stateBackend,
timerService);
}
protected Counter setupNumRecordsInCounter(StreamOperator streamOperator) {
try {
return ((OperatorMetricGroup) streamOperator.getMetricGroup()).getIOMetricGroup().getNumRecordsInCounter();
} catch (Exception e) {
LOG.warn("An exception occurred during the metrics setup.", e);
return new SimpleCounter();
}
}
@Override
public final void invoke() throws Exception {
boolean disposed = false;
try {
// -------- Initialize ---------
LOG.debug("Initializing {}.", getName());
asyncOperationsThreadPool = Executors.newCachedThreadPool(new ExecutorThreadFactory("AsyncOperations", uncaughtExceptionHandler));
stateBackend = createStateBackend();
checkpointStorage = stateBackend.createCheckpointStorage(getEnvironment().getJobID());
// if the clock is not already set, then assign a default TimeServiceProvider
if (timerService == null) {
ThreadFactory timerThreadFactory =
new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName());
timerService = new SystemProcessingTimeService(new TimerInvocationContext(), timerThreadFactory);
}
operatorChain = new OperatorChain<>(this, recordWriters);
headOperator = operatorChain.getHeadOperator();
// check environment for selective reading
checkSelectiveReadingEnv();
// task specific initialization
init();
// save the work of reloading state, etc, if the task is already canceled
if (canceled) {
throw new CancelTaskException();
}
// -------- Invoke --------
LOG.debug("Invoking {}", getName());
// we need to make sure that any triggers scheduled in open() cannot be
// executed before all operators are opened
synchronized (lock) {
// both the following operations are protected by the lock
// so that we avoid race conditions in the case that initializeState()
// registers a timer, that fires before the open() is called.
initializeState();
openAllOperators();
}
// final check to exit early before starting to run
if (canceled) {
throw new CancelTaskException();
}
// let the task do its work
isRunning = true;
mailboxProcessor.runMailboxLoop();
// if this left the run() method cleanly despite the fact that this was canceled,
// make sure the "clean shutdown" is not attempted
if (canceled) {
throw new CancelTaskException();
}
LOG.debug("Finished task {}", getName());
// make sure no further checkpoint and notification actions happen.
// we make sure that no other thread is currently in the locked scope before
// we close the operators by trying to acquire the checkpoint scope lock
// we also need to make sure that no triggers fire concurrently with the close logic
// at the same time, this makes sure that during any "regular" exit where still
synchronized (lock) {
// this is part of the main logic, so if this fails, the task is considered failed
closeAllOperators();
// make sure no new timers can come
timerService.quiesce();
// let mailbox execution reject all new letters from this point
mailboxProcessor.prepareClose();
// only set the StreamTask to not running after all operators have been closed!
// See FLINK-7430
isRunning = false;
}
MailboxExecutor mainMailboxExecutor = mailboxProcessor.getMainMailboxExecutor();
while (mainMailboxExecutor.tryYield()) {
// Run until we have processed all remaining letters.
}
// make sure all timers finish
timerService.awaitPendingAfterQuiesce();
LOG.debug("Closed operators for task {}", getName());
// make sure all buffered data is flushed
operatorChain.flushOutputs();
// make an attempt to dispose the operators such that failures in the dispose call
// still let the computation fail
tryDisposeAllOperators();
disposed = true;
}
finally {
// clean up everything we initialized
isRunning = false;
// Now that we are outside the user code, we do not want to be interrupted further
// upon cancellation. The shutdown logic below needs to make sure it does not issue calls
// that block and stall shutdown.
// Additionally, the cancellation watch dog will issue a hard-cancel (kill the TaskManager
// process) as a backup in case some shutdown procedure blocks outside our control.
setShouldInterruptOnCancel(false);
// clear any previously issued interrupt for a more graceful shutdown
Thread.interrupted();
// stop all timers and threads
tryShutdownTimerService();
// stop all asynchronous checkpoint threads
try {
cancelables.close();
shutdownAsyncThreads();
}
catch (Throwable t) {
// catch and log the exception to not replace the original exception
LOG.error("Could not shut down async checkpoint threads", t);
}
// we must! perform this cleanup
try {
cleanup();
}
catch (Throwable t) {
// catch and log the exception to not replace the original exception
LOG.error("Error during cleanup of stream task", t);
}
// if the operators were not disposed before, do a hard dispose
if (!disposed) {
disposeAllOperators();
}
// release the output resources. this method should never fail.
if (operatorChain != null) {
// beware: without synchronization, #performCheckpoint() may run in
// parallel and this call is not thread-safe
synchronized (lock) {
operatorChain.releaseOutputs();
}
} else {
// failed to allocate operatorChain, clean up record writers
for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> writer: recordWriters) {
writer.close();
}
}
mailboxProcessor.close();
}
}
@Override
public final void cancel() throws Exception {
isRunning = false;
canceled = true;
// the "cancel task" call must come first, but the cancelables must be
// closed no matter what
try {
cancelTask();
}
finally {
mailboxProcessor.allActionsCompleted();
cancelables.close();
}
}
public MailboxExecutorFactory getMailboxExecutorFactory() {
return this.mailboxProcessor::getMailboxExecutor;
}
public final boolean isRunning() {
return isRunning;
}
public final boolean isCanceled() {
return canceled;
}
/**
* Execute {@link StreamOperator#open()} of each operator in the chain of this
* {@link StreamTask}. Opening happens from <b>tail to head</b> operator in the chain, contrary
* to {@link StreamOperator#close()} which happens <b>head to tail</b>
* (see {@link #closeAllOperators()}.
*/
private void openAllOperators() throws Exception {
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
if (operator != null) {
operator.open();
}
}
}
/**
* Execute {@link StreamOperator#close()} of each operator in the chain of this
* {@link StreamTask}. Closing happens from <b>head to tail</b> operator in the chain,
* contrary to {@link StreamOperator#open()} which happens <b>tail to head</b>
* (see {@link #openAllOperators()}.
*/
private void closeAllOperators() throws Exception {
// We need to close them first to last, since upstream operators in the chain might emit
// elements in their close methods.
StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
for (int i = allOperators.length - 1; i >= 0; i--) {
StreamOperator<?> operator = allOperators[i];
if (operator != null) {
operator.close();
}
}
}
/**
* Execute {@link StreamOperator#dispose()} of each operator in the chain of this
* {@link StreamTask}. Disposing happens from <b>tail to head</b> operator in the chain.
*/
private void tryDisposeAllOperators() throws Exception {
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
if (operator != null) {
operator.dispose();
}
}
}
private void shutdownAsyncThreads() throws Exception {
if (!asyncOperationsThreadPool.isShutdown()) {
asyncOperationsThreadPool.shutdownNow();
}
}
/**
* Execute @link StreamOperator#dispose()} of each operator in the chain of this
* {@link StreamTask}. Disposing happens from <b>tail to head</b> operator in the chain.
*
* <p>The difference with the {@link #tryDisposeAllOperators()} is that in case of an
* exception, this method catches it and logs the message.
*/
private void disposeAllOperators() {
if (operatorChain != null) {
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
try {
if (operator != null) {
operator.dispose();
}
}
catch (Throwable t) {
LOG.error("Error during disposal of stream operator.", t);
}
}
}
}
/**
* The finalize method shuts down the timer. This is a fail-safe shutdown, in case the original
* shutdown method was never called.
*
* <p>This should not be relied upon! It will cause shutdown to happen much later than if manual
* shutdown is attempted, and cause threads to linger for longer than needed.
*/
@Override
protected void finalize() throws Throwable {
super.finalize();
if (timerService != null) {
if (!timerService.isTerminated()) {
LOG.info("Timer service is shutting down.");
timerService.shutdownService();
}
}
cancelables.close();
}
boolean isSerializingTimestamps() {
TimeCharacteristic tc = configuration.getTimeCharacteristic();
return tc == TimeCharacteristic.EventTime | tc == TimeCharacteristic.IngestionTime;
}
private void checkSelectiveReadingEnv() {
boolean isCreditBased = getEnvironment()
.getTaskManagerInfo()
.getConfiguration()
.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL);
if (!isCreditBased && operatorChain.hasSelectiveReadingOperator()) {
throw new UnsupportedOperationException(
"The operator that implements the InputSelectable interface is not supported in the non-credited" +
" network mode. (please set 'taskmanager.network.credit-model' to 'true')");
}
}
// ------------------------------------------------------------------------
// Access to properties and utilities
// ------------------------------------------------------------------------
/**
* Gets the name of the task, in the form "taskname (2/5)".
* @return The name of the task.
*/
public String getName() {
return getEnvironment().getTaskInfo().getTaskNameWithSubtasks();
}
/**
* Gets the name of the task, appended with the subtask indicator and execution id.
*
* @return The name of the task, with subtask indicator and execution id.
*/
String getTaskNameWithSubtaskAndId() {
return getEnvironment().getTaskInfo().getTaskNameWithSubtasks() +
" (" + getEnvironment().getExecutionId() + ')';
}
/**
* Gets the lock object on which all operations that involve data and state mutation have to lock.
* @return The checkpoint lock object.
*/
public Object getCheckpointLock() {
return lock;
}
public CheckpointStorageWorkerView getCheckpointStorage() {
return checkpointStorage;
}
public StreamConfig getConfiguration() {
return configuration;
}
public Map<String, Accumulator<?, ?>> getAccumulatorMap() {
return accumulatorMap;
}
public StreamStatusMaintainer getStreamStatusMaintainer() {
return operatorChain;
}
RecordWriterOutput<?>[] getStreamOutputs() {
return operatorChain.getStreamOutputs();
}
// ------------------------------------------------------------------------
// Checkpoint and Restore
// ------------------------------------------------------------------------
@Override
public Future<Boolean> triggerCheckpointAsync(
CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions,
boolean advanceToEndOfEventTime) {
return mailboxProcessor.getMainMailboxExecutor().submit(() -> {
try {
// No alignment if we inject a checkpoint
CheckpointMetrics checkpointMetrics = new CheckpointMetrics()
.setBytesBufferedInAlignment(0L)
.setAlignmentDurationNanos(0L);
boolean success = performCheckpoint(checkpointMetaData, checkpointOptions, checkpointMetrics, advanceToEndOfEventTime);
if (!success) {
declineCheckpoint(checkpointMetaData.getCheckpointId());
}
return success;
} catch (Exception e) {
// propagate exceptions only if the task is still in "running" state
if (isRunning) {
Exception exception = new Exception("Could not perform checkpoint " + checkpointMetaData.getCheckpointId() +
" for operator " + getName() + '.', e);
handleCheckpointException(exception);
throw exception;
} else {
LOG.debug("Could not perform checkpoint {} for operator {} while the " +
"invokable was not in state running.", checkpointMetaData.getCheckpointId(), getName(), e);
return false;
}
}
});
}
@Override
public void triggerCheckpointOnBarrier(
CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions,
CheckpointMetrics checkpointMetrics) throws Exception {
try {
if (performCheckpoint(checkpointMetaData, checkpointOptions, checkpointMetrics, false)) {
if (isSynchronousSavepointId(checkpointMetaData.getCheckpointId())) {
runSynchronousSavepointMailboxLoop();
}
}
}
catch (CancelTaskException e) {
LOG.info("Operator {} was cancelled while performing checkpoint {}.",
getName(), checkpointMetaData.getCheckpointId());
throw e;
}
catch (Exception e) {
throw new Exception("Could not perform checkpoint " + checkpointMetaData.getCheckpointId() + " for operator " +
getName() + '.', e);
}
}
@Override
public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) throws Exception {
LOG.debug("Aborting checkpoint via cancel-barrier {} for task {}", checkpointId, getName());
// notify the coordinator that we decline this checkpoint
getEnvironment().declineCheckpoint(checkpointId, cause);
// notify all downstream operators that they should not wait for a barrier from us
synchronized (lock) {
operatorChain.broadcastCheckpointCancelMarker(checkpointId);
}
}
private boolean performCheckpoint(
CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions,
CheckpointMetrics checkpointMetrics,
boolean advanceToEndOfTime) throws Exception {
LOG.debug("Starting checkpoint ({}) {} on task {}",
checkpointMetaData.getCheckpointId(), checkpointOptions.getCheckpointType(), getName());
final long checkpointId = checkpointMetaData.getCheckpointId();
synchronized (lock) {
if (isRunning) {
if (checkpointOptions.getCheckpointType().isSynchronous()) {
setSynchronousSavepointId(checkpointId);
if (advanceToEndOfTime) {
advanceToEndOfEventTime();
}
}
// All of the following steps happen as an atomic step from the perspective of barriers and
// records/watermarks/timers/callbacks.
// We generally try to emit the checkpoint barrier as soon as possible to not affect downstream
// checkpoint alignments
// Step (1): Prepare the checkpoint, allow operators to do some pre-barrier work.
// The pre-barrier work should be nothing or minimal in the common case.
operatorChain.prepareSnapshotPreBarrier(checkpointId);
// Step (2): Send the checkpoint barrier downstream
operatorChain.broadcastCheckpointBarrier(
checkpointId,
checkpointMetaData.getTimestamp(),
checkpointOptions);
// Step (3): Take the state snapshot. This should be largely asynchronous, to not
// impact progress of the streaming topology
checkpointState(checkpointMetaData, checkpointOptions, checkpointMetrics);
return true;
}
else {
// we cannot perform our checkpoint - let the downstream operators know that they
// should not wait for any input from this operator
// we cannot broadcast the cancellation markers on the 'operator chain', because it may not
// yet be created
final CancelCheckpointMarker message = new CancelCheckpointMarker(checkpointMetaData.getCheckpointId());
Exception exception = null;
for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> recordWriter : recordWriters) {
try {
recordWriter.broadcastEvent(message);
} catch (Exception e) {
exception = ExceptionUtils.firstOrSuppressed(
new Exception("Could not send cancel checkpoint marker to downstream tasks.", e),
exception);
}
}
if (exception != null) {
throw exception;
}
return false;
}
}
}
protected void declineCheckpoint(long checkpointId) {
getEnvironment().declineCheckpoint(
checkpointId,
new CheckpointException("Task Name" + getName(), CheckpointFailureReason.CHECKPOINT_DECLINED_TASK_NOT_READY));
}
protected void handleCheckpointException(Exception exception) {
handleException(exception);
}
public ExecutorService getAsyncOperationsThreadPool() {
return asyncOperationsThreadPool;
}
@Override
public Future<Void> notifyCheckpointCompleteAsync(long checkpointId) {
return mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).submit(() -> {
try {
boolean success = false;
synchronized (lock) {
if (isRunning) {
LOG.debug("Notification of complete checkpoint for task {}", getName());
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
if (operator != null) {
operator.notifyCheckpointComplete(checkpointId);
}
}
success = true;
} else {
LOG.debug("Ignoring notification of complete checkpoint for not-running task {}", getName());
}
}
getEnvironment().getTaskStateManager().notifyCheckpointComplete(checkpointId);
if (success && isSynchronousSavepointId(checkpointId)) {
finishTask();
// Reset to "notify" the internal synchronous savepoint mailbox loop.
resetSynchronousSavepointId();
}
return null;
} catch (Exception e) {
handleException(new RuntimeException(
"Error while confirming checkpoint",
e));
throw e;
}
});
}
private void tryShutdownTimerService() {
if (timerService != null && !timerService.isTerminated()) {
try {
final long timeoutMs = getEnvironment().getTaskManagerInfo().getConfiguration().
getLong(TaskManagerOptions.TASK_CANCELLATION_TIMEOUT_TIMERS);
if (!timerService.shutdownServiceUninterruptible(timeoutMs)) {
LOG.warn("Timer service shutdown exceeded time limit of {} ms while waiting for pending " +
"timers. Will continue with shutdown procedure.", timeoutMs);
}
} catch (Throwable t) {
// catch and log the exception to not replace the original exception
LOG.error("Could not shut down timer service", t);
}
}
}
private void checkpointState(
CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions,
CheckpointMetrics checkpointMetrics) throws Exception {
CheckpointStreamFactory storage = checkpointStorage.resolveCheckpointStorageLocation(
checkpointMetaData.getCheckpointId(),
checkpointOptions.getTargetLocation());
CheckpointingOperation checkpointingOperation = new CheckpointingOperation(
this,
checkpointMetaData,
checkpointOptions,
storage,
checkpointMetrics);
checkpointingOperation.executeCheckpointing();
}
private void initializeState() throws Exception {
StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
for (StreamOperator<?> operator : allOperators) {
if (null != operator) {
operator.initializeState();
}
}
}
// ------------------------------------------------------------------------
// State backend
// ------------------------------------------------------------------------
private StateBackend createStateBackend() throws Exception {
final StateBackend fromApplication = configuration.getStateBackend(getUserCodeClassLoader());
return StateBackendLoader.fromApplicationOrConfigOrDefault(
fromApplication,
getEnvironment().getTaskManagerInfo().getConfiguration(),
getUserCodeClassLoader(),
LOG);
}
/**
* Returns the {@link ProcessingTimeService} responsible for telling the current
* processing time and registering timers.
*/
public ProcessingTimeService getProcessingTimeService() {
if (timerService == null) {
throw new IllegalStateException("The timer service has not been initialized.");
}
return timerService;
}
/**
* Handles an exception thrown by another thread (e.g. a TriggerTask),
* other than the one executing the main task by failing the task entirely.
*
* <p>In more detail, it marks task execution failed for an external reason
* (a reason other than the task code itself throwing an exception). If the task
* is already in a terminal state (such as FINISHED, CANCELED, FAILED), or if the
* task is already canceling this does nothing. Otherwise it sets the state to
* FAILED, and, if the invokable code is running, starts an asynchronous thread
* that aborts that code.
*
* <p>This method never blocks.
*/
@Override
public void handleAsyncException(String message, Throwable exception) {
if (isRunning) {