-
Notifications
You must be signed in to change notification settings - Fork 13k
/
JobMaster.java
1676 lines (1384 loc) · 61.2 KB
/
JobMaster.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.jobmaster;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.core.io.InputSplitAssigner;
import org.apache.flink.queryablestate.KvStateID;
import org.apache.flink.runtime.JobException;
import org.apache.flink.runtime.StoppingException;
import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
import org.apache.flink.runtime.blob.BlobServer;
import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
import org.apache.flink.runtime.checkpoint.CheckpointDeclineReason;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.CheckpointTriggerException;
import org.apache.flink.runtime.checkpoint.Checkpoints;
import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.execution.SuppressRestartsException;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionGraph;
import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder;
import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.IntermediateResult;
import org.apache.flink.runtime.executiongraph.JobStatusListener;
import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
import org.apache.flink.runtime.executiongraph.restart.RestartStrategyResolving;
import org.apache.flink.runtime.heartbeat.HeartbeatListener;
import org.apache.flink.runtime.heartbeat.HeartbeatManager;
import org.apache.flink.runtime.heartbeat.HeartbeatServices;
import org.apache.flink.runtime.heartbeat.HeartbeatTarget;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException;
import org.apache.flink.runtime.jobmaster.exceptions.JobModificationException;
import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory;
import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
import org.apache.flink.runtime.jobmaster.slotpool.SlotPool;
import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolFactory;
import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolGateway;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup;
import org.apache.flink.runtime.query.KvStateLocation;
import org.apache.flink.runtime.query.KvStateLocationRegistry;
import org.apache.flink.runtime.query.UnknownKvStateLocation;
import org.apache.flink.runtime.registration.RegisteredRpcConnection;
import org.apache.flink.runtime.registration.RegistrationResponse;
import org.apache.flink.runtime.registration.RetryingRegistration;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker;
import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats;
import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.FencedRpcEndpoint;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.RpcTimeout;
import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.taskexecutor.AccumulatorReport;
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
import org.apache.flink.runtime.taskmanager.TaskExecutionState;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
import org.apache.flink.types.SerializableOptional;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import javax.annotation.Nullable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeoutException;
import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;
/**
* JobMaster implementation. The job master is responsible for the execution of a single
* {@link JobGraph}.
*
* <p>It offers the following methods as part of its rpc interface to interact with the JobMaster
* remotely:
* <ul>
* <li>{@link #updateTaskExecutionState} updates the task execution state for
* given task</li>
* </ul>
*/
public class JobMaster extends FencedRpcEndpoint<JobMasterId> implements JobMasterGateway {
/** Default names for Flink's distributed components. */
public static final String JOB_MANAGER_NAME = "jobmanager";
public static final String ARCHIVE_NAME = "archive";
// ------------------------------------------------------------------------
private final JobMasterConfiguration jobMasterConfiguration;
private final ResourceID resourceId;
private final JobGraph jobGraph;
private final Time rpcTimeout;
private final HighAvailabilityServices highAvailabilityServices;
private final BlobServer blobServer;
private final JobManagerJobMetricGroupFactory jobMetricGroupFactory;
private final HeartbeatManager<AccumulatorReport, Void> taskManagerHeartbeatManager;
private final HeartbeatManager<Void, Void> resourceManagerHeartbeatManager;
private final ScheduledExecutorService scheduledExecutorService;
private final OnCompletionActions jobCompletionActions;
private final FatalErrorHandler fatalErrorHandler;
private final ClassLoader userCodeLoader;
private final SlotPool slotPool;
private final SlotPoolGateway slotPoolGateway;
private final RestartStrategy restartStrategy;
// --------- BackPressure --------
private final BackPressureStatsTracker backPressureStatsTracker;
// --------- ResourceManager --------
private final LeaderRetrievalService resourceManagerLeaderRetriever;
// --------- TaskManagers --------
private final Map<ResourceID, Tuple2<TaskManagerLocation, TaskExecutorGateway>> registeredTaskManagers;
// -------- Mutable fields ---------
private ExecutionGraph executionGraph;
@Nullable
private JobManagerJobStatusListener jobStatusListener;
@Nullable
private JobManagerJobMetricGroup jobManagerJobMetricGroup;
@Nullable
private String lastInternalSavepoint;
@Nullable
private ResourceManagerAddress resourceManagerAddress;
@Nullable
private ResourceManagerConnection resourceManagerConnection;
@Nullable
private EstablishedResourceManagerConnection establishedResourceManagerConnection;
// ------------------------------------------------------------------------
public JobMaster(
RpcService rpcService,
JobMasterConfiguration jobMasterConfiguration,
ResourceID resourceId,
JobGraph jobGraph,
HighAvailabilityServices highAvailabilityService,
SlotPoolFactory slotPoolFactory,
JobManagerSharedServices jobManagerSharedServices,
HeartbeatServices heartbeatServices,
BlobServer blobServer,
JobManagerJobMetricGroupFactory jobMetricGroupFactory,
OnCompletionActions jobCompletionActions,
FatalErrorHandler fatalErrorHandler,
ClassLoader userCodeLoader) throws Exception {
super(rpcService, AkkaRpcServiceUtils.createRandomName(JOB_MANAGER_NAME));
final JobMasterGateway selfGateway = getSelfGateway(JobMasterGateway.class);
this.jobMasterConfiguration = checkNotNull(jobMasterConfiguration);
this.resourceId = checkNotNull(resourceId);
this.jobGraph = checkNotNull(jobGraph);
this.rpcTimeout = jobMasterConfiguration.getRpcTimeout();
this.highAvailabilityServices = checkNotNull(highAvailabilityService);
this.blobServer = checkNotNull(blobServer);
this.scheduledExecutorService = jobManagerSharedServices.getScheduledExecutorService();
this.jobCompletionActions = checkNotNull(jobCompletionActions);
this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
this.userCodeLoader = checkNotNull(userCodeLoader);
this.jobMetricGroupFactory = checkNotNull(jobMetricGroupFactory);
this.taskManagerHeartbeatManager = heartbeatServices.createHeartbeatManagerSender(
resourceId,
new TaskManagerHeartbeatListener(selfGateway),
rpcService.getScheduledExecutor(),
log);
this.resourceManagerHeartbeatManager = heartbeatServices.createHeartbeatManager(
resourceId,
new ResourceManagerHeartbeatListener(),
rpcService.getScheduledExecutor(),
log);
final String jobName = jobGraph.getName();
final JobID jid = jobGraph.getJobID();
log.info("Initializing job {} ({}).", jobName, jid);
final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
jobGraph.getSerializedExecutionConfig()
.deserializeValue(userCodeLoader)
.getRestartStrategy();
this.restartStrategy = RestartStrategyResolving.resolve(restartStrategyConfiguration,
jobManagerSharedServices.getRestartStrategyFactory(),
jobGraph.isCheckpointingEnabled());
log.info("Using restart strategy {} for {} ({}).", this.restartStrategy, jobName, jid);
resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
this.slotPool = checkNotNull(slotPoolFactory).createSlotPool(jobGraph.getJobID());
this.slotPoolGateway = slotPool.getSelfGateway(SlotPoolGateway.class);
this.registeredTaskManagers = new HashMap<>(4);
this.backPressureStatsTracker = checkNotNull(jobManagerSharedServices.getBackPressureStatsTracker());
this.lastInternalSavepoint = null;
this.jobManagerJobMetricGroup = jobMetricGroupFactory.create(jobGraph);
this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup);
this.jobStatusListener = null;
this.resourceManagerConnection = null;
this.establishedResourceManagerConnection = null;
}
//----------------------------------------------------------------------------------------------
// Lifecycle management
//----------------------------------------------------------------------------------------------
@Override
public void start() {
throw new UnsupportedOperationException("Should never call start() without leader ID");
}
/**
* Start the rpc service and begin to run the job.
*
* @param newJobMasterId The necessary fencing token to run the job
* @param timeout for the operation
* @return Future acknowledge if the job could be started. Otherwise the future contains an exception
*/
public CompletableFuture<Acknowledge> start(final JobMasterId newJobMasterId, final Time timeout) throws Exception {
// make sure we receive RPC and async calls
super.start();
return callAsyncWithoutFencing(() -> startJobExecution(newJobMasterId), timeout);
}
/**
* Suspending job, all the running tasks will be cancelled, and communication with other components
* will be disposed.
*
* <p>Mostly job is suspended because of the leadership has been revoked, one can be restart this job by
* calling the {@link #start(JobMasterId, Time)} method once we take the leadership back again.
*
* <p>This method is executed asynchronously
*
* @param cause The reason of why this job been suspended.
* @param timeout for this operation
* @return Future acknowledge indicating that the job has been suspended. Otherwise the future contains an exception
*/
public CompletableFuture<Acknowledge> suspend(final Exception cause, final Time timeout) {
CompletableFuture<Acknowledge> suspendFuture = callAsyncWithoutFencing(() -> suspendExecution(cause), timeout);
stop();
return suspendFuture;
}
/**
* Suspend the job and shutdown all other services including rpc.
*/
@Override
public CompletableFuture<Void> postStop() {
log.info("Stopping the JobMaster for job {}({}).", jobGraph.getName(), jobGraph.getJobID());
// disconnect from all registered TaskExecutors
final Set<ResourceID> taskManagerResourceIds = new HashSet<>(registeredTaskManagers.keySet());
final FlinkException cause = new FlinkException("Stopping JobMaster for job " + jobGraph.getName() +
'(' + jobGraph.getJobID() + ").");
for (ResourceID taskManagerResourceId : taskManagerResourceIds) {
disconnectTaskManager(taskManagerResourceId, cause);
}
taskManagerHeartbeatManager.stop();
resourceManagerHeartbeatManager.stop();
// make sure there is a graceful exit
suspendExecution(new FlinkException("JobManager is shutting down."));
// shut down will internally release all registered slots
slotPool.shutDown();
final CompletableFuture<Void> disposeInternalSavepointFuture;
if (lastInternalSavepoint != null) {
disposeInternalSavepointFuture = CompletableFuture.runAsync(() -> disposeSavepoint(lastInternalSavepoint));
} else {
disposeInternalSavepointFuture = CompletableFuture.completedFuture(null);
}
final CompletableFuture<Void> slotPoolTerminationFuture = slotPool.getTerminationFuture();
return FutureUtils.completeAll(Arrays.asList(disposeInternalSavepointFuture, slotPoolTerminationFuture));
}
//----------------------------------------------------------------------------------------------
// RPC methods
//----------------------------------------------------------------------------------------------
@Override
public CompletableFuture<Acknowledge> cancel(Time timeout) {
executionGraph.cancel();
return CompletableFuture.completedFuture(Acknowledge.get());
}
@Override
public CompletableFuture<Acknowledge> stop(Time timeout) {
try {
executionGraph.stop();
} catch (StoppingException e) {
return FutureUtils.completedExceptionally(e);
}
return CompletableFuture.completedFuture(Acknowledge.get());
}
@Override
public CompletableFuture<Acknowledge> rescaleJob(
int newParallelism,
RescalingBehaviour rescalingBehaviour,
Time timeout) {
final ArrayList<JobVertexID> allOperators = new ArrayList<>(jobGraph.getNumberOfVertices());
for (JobVertex jobVertex : jobGraph.getVertices()) {
allOperators.add(jobVertex.getID());
}
return rescaleOperators(allOperators, newParallelism, rescalingBehaviour, timeout);
}
@Override
public CompletableFuture<Acknowledge> rescaleOperators(
Collection<JobVertexID> operators,
int newParallelism,
RescalingBehaviour rescalingBehaviour,
Time timeout) {
if (newParallelism <= 0) {
return FutureUtils.completedExceptionally(
new JobModificationException("The target parallelism of a rescaling operation must be larger than 0."));
}
// 1. Check whether we can rescale the job & rescale the respective vertices
try {
rescaleJobGraph(operators, newParallelism, rescalingBehaviour);
} catch (FlinkException e) {
final String msg = String.format("Cannot rescale job %s.", jobGraph.getName());
log.info(msg, e);
return FutureUtils.completedExceptionally(new JobModificationException(msg, e));
}
final ExecutionGraph currentExecutionGraph = executionGraph;
final JobManagerJobMetricGroup newJobManagerJobMetricGroup = jobMetricGroupFactory.create(jobGraph);
final ExecutionGraph newExecutionGraph;
try {
newExecutionGraph = createExecutionGraph(newJobManagerJobMetricGroup);
} catch (JobExecutionException | JobException e) {
return FutureUtils.completedExceptionally(
new JobModificationException("Could not create rescaled ExecutionGraph.", e));
}
// 3. disable checkpoint coordinator to suppress subsequent checkpoints
final CheckpointCoordinator checkpointCoordinator = currentExecutionGraph.getCheckpointCoordinator();
checkpointCoordinator.stopCheckpointScheduler();
// 4. take a savepoint
final CompletableFuture<String> savepointFuture = getJobModificationSavepoint(timeout);
final CompletableFuture<ExecutionGraph> executionGraphFuture = restoreExecutionGraphFromRescalingSavepoint(
newExecutionGraph,
savepointFuture)
.handleAsync(
(ExecutionGraph executionGraph, Throwable failure) -> {
if (failure != null) {
// in case that we couldn't take a savepoint or restore from it, let's restart the checkpoint
// coordinator and abort the rescaling operation
if (checkpointCoordinator.isPeriodicCheckpointingConfigured()) {
checkpointCoordinator.startCheckpointScheduler();
}
throw new CompletionException(ExceptionUtils.stripCompletionException(failure));
} else {
return executionGraph;
}
},
getMainThreadExecutor());
// 5. suspend the current job
final CompletableFuture<JobStatus> terminationFuture = executionGraphFuture.thenComposeAsync(
(ExecutionGraph ignored) -> {
suspendExecutionGraph(new FlinkException("Job is being rescaled."));
return currentExecutionGraph.getTerminationFuture();
},
getMainThreadExecutor());
final CompletableFuture<Void> suspendedFuture = terminationFuture.thenAccept(
(JobStatus jobStatus) -> {
if (jobStatus != JobStatus.SUSPENDED) {
final String msg = String.format("Job %s rescaling failed because we could not suspend the execution graph.", jobGraph.getName());
log.info(msg);
throw new CompletionException(new JobModificationException(msg));
}
});
// 6. resume the new execution graph from the taken savepoint
final CompletableFuture<Acknowledge> rescalingFuture = suspendedFuture.thenCombineAsync(
executionGraphFuture,
(Void ignored, ExecutionGraph restoredExecutionGraph) -> {
// check if the ExecutionGraph is still the same
if (executionGraph == currentExecutionGraph) {
clearExecutionGraphFields();
assignExecutionGraph(restoredExecutionGraph, newJobManagerJobMetricGroup);
scheduleExecutionGraph();
return Acknowledge.get();
} else {
throw new CompletionException(new JobModificationException("Detected concurrent modification of ExecutionGraph. Aborting the rescaling."));
}
},
getMainThreadExecutor());
rescalingFuture.whenComplete(
(Acknowledge ignored, Throwable throwable) -> {
if (throwable != null) {
// fail the newly created execution graph
newExecutionGraph.failGlobal(
new SuppressRestartsException(
new FlinkException(
String.format("Failed to rescale the job %s.", jobGraph.getJobID()),
throwable)));
}
});
return rescalingFuture;
}
/**
* Updates the task execution state for a given task.
*
* @param taskExecutionState New task execution state for a given task
* @return Acknowledge the task execution state update
*/
@Override
public CompletableFuture<Acknowledge> updateTaskExecutionState(
final TaskExecutionState taskExecutionState) {
checkNotNull(taskExecutionState, "taskExecutionState");
if (executionGraph.updateState(taskExecutionState)) {
return CompletableFuture.completedFuture(Acknowledge.get());
} else {
return FutureUtils.completedExceptionally(
new ExecutionGraphException("The execution attempt " +
taskExecutionState.getID() + " was not found."));
}
}
@Override
public CompletableFuture<SerializedInputSplit> requestNextInputSplit(
final JobVertexID vertexID,
final ExecutionAttemptID executionAttempt) {
final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt);
if (execution == null) {
// can happen when JobManager had already unregistered this execution upon on task failure,
// but TaskManager get some delay to aware of that situation
if (log.isDebugEnabled()) {
log.debug("Can not find Execution for attempt {}.", executionAttempt);
}
// but we should TaskManager be aware of this
return FutureUtils.completedExceptionally(new Exception("Can not find Execution for attempt " + executionAttempt));
}
final ExecutionJobVertex vertex = executionGraph.getJobVertex(vertexID);
if (vertex == null) {
log.error("Cannot find execution vertex for vertex ID {}.", vertexID);
return FutureUtils.completedExceptionally(new Exception("Cannot find execution vertex for vertex ID " + vertexID));
}
final InputSplitAssigner splitAssigner = vertex.getSplitAssigner();
if (splitAssigner == null) {
log.error("No InputSplitAssigner for vertex ID {}.", vertexID);
return FutureUtils.completedExceptionally(new Exception("No InputSplitAssigner for vertex ID " + vertexID));
}
final LogicalSlot slot = execution.getAssignedResource();
final int taskId = execution.getVertex().getParallelSubtaskIndex();
final String host = slot != null ? slot.getTaskManagerLocation().getHostname() : null;
final InputSplit nextInputSplit = splitAssigner.getNextInputSplit(host, taskId);
if (log.isDebugEnabled()) {
log.debug("Send next input split {}.", nextInputSplit);
}
try {
final byte[] serializedInputSplit = InstantiationUtil.serializeObject(nextInputSplit);
return CompletableFuture.completedFuture(new SerializedInputSplit(serializedInputSplit));
} catch (Exception ex) {
log.error("Could not serialize the next input split of class {}.", nextInputSplit.getClass(), ex);
IOException reason = new IOException("Could not serialize the next input split of class " +
nextInputSplit.getClass() + ".", ex);
vertex.fail(reason);
return FutureUtils.completedExceptionally(reason);
}
}
@Override
public CompletableFuture<ExecutionState> requestPartitionState(
final IntermediateDataSetID intermediateResultId,
final ResultPartitionID resultPartitionId) {
final Execution execution = executionGraph.getRegisteredExecutions().get(resultPartitionId.getProducerId());
if (execution != null) {
return CompletableFuture.completedFuture(execution.getState());
}
else {
final IntermediateResult intermediateResult =
executionGraph.getAllIntermediateResults().get(intermediateResultId);
if (intermediateResult != null) {
// Try to find the producing execution
Execution producerExecution = intermediateResult
.getPartitionById(resultPartitionId.getPartitionId())
.getProducer()
.getCurrentExecutionAttempt();
if (producerExecution.getAttemptId().equals(resultPartitionId.getProducerId())) {
return CompletableFuture.completedFuture(producerExecution.getState());
} else {
return FutureUtils.completedExceptionally(new PartitionProducerDisposedException(resultPartitionId));
}
} else {
return FutureUtils.completedExceptionally(new IllegalArgumentException("Intermediate data set with ID "
+ intermediateResultId + " not found."));
}
}
}
@Override
public CompletableFuture<Acknowledge> scheduleOrUpdateConsumers(
final ResultPartitionID partitionID,
final Time timeout) {
try {
executionGraph.scheduleOrUpdateConsumers(partitionID);
return CompletableFuture.completedFuture(Acknowledge.get());
} catch (Exception e) {
return FutureUtils.completedExceptionally(e);
}
}
@Override
public CompletableFuture<Acknowledge> disconnectTaskManager(final ResourceID resourceID, final Exception cause) {
log.debug("Disconnect TaskExecutor {} because: {}", resourceID, cause.getMessage());
taskManagerHeartbeatManager.unmonitorTarget(resourceID);
CompletableFuture<Acknowledge> releaseFuture = slotPoolGateway.releaseTaskManager(resourceID, cause);
Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManagerConnection = registeredTaskManagers.remove(resourceID);
if (taskManagerConnection != null) {
taskManagerConnection.f1.disconnectJobManager(jobGraph.getJobID(), cause);
}
return releaseFuture;
}
// TODO: This method needs a leader session ID
@Override
public void acknowledgeCheckpoint(
final JobID jobID,
final ExecutionAttemptID executionAttemptID,
final long checkpointId,
final CheckpointMetrics checkpointMetrics,
final TaskStateSnapshot checkpointState) {
final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
final AcknowledgeCheckpoint ackMessage = new AcknowledgeCheckpoint(
jobID,
executionAttemptID,
checkpointId,
checkpointMetrics,
checkpointState);
if (checkpointCoordinator != null) {
getRpcService().execute(() -> {
try {
checkpointCoordinator.receiveAcknowledgeMessage(ackMessage);
} catch (Throwable t) {
log.warn("Error while processing checkpoint acknowledgement message");
}
});
} else {
log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
jobGraph.getJobID());
}
}
// TODO: This method needs a leader session ID
@Override
public void declineCheckpoint(
final JobID jobID,
final ExecutionAttemptID executionAttemptID,
final long checkpointID,
final Throwable reason) {
final DeclineCheckpoint decline = new DeclineCheckpoint(
jobID, executionAttemptID, checkpointID, reason);
final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
if (checkpointCoordinator != null) {
getRpcService().execute(() -> {
try {
checkpointCoordinator.receiveDeclineMessage(decline);
} catch (Exception e) {
log.error("Error in CheckpointCoordinator while processing {}", decline, e);
}
});
} else {
log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
jobGraph.getJobID());
}
}
@Override
public CompletableFuture<KvStateLocation> requestKvStateLocation(final JobID jobId, final String registrationName) {
// sanity check for the correct JobID
if (jobGraph.getJobID().equals(jobId)) {
if (log.isDebugEnabled()) {
log.debug("Lookup key-value state for job {} with registration " +
"name {}.", jobGraph.getJobID(), registrationName);
}
final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
final KvStateLocation location = registry.getKvStateLocation(registrationName);
if (location != null) {
return CompletableFuture.completedFuture(location);
} else {
return FutureUtils.completedExceptionally(new UnknownKvStateLocation(registrationName));
}
} else {
if (log.isDebugEnabled()) {
log.debug("Request of key-value state location for unknown job {} received.", jobId);
}
return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
}
}
@Override
public CompletableFuture<Acknowledge> notifyKvStateRegistered(
final JobID jobId,
final JobVertexID jobVertexId,
final KeyGroupRange keyGroupRange,
final String registrationName,
final KvStateID kvStateId,
final InetSocketAddress kvStateServerAddress) {
if (jobGraph.getJobID().equals(jobId)) {
if (log.isDebugEnabled()) {
log.debug("Key value state registered for job {} under name {}.",
jobGraph.getJobID(), registrationName);
}
try {
executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress);
return CompletableFuture.completedFuture(Acknowledge.get());
} catch (Exception e) {
log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
return FutureUtils.completedExceptionally(e);
}
} else {
if (log.isDebugEnabled()) {
log.debug("Notification about key-value state registration for unknown job {} received.", jobId);
}
return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
}
}
@Override
public CompletableFuture<Acknowledge> notifyKvStateUnregistered(
JobID jobId,
JobVertexID jobVertexId,
KeyGroupRange keyGroupRange,
String registrationName) {
if (jobGraph.getJobID().equals(jobId)) {
if (log.isDebugEnabled()) {
log.debug("Key value state unregistered for job {} under name {}.",
jobGraph.getJobID(), registrationName);
}
try {
executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
jobVertexId, keyGroupRange, registrationName);
return CompletableFuture.completedFuture(Acknowledge.get());
} catch (Exception e) {
log.error("Failed to notify KvStateRegistry about registration {}.", registrationName, e);
return FutureUtils.completedExceptionally(e);
}
} else {
if (log.isDebugEnabled()) {
log.debug("Notification about key-value state deregistration for unknown job {} received.", jobId);
}
return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
}
}
@Override
public CompletableFuture<ClassloadingProps> requestClassloadingProps() {
return CompletableFuture.completedFuture(
new ClassloadingProps(blobServer.getPort(),
executionGraph.getRequiredJarFiles(),
executionGraph.getRequiredClasspaths()));
}
@Override
public CompletableFuture<Collection<SlotOffer>> offerSlots(
final ResourceID taskManagerId,
final Collection<SlotOffer> slots,
final Time timeout) {
Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManager = registeredTaskManagers.get(taskManagerId);
if (taskManager == null) {
return FutureUtils.completedExceptionally(new Exception("Unknown TaskManager " + taskManagerId));
}
final TaskManagerLocation taskManagerLocation = taskManager.f0;
final TaskExecutorGateway taskExecutorGateway = taskManager.f1;
final RpcTaskManagerGateway rpcTaskManagerGateway = new RpcTaskManagerGateway(taskExecutorGateway, getFencingToken());
return slotPoolGateway.offerSlots(
taskManagerLocation,
rpcTaskManagerGateway,
slots);
}
@Override
public void failSlot(
final ResourceID taskManagerId,
final AllocationID allocationId,
final Exception cause) {
if (registeredTaskManagers.containsKey(taskManagerId)) {
internalFailAllocation(allocationId, cause);
} else {
log.warn("Cannot fail slot " + allocationId + " because the TaskManager " +
taskManagerId + " is unknown.");
}
}
private void internalFailAllocation(AllocationID allocationId, Exception cause) {
final CompletableFuture<SerializableOptional<ResourceID>> emptyTaskExecutorFuture = slotPoolGateway.failAllocation(allocationId, cause);
emptyTaskExecutorFuture.thenAcceptAsync(
resourceIdOptional -> resourceIdOptional.ifPresent(this::releaseEmptyTaskManager),
getMainThreadExecutor());
}
private CompletableFuture<Acknowledge> releaseEmptyTaskManager(ResourceID resourceId) {
return disconnectTaskManager(resourceId, new FlinkException(String.format("No more slots registered at JobMaster %s.", resourceId)));
}
@Override
public CompletableFuture<RegistrationResponse> registerTaskManager(
final String taskManagerRpcAddress,
final TaskManagerLocation taskManagerLocation,
final Time timeout) {
final ResourceID taskManagerId = taskManagerLocation.getResourceID();
if (registeredTaskManagers.containsKey(taskManagerId)) {
final RegistrationResponse response = new JMTMRegistrationSuccess(resourceId);
return CompletableFuture.completedFuture(response);
} else {
return getRpcService()
.connect(taskManagerRpcAddress, TaskExecutorGateway.class)
.handleAsync(
(TaskExecutorGateway taskExecutorGateway, Throwable throwable) -> {
if (throwable != null) {
return new RegistrationResponse.Decline(throwable.getMessage());
}
slotPoolGateway.registerTaskManager(taskManagerId);
registeredTaskManagers.put(taskManagerId, Tuple2.of(taskManagerLocation, taskExecutorGateway));
// monitor the task manager as heartbeat target
taskManagerHeartbeatManager.monitorTarget(taskManagerId, new HeartbeatTarget<Void>() {
@Override
public void receiveHeartbeat(ResourceID resourceID, Void payload) {
// the task manager will not request heartbeat, so this method will never be called currently
}
@Override
public void requestHeartbeat(ResourceID resourceID, Void payload) {
taskExecutorGateway.heartbeatFromJobManager(resourceID);
}
});
return new JMTMRegistrationSuccess(resourceId);
},
getMainThreadExecutor());
}
}
@Override
public void disconnectResourceManager(
final ResourceManagerId resourceManagerId,
final Exception cause) {
if (isConnectingToResourceManager(resourceManagerId)) {
reconnectToResourceManager(cause);
}
}
private boolean isConnectingToResourceManager(ResourceManagerId resourceManagerId) {
return resourceManagerAddress != null
&& resourceManagerAddress.getResourceManagerId().equals(resourceManagerId);
}
@Override
public void heartbeatFromTaskManager(final ResourceID resourceID, AccumulatorReport accumulatorReport) {
taskManagerHeartbeatManager.receiveHeartbeat(resourceID, accumulatorReport);
}
@Override
public void heartbeatFromResourceManager(final ResourceID resourceID) {
resourceManagerHeartbeatManager.requestHeartbeat(resourceID, null);
}
@Override
public CompletableFuture<JobDetails> requestJobDetails(@RpcTimeout Time timeout) {
final ExecutionGraph currentExecutionGraph = executionGraph;
return CompletableFuture.supplyAsync(() -> WebMonitorUtils.createDetailsForJob(currentExecutionGraph), scheduledExecutorService);
}
@Override
public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
return CompletableFuture.completedFuture(executionGraph.getState());
}
@Override
public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
return CompletableFuture.completedFuture(ArchivedExecutionGraph.createFrom(executionGraph));
}
@Override
public CompletableFuture<String> triggerSavepoint(
@Nullable final String targetDirectory,
final boolean cancelJob,
final Time timeout) {
final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
if (checkpointCoordinator == null) {
return FutureUtils.completedExceptionally(new IllegalStateException(
String.format("Job %s is not a streaming job.", jobGraph.getJobID())));
}
if (cancelJob) {
checkpointCoordinator.stopCheckpointScheduler();
}
return checkpointCoordinator
.triggerSavepoint(System.currentTimeMillis(), targetDirectory)
.thenApply(CompletedCheckpoint::getExternalPointer)
.thenApplyAsync(path -> {
if (cancelJob) {
log.info("Savepoint stored in {}. Now cancelling {}.", path, jobGraph.getJobID());
cancel(timeout);
}
return path;
}, getMainThreadExecutor())
.exceptionally(throwable -> {
if (cancelJob) {
startCheckpointScheduler(checkpointCoordinator);
}
throw new CompletionException(throwable);
});
}
private void startCheckpointScheduler(final CheckpointCoordinator checkpointCoordinator) {
if (checkpointCoordinator.isPeriodicCheckpointingConfigured()) {
try {
checkpointCoordinator.startCheckpointScheduler();
} catch (IllegalStateException ignored) {
// Concurrent shut down of the coordinator
}
}
}
@Override
public CompletableFuture<OperatorBackPressureStatsResponse> requestOperatorBackPressureStats(final JobVertexID jobVertexId) {
final ExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexId);
if (jobVertex == null) {
return FutureUtils.completedExceptionally(new FlinkException("JobVertexID not found " +
jobVertexId));
}
final Optional<OperatorBackPressureStats> operatorBackPressureStats =
backPressureStatsTracker.getOperatorBackPressureStats(jobVertex);
return CompletableFuture.completedFuture(OperatorBackPressureStatsResponse.of(
operatorBackPressureStats.orElse(null)));
}
@Override
public void notifyAllocationFailure(AllocationID allocationID, Exception cause) {
internalFailAllocation(allocationID, cause);
}