-
Notifications
You must be signed in to change notification settings - Fork 13k
/
Task.java
1821 lines (1571 loc) · 77.9 KB
/
Task.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.runtime.taskmanager;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.TaskInfo;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.fs.FileSystemSafetyNet;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.security.FlinkSecurityManager;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
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.CheckpointOptions;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
import org.apache.flink.runtime.execution.CancelTaskException;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.JobInformation;
import org.apache.flink.runtime.executiongraph.TaskInformation;
import org.apache.flink.runtime.externalresource.ExternalResourceInfoProvider;
import org.apache.flink.runtime.filecache.FileCache;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.network.NettyShuffleEnvironment;
import org.apache.flink.runtime.io.network.TaskEventDispatcher;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider;
import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointableTask;
import org.apache.flink.runtime.jobgraph.tasks.CoordinatedTask;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.runtime.jobgraph.tasks.TaskInvokable;
import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.shuffle.ShuffleEnvironment;
import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext;
import org.apache.flink.runtime.state.TaskStateManager;
import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager;
import org.apache.flink.runtime.taskexecutor.KvStateService;
import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker;
import org.apache.flink.runtime.taskexecutor.slot.TaskSlotPayload;
import org.apache.flink.runtime.throughput.ThroughputCalculator;
import org.apache.flink.types.Either;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FatalExitExceptionHandler;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;
import org.apache.flink.util.TaskManagerExceptionUtils;
import org.apache.flink.util.UserCodeClassLoader;
import org.apache.flink.util.WrappingRuntimeException;
import org.apache.flink.util.clock.SystemClock;
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.flink.util.function.RunnableWithException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import java.util.function.Consumer;
import static org.apache.flink.configuration.TaskManagerOptions.BUFFER_DEBLOAT_SAMPLES;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;
/**
* The Task represents one execution of a parallel subtask on a TaskManager. A Task wraps a Flink
* operator (which may be a user function) and runs it, providing all services necessary for example
* to consume input data, produce its results (intermediate result partitions) and communicate with
* the JobManager.
*
* <p>The Flink operators (implemented as subclasses of {@link TaskInvokable} have only data
* readers, writers, and certain event callbacks. The task connects those to the network stack and
* actor messages, and tracks the state of the execution and handles exceptions.
*
* <p>Tasks have no knowledge about how they relate to other tasks, or whether they are the first
* attempt to execute the task, or a repeated attempt. All of that is only known to the JobManager.
* All the task knows are its own runnable code, the task's configuration, and the IDs of the
* intermediate results to consume and produce (if any).
*
* <p>Each Task is run by one dedicated thread.
*/
public class Task
implements Runnable, TaskSlotPayload, TaskActions, PartitionProducerStateProvider {
/** The class logger. */
private static final Logger LOG = LoggerFactory.getLogger(Task.class);
/** The thread group that contains all task threads. */
private static final ThreadGroup TASK_THREADS_GROUP = new ThreadGroup("Flink Task Threads");
/** For atomic state updates. */
private static final AtomicReferenceFieldUpdater<Task, ExecutionState> STATE_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(
Task.class, ExecutionState.class, "executionState");
// ------------------------------------------------------------------------
// Constant fields that are part of the initial Task construction
// ------------------------------------------------------------------------
/** The job that the task belongs to. */
private final JobID jobId;
/** The vertex in the JobGraph whose code the task executes. */
private final JobVertexID vertexId;
/** The execution attempt of the parallel subtask. */
private final ExecutionAttemptID executionId;
/** ID which identifies the slot in which the task is supposed to run. */
private final AllocationID allocationId;
/** TaskInfo object for this task. */
private final TaskInfo taskInfo;
/** The name of the task, including subtask indexes. */
private final String taskNameWithSubtask;
/** The job-wide configuration object. */
private final Configuration jobConfiguration;
/** The task-specific configuration. */
private final Configuration taskConfiguration;
/** The jar files used by this task. */
private final Collection<PermanentBlobKey> requiredJarFiles;
/** The classpaths used by this task. */
private final Collection<URL> requiredClasspaths;
/** The name of the class that holds the invokable code. */
private final String nameOfInvokableClass;
/** Access to task manager configuration and host names. */
private final TaskManagerRuntimeInfo taskManagerConfig;
/** The memory manager to be used by this task. */
private final MemoryManager memoryManager;
/** The I/O manager to be used by this task. */
private final IOManager ioManager;
/** The BroadcastVariableManager to be used by this task. */
private final BroadcastVariableManager broadcastVariableManager;
private final TaskEventDispatcher taskEventDispatcher;
/** Information provider for external resources. */
private final ExternalResourceInfoProvider externalResourceInfoProvider;
/** The manager for state of operators running in this task/slot. */
private final TaskStateManager taskStateManager;
/**
* Serialized version of the job specific execution configuration (see {@link ExecutionConfig}).
*/
private final SerializedValue<ExecutionConfig> serializedExecutionConfig;
private final ResultPartitionWriter[] consumableNotifyingPartitionWriters;
private final IndexedInputGate[] inputGates;
/** Connection to the task manager. */
private final TaskManagerActions taskManagerActions;
/** Input split provider for the task. */
private final InputSplitProvider inputSplitProvider;
/** Checkpoint notifier used to communicate with the CheckpointCoordinator. */
private final CheckpointResponder checkpointResponder;
/**
* The gateway for operators to send messages to the operator coordinators on the Job Manager.
*/
private final TaskOperatorEventGateway operatorCoordinatorEventGateway;
/** GlobalAggregateManager used to update aggregates on the JobMaster. */
private final GlobalAggregateManager aggregateManager;
/** The library cache, from which the task can request its class loader. */
private final LibraryCacheManager.ClassLoaderHandle classLoaderHandle;
/** The cache for user-defined files that the invokable requires. */
private final FileCache fileCache;
/** The service for kvState registration of this task. */
private final KvStateService kvStateService;
/** The registry of this task which enables live reporting of accumulators. */
private final AccumulatorRegistry accumulatorRegistry;
/** The thread that executes the task. */
private final Thread executingThread;
/** Parent group for all metrics of this task. */
private final TaskMetricGroup metrics;
/** Partition producer state checker to request partition states from. */
private final PartitionProducerStateChecker partitionProducerStateChecker;
/** Executor to run future callbacks. */
private final Executor executor;
/** Future that is completed once {@link #run()} exits. */
private final CompletableFuture<ExecutionState> terminationFuture = new CompletableFuture<>();
// ------------------------------------------------------------------------
// Fields that control the task execution. All these fields are volatile
// (which means that they introduce memory barriers), to establish
// proper happens-before semantics on parallel modification
// ------------------------------------------------------------------------
/** atomic flag that makes sure the invokable is canceled exactly once upon error. */
private final AtomicBoolean invokableHasBeenCanceled;
/**
* The invokable of this task, if initialized. All accesses must copy the reference and check
* for null, as this field is cleared as part of the disposal logic.
*/
@Nullable private volatile TaskInvokable invokable;
/** The current execution state of the task. */
private volatile ExecutionState executionState = ExecutionState.CREATED;
/** The observed exception, in case the task execution failed. */
private volatile Throwable failureCause;
/** Initialized from the Flink configuration. May also be set at the ExecutionConfig */
private long taskCancellationInterval;
/** Initialized from the Flink configuration. May also be set at the ExecutionConfig */
private long taskCancellationTimeout;
/**
* This class loader should be set as the context class loader for threads that may dynamically
* load user code.
*/
private UserCodeClassLoader userCodeClassLoader;
/** The only one throughput meter per subtask. */
private ThroughputCalculator throughputCalculator;
/**
* <b>IMPORTANT:</b> This constructor may not start any work that would need to be undone in the
* case of a failing task deployment.
*/
public Task(
JobInformation jobInformation,
TaskInformation taskInformation,
ExecutionAttemptID executionAttemptID,
AllocationID slotAllocationId,
int subtaskIndex,
int attemptNumber,
List<ResultPartitionDeploymentDescriptor> resultPartitionDeploymentDescriptors,
List<InputGateDeploymentDescriptor> inputGateDeploymentDescriptors,
MemoryManager memManager,
IOManager ioManager,
ShuffleEnvironment<?, ?> shuffleEnvironment,
KvStateService kvStateService,
BroadcastVariableManager bcVarManager,
TaskEventDispatcher taskEventDispatcher,
ExternalResourceInfoProvider externalResourceInfoProvider,
TaskStateManager taskStateManager,
TaskManagerActions taskManagerActions,
InputSplitProvider inputSplitProvider,
CheckpointResponder checkpointResponder,
TaskOperatorEventGateway operatorCoordinatorEventGateway,
GlobalAggregateManager aggregateManager,
LibraryCacheManager.ClassLoaderHandle classLoaderHandle,
FileCache fileCache,
TaskManagerRuntimeInfo taskManagerConfig,
@Nonnull TaskMetricGroup metricGroup,
ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
PartitionProducerStateChecker partitionProducerStateChecker,
Executor executor) {
Preconditions.checkNotNull(jobInformation);
Preconditions.checkNotNull(taskInformation);
Preconditions.checkArgument(0 <= subtaskIndex, "The subtask index must be positive.");
Preconditions.checkArgument(0 <= attemptNumber, "The attempt number must be positive.");
this.taskInfo =
new TaskInfo(
taskInformation.getTaskName(),
taskInformation.getMaxNumberOfSubtasks(),
subtaskIndex,
taskInformation.getNumberOfSubtasks(),
attemptNumber,
String.valueOf(slotAllocationId));
this.jobId = jobInformation.getJobId();
this.vertexId = taskInformation.getJobVertexId();
this.executionId = Preconditions.checkNotNull(executionAttemptID);
this.allocationId = Preconditions.checkNotNull(slotAllocationId);
this.taskNameWithSubtask = taskInfo.getTaskNameWithSubtasks();
this.jobConfiguration = jobInformation.getJobConfiguration();
this.taskConfiguration = taskInformation.getTaskConfiguration();
this.requiredJarFiles = jobInformation.getRequiredJarFileBlobKeys();
this.requiredClasspaths = jobInformation.getRequiredClasspathURLs();
this.nameOfInvokableClass = taskInformation.getInvokableClassName();
this.serializedExecutionConfig = jobInformation.getSerializedExecutionConfig();
Configuration tmConfig = taskManagerConfig.getConfiguration();
this.taskCancellationInterval =
tmConfig.getLong(TaskManagerOptions.TASK_CANCELLATION_INTERVAL);
this.taskCancellationTimeout =
tmConfig.getLong(TaskManagerOptions.TASK_CANCELLATION_TIMEOUT);
this.memoryManager = Preconditions.checkNotNull(memManager);
this.ioManager = Preconditions.checkNotNull(ioManager);
this.broadcastVariableManager = Preconditions.checkNotNull(bcVarManager);
this.taskEventDispatcher = Preconditions.checkNotNull(taskEventDispatcher);
this.taskStateManager = Preconditions.checkNotNull(taskStateManager);
this.accumulatorRegistry = new AccumulatorRegistry(jobId, executionId);
this.inputSplitProvider = Preconditions.checkNotNull(inputSplitProvider);
this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder);
this.operatorCoordinatorEventGateway =
Preconditions.checkNotNull(operatorCoordinatorEventGateway);
this.aggregateManager = Preconditions.checkNotNull(aggregateManager);
this.taskManagerActions = checkNotNull(taskManagerActions);
this.externalResourceInfoProvider = checkNotNull(externalResourceInfoProvider);
this.classLoaderHandle = Preconditions.checkNotNull(classLoaderHandle);
this.fileCache = Preconditions.checkNotNull(fileCache);
this.kvStateService = Preconditions.checkNotNull(kvStateService);
this.taskManagerConfig = Preconditions.checkNotNull(taskManagerConfig);
this.metrics = metricGroup;
this.partitionProducerStateChecker =
Preconditions.checkNotNull(partitionProducerStateChecker);
this.executor = Preconditions.checkNotNull(executor);
// create the reader and writer structures
final String taskNameWithSubtaskAndId = taskNameWithSubtask + " (" + executionId + ')';
final ShuffleIOOwnerContext taskShuffleContext =
shuffleEnvironment.createShuffleIOOwnerContext(
taskNameWithSubtaskAndId, executionId, metrics.getIOMetricGroup());
// produced intermediate result partitions
final ResultPartitionWriter[] resultPartitionWriters =
shuffleEnvironment
.createResultPartitionWriters(
taskShuffleContext, resultPartitionDeploymentDescriptors)
.toArray(new ResultPartitionWriter[] {});
this.consumableNotifyingPartitionWriters =
ConsumableNotifyingResultPartitionWriterDecorator.decorate(
resultPartitionDeploymentDescriptors,
resultPartitionWriters,
this,
jobId,
resultPartitionConsumableNotifier);
// consumed intermediate result partitions
final IndexedInputGate[] gates =
shuffleEnvironment
.createInputGates(taskShuffleContext, this, inputGateDeploymentDescriptors)
.toArray(new IndexedInputGate[0]);
this.inputGates = new IndexedInputGate[gates.length];
this.throughputCalculator =
new ThroughputCalculator(
SystemClock.getInstance(), taskConfiguration.get(BUFFER_DEBLOAT_SAMPLES));
int counter = 0;
for (IndexedInputGate gate : gates) {
inputGates[counter++] =
new InputGateWithMetrics(
gate,
metrics.getIOMetricGroup().getNumBytesInCounter(),
throughputCalculator);
}
if (shuffleEnvironment instanceof NettyShuffleEnvironment) {
//noinspection deprecation
((NettyShuffleEnvironment) shuffleEnvironment)
.registerLegacyNetworkMetrics(
metrics.getIOMetricGroup(), resultPartitionWriters, gates);
}
invokableHasBeenCanceled = new AtomicBoolean(false);
// finally, create the executing thread, but do not start it
executingThread = new Thread(TASK_THREADS_GROUP, this, taskNameWithSubtask);
}
// ------------------------------------------------------------------------
// Accessors
// ------------------------------------------------------------------------
@Override
public JobID getJobID() {
return jobId;
}
public JobVertexID getJobVertexId() {
return vertexId;
}
@Override
public ExecutionAttemptID getExecutionId() {
return executionId;
}
@Override
public AllocationID getAllocationId() {
return allocationId;
}
public TaskInfo getTaskInfo() {
return taskInfo;
}
public Configuration getJobConfiguration() {
return jobConfiguration;
}
public Configuration getTaskConfiguration() {
return this.taskConfiguration;
}
public AccumulatorRegistry getAccumulatorRegistry() {
return accumulatorRegistry;
}
public TaskMetricGroup getMetricGroup() {
return metrics;
}
public Thread getExecutingThread() {
return executingThread;
}
@Override
public CompletableFuture<ExecutionState> getTerminationFuture() {
return terminationFuture;
}
@VisibleForTesting
long getTaskCancellationInterval() {
return taskCancellationInterval;
}
@VisibleForTesting
long getTaskCancellationTimeout() {
return taskCancellationTimeout;
}
@Nullable
@VisibleForTesting
TaskInvokable getInvokable() {
return invokable;
}
public boolean isBackPressured() {
if (invokable == null
|| consumableNotifyingPartitionWriters.length == 0
|| (executionState != ExecutionState.INITIALIZING
&& executionState != ExecutionState.RUNNING)) {
return false;
}
for (int i = 0; i < consumableNotifyingPartitionWriters.length; ++i) {
if (!consumableNotifyingPartitionWriters[i].isAvailable()) {
return true;
}
}
return false;
}
// ------------------------------------------------------------------------
// Task Execution
// ------------------------------------------------------------------------
/**
* Returns the current execution state of the task.
*
* @return The current execution state of the task.
*/
public ExecutionState getExecutionState() {
return this.executionState;
}
/**
* Checks whether the task has failed, is canceled, or is being canceled at the moment.
*
* @return True is the task in state FAILED, CANCELING, or CANCELED, false otherwise.
*/
public boolean isCanceledOrFailed() {
return executionState == ExecutionState.CANCELING
|| executionState == ExecutionState.CANCELED
|| executionState == ExecutionState.FAILED;
}
/**
* If the task has failed, this method gets the exception that caused this task to fail.
* Otherwise this method returns null.
*
* @return The exception that caused the task to fail, or null, if the task has not failed.
*/
public Throwable getFailureCause() {
return failureCause;
}
/** Starts the task's thread. */
public void startTaskThread() {
executingThread.start();
}
/** The core work method that bootstraps the task and executes its code. */
@Override
public void run() {
try {
doRun();
} finally {
terminationFuture.complete(executionState);
}
}
private void doRun() {
// ----------------------------
// Initial State transition
// ----------------------------
while (true) {
ExecutionState current = this.executionState;
if (current == ExecutionState.CREATED) {
if (transitionState(ExecutionState.CREATED, ExecutionState.DEPLOYING)) {
// success, we can start our work
break;
}
} else if (current == ExecutionState.FAILED) {
// we were immediately failed. tell the TaskManager that we reached our final state
notifyFinalState();
if (metrics != null) {
metrics.close();
}
return;
} else if (current == ExecutionState.CANCELING) {
if (transitionState(ExecutionState.CANCELING, ExecutionState.CANCELED)) {
// we were immediately canceled. tell the TaskManager that we reached our final
// state
notifyFinalState();
if (metrics != null) {
metrics.close();
}
return;
}
} else {
if (metrics != null) {
metrics.close();
}
throw new IllegalStateException(
"Invalid state for beginning of operation of task " + this + '.');
}
}
// all resource acquisitions and registrations from here on
// need to be undone in the end
Map<String, Future<Path>> distributedCacheEntries = new HashMap<>();
TaskInvokable invokable = null;
try {
// ----------------------------
// Task Bootstrap - We periodically
// check for canceling as a shortcut
// ----------------------------
// activate safety net for task thread
LOG.debug("Creating FileSystem stream leak safety net for task {}", this);
FileSystemSafetyNet.initializeSafetyNetForThread();
// first of all, get a user-code classloader
// this may involve downloading the job's JAR files and/or classes
LOG.info("Loading JAR files for task {}.", this);
userCodeClassLoader = createUserCodeClassloader();
final ExecutionConfig executionConfig =
serializedExecutionConfig.deserializeValue(userCodeClassLoader.asClassLoader());
if (executionConfig.getTaskCancellationInterval() >= 0) {
// override task cancellation interval from Flink config if set in ExecutionConfig
taskCancellationInterval = executionConfig.getTaskCancellationInterval();
}
if (executionConfig.getTaskCancellationTimeout() >= 0) {
// override task cancellation timeout from Flink config if set in ExecutionConfig
taskCancellationTimeout = executionConfig.getTaskCancellationTimeout();
}
if (isCanceledOrFailed()) {
throw new CancelTaskException();
}
// ----------------------------------------------------------------
// register the task with the network stack
// this operation may fail if the system does not have enough
// memory to run the necessary data exchanges
// the registration must also strictly be undone
// ----------------------------------------------------------------
LOG.debug("Registering task at network: {}.", this);
setupPartitionsAndGates(consumableNotifyingPartitionWriters, inputGates);
for (ResultPartitionWriter partitionWriter : consumableNotifyingPartitionWriters) {
taskEventDispatcher.registerPartition(partitionWriter.getPartitionId());
}
// next, kick off the background copying of files for the distributed cache
try {
for (Map.Entry<String, DistributedCache.DistributedCacheEntry> entry :
DistributedCache.readFileInfoFromConfig(jobConfiguration)) {
LOG.info("Obtaining local cache file for '{}'.", entry.getKey());
Future<Path> cp =
fileCache.createTmpFile(
entry.getKey(), entry.getValue(), jobId, executionId);
distributedCacheEntries.put(entry.getKey(), cp);
}
} catch (Exception e) {
throw new Exception(
String.format(
"Exception while adding files to distributed cache of task %s (%s).",
taskNameWithSubtask, executionId),
e);
}
if (isCanceledOrFailed()) {
throw new CancelTaskException();
}
// ----------------------------------------------------------------
// call the user code initialization methods
// ----------------------------------------------------------------
TaskKvStateRegistry kvStateRegistry =
kvStateService.createKvStateTaskRegistry(jobId, getJobVertexId());
Environment env =
new RuntimeEnvironment(
jobId,
vertexId,
executionId,
executionConfig,
taskInfo,
jobConfiguration,
taskConfiguration,
userCodeClassLoader,
memoryManager,
ioManager,
broadcastVariableManager,
taskStateManager,
aggregateManager,
accumulatorRegistry,
kvStateRegistry,
inputSplitProvider,
distributedCacheEntries,
consumableNotifyingPartitionWriters,
inputGates,
taskEventDispatcher,
checkpointResponder,
operatorCoordinatorEventGateway,
taskManagerConfig,
metrics,
this,
externalResourceInfoProvider,
throughputCalculator);
// Make sure the user code classloader is accessible thread-locally.
// We are setting the correct context class loader before instantiating the invokable
// so that it is available to the invokable during its entire lifetime.
executingThread.setContextClassLoader(userCodeClassLoader.asClassLoader());
// When constructing invokable, separate threads can be constructed and thus should be
// monitored for system exit (in addition to invoking thread itself monitored below).
FlinkSecurityManager.monitorUserSystemExitForCurrentThread();
try {
// now load and instantiate the task's invokable code
invokable =
loadAndInstantiateInvokable(
userCodeClassLoader.asClassLoader(), nameOfInvokableClass, env);
} finally {
FlinkSecurityManager.unmonitorUserSystemExitForCurrentThread();
}
// ----------------------------------------------------------------
// actual task core work
// ----------------------------------------------------------------
// we must make strictly sure that the invokable is accessible to the cancel() call
// by the time we switched to running.
this.invokable = invokable;
restoreAndInvoke(invokable);
// make sure, we enter the catch block if the task leaves the invoke() method due
// to the fact that it has been canceled
if (isCanceledOrFailed()) {
throw new CancelTaskException();
}
// ----------------------------------------------------------------
// finalization of a successful execution
// ----------------------------------------------------------------
// finish the produced partitions. if this fails, we consider the execution failed.
for (ResultPartitionWriter partitionWriter : consumableNotifyingPartitionWriters) {
if (partitionWriter != null) {
partitionWriter.finish();
}
}
// try to mark the task as finished
// if that fails, the task was canceled/failed in the meantime
if (!transitionState(ExecutionState.RUNNING, ExecutionState.FINISHED)) {
throw new CancelTaskException();
}
} catch (Throwable t) {
// unwrap wrapped exceptions to make stack traces more compact
if (t instanceof WrappingRuntimeException) {
t = ((WrappingRuntimeException) t).unwrap();
}
// ----------------------------------------------------------------
// the execution failed. either the invokable code properly failed, or
// an exception was thrown as a side effect of cancelling
// ----------------------------------------------------------------
TaskManagerExceptionUtils.tryEnrichTaskManagerError(t);
try {
// check if the exception is unrecoverable
if (ExceptionUtils.isJvmFatalError(t)
|| (t instanceof OutOfMemoryError
&& taskManagerConfig.shouldExitJvmOnOutOfMemoryError())) {
// terminate the JVM immediately
// don't attempt a clean shutdown, because we cannot expect the clean shutdown
// to complete
try {
LOG.error(
"Encountered fatal error {} - terminating the JVM",
t.getClass().getName(),
t);
} finally {
Runtime.getRuntime().halt(-1);
}
}
// transition into our final state. we should be either in DEPLOYING, INITIALIZING,
// RUNNING, CANCELING, or FAILED
// loop for multiple retries during concurrent state changes via calls to cancel()
// or to failExternally()
while (true) {
ExecutionState current = this.executionState;
if (current == ExecutionState.RUNNING
|| current == ExecutionState.INITIALIZING
|| current == ExecutionState.DEPLOYING) {
if (t instanceof CancelTaskException) {
if (transitionState(current, ExecutionState.CANCELED)) {
cancelInvokable(invokable);
break;
}
} else {
if (transitionState(current, ExecutionState.FAILED, t)) {
// proper failure of the task. record the exception as the root
// cause
failureCause = t;
cancelInvokable(invokable);
break;
}
}
} else if (current == ExecutionState.CANCELING) {
if (transitionState(current, ExecutionState.CANCELED)) {
break;
}
} else if (current == ExecutionState.FAILED) {
// in state failed already, no transition necessary any more
break;
}
// unexpected state, go to failed
else if (transitionState(current, ExecutionState.FAILED, t)) {
LOG.error(
"Unexpected state in task {} ({}) during an exception: {}.",
taskNameWithSubtask,
executionId,
current);
break;
}
// else fall through the loop and
}
} catch (Throwable tt) {
String message =
String.format(
"FATAL - exception in exception handler of task %s (%s).",
taskNameWithSubtask, executionId);
LOG.error(message, tt);
notifyFatalError(message, tt);
}
} finally {
try {
LOG.info("Freeing task resources for {} ({}).", taskNameWithSubtask, executionId);
// clear the reference to the invokable. this helps guard against holding references
// to the invokable and its structures in cases where this Task object is still
// referenced
this.invokable = null;
// free the network resources
releaseResources();
// free memory resources
if (invokable != null) {
memoryManager.releaseAll(invokable);
}
// remove all of the tasks resources
fileCache.releaseJob(jobId, executionId);
// close and de-activate safety net for task thread
LOG.debug("Ensuring all FileSystem streams are closed for task {}", this);
FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
notifyFinalState();
} catch (Throwable t) {
// an error in the resource cleanup is fatal
String message =
String.format(
"FATAL - exception in resource cleanup of task %s (%s).",
taskNameWithSubtask, executionId);
LOG.error(message, t);
notifyFatalError(message, t);
}
// un-register the metrics at the end so that the task may already be
// counted as finished when this happens
// errors here will only be logged
try {
metrics.close();
} catch (Throwable t) {
LOG.error(
"Error during metrics de-registration of task {} ({}).",
taskNameWithSubtask,
executionId,
t);
}
}
}
private void restoreAndInvoke(TaskInvokable finalInvokable) throws Exception {
try {
// switch to the INITIALIZING state, if that fails, we have been canceled/failed in the
// meantime
if (!transitionState(ExecutionState.DEPLOYING, ExecutionState.INITIALIZING)) {
throw new CancelTaskException();
}
taskManagerActions.updateTaskExecutionState(
new TaskExecutionState(executionId, ExecutionState.INITIALIZING));
// make sure the user code classloader is accessible thread-locally
executingThread.setContextClassLoader(userCodeClassLoader.asClassLoader());
runWithSystemExitMonitoring(finalInvokable::restore);
if (!transitionState(ExecutionState.INITIALIZING, ExecutionState.RUNNING)) {
throw new CancelTaskException();
}
// notify everyone that we switched to running
taskManagerActions.updateTaskExecutionState(
new TaskExecutionState(executionId, ExecutionState.RUNNING));
runWithSystemExitMonitoring(finalInvokable::invoke);
} catch (Throwable throwable) {
try {
runWithSystemExitMonitoring(() -> finalInvokable.cleanUp(throwable));
} catch (Throwable cleanUpThrowable) {
throwable.addSuppressed(cleanUpThrowable);
}
throw throwable;
}
runWithSystemExitMonitoring(() -> finalInvokable.cleanUp(null));
}
/**
* Monitor user codes from exiting JVM covering user function invocation. This can be done in a
* finer-grained way like enclosing user callback functions individually, but as exit triggered
* by framework is not performed and expected in this invoke function anyhow, we can monitor
* exiting JVM for entire scope.
*/
private void runWithSystemExitMonitoring(RunnableWithException action) throws Exception {
FlinkSecurityManager.monitorUserSystemExitForCurrentThread();
try {
action.run();
} finally {
FlinkSecurityManager.unmonitorUserSystemExitForCurrentThread();
}
}
@VisibleForTesting
public static void setupPartitionsAndGates(
ResultPartitionWriter[] producedPartitions, InputGate[] inputGates) throws IOException {
for (ResultPartitionWriter partition : producedPartitions) {
partition.setup();
}
// InputGates must be initialized after the partitions, since during InputGate#setup
// we are requesting partitions
for (InputGate gate : inputGates) {
gate.setup();
}
}
/**
* Releases resources before task exits. We should also fail the partition to release if the
* task has failed, is canceled, or is being canceled at the moment.
*/
private void releaseResources() {
LOG.debug(
"Release task {} network resources (state: {}).",
taskNameWithSubtask,
getExecutionState());
for (ResultPartitionWriter partitionWriter : consumableNotifyingPartitionWriters) {
taskEventDispatcher.unregisterPartition(partitionWriter.getPartitionId());
}
// close network resources
if (isCanceledOrFailed()) {
failAllResultPartitions();
}
closeAllResultPartitions();
closeAllInputGates();
try {