-
Notifications
You must be signed in to change notification settings - Fork 13k
/
JobMaster.java
1157 lines (958 loc) · 40.5 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.api.common.JobExecutionResult;
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.configuration.Configuration;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.core.io.InputSplitAssigner;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
import org.apache.flink.runtime.checkpoint.CheckpointException;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.checkpoint.SubtaskState;
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.BiFunction;
import org.apache.flink.runtime.concurrent.Future;
import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
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.RestartStrategyFactory;
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.highavailability.LeaderIdMismatchException;
import org.apache.flink.runtime.instance.Slot;
import org.apache.flink.runtime.instance.SlotPool;
import org.apache.flink.runtime.instance.SlotPoolGateway;
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.JobVertexID;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException;
import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
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.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
import org.apache.flink.runtime.query.KvStateID;
import org.apache.flink.runtime.query.KvStateLocation;
import org.apache.flink.runtime.query.KvStateLocationRegistry;
import org.apache.flink.runtime.query.KvStateServerAddress;
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.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcEndpoint;
import org.apache.flink.runtime.rpc.RpcMethod;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.StartStoppable;
import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
import org.apache.flink.runtime.state.KeyGroupRange;
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.util.SerializedThrowable;
import org.apache.flink.util.InstantiationUtil;
import org.slf4j.Logger;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* 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 RpcEndpoint<JobMasterGateway> {
/** Default names for Flink's distributed components */
public static final String JOB_MANAGER_NAME = "jobmanager";
public static final String ARCHIVE_NAME = "archive";
private static final AtomicReferenceFieldUpdater<JobMaster, UUID> LEADER_ID_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(JobMaster.class, UUID.class, "leaderSessionID");
// ------------------------------------------------------------------------
private final ResourceID resourceId;
/** Logical representation of the job */
private final JobGraph jobGraph;
/** Configuration of the JobManager */
private final Configuration configuration;
private final Time rpcTimeout;
/** Service to contend for and retrieve the leadership of JM and RM */
private final HighAvailabilityServices highAvailabilityServices;
/** Blob cache manager used across jobs */
private final BlobLibraryCacheManager libraryCacheManager;
/** The metrics for the JobManager itself */
private final MetricGroup jobManagerMetricGroup;
/** The metrics for the job */
private final MetricGroup jobMetricGroup;
/** The heartbeat manager with task managers */
private final HeartbeatManager<Void, Void> taskManagerHeartbeatManager;
/** The heartbeat manager with resource manager */
private final HeartbeatManager<Void, Void> resourceManagerHeartbeatManager;
/** The execution context which is used to execute futures */
private final Executor executor;
private final OnCompletionActions jobCompletionActions;
private final FatalErrorHandler errorHandler;
private final ClassLoader userCodeLoader;
/** The execution graph of this job */
private final ExecutionGraph executionGraph;
private final SlotPool slotPool;
private final SlotPoolGateway slotPoolGateway;
private volatile UUID leaderSessionID;
// --------- ResourceManager --------
/** Leader retriever service used to locate ResourceManager's address */
private LeaderRetrievalService resourceManagerLeaderRetriever;
/** Connection with ResourceManager, null if not located address yet or we close it initiative */
private ResourceManagerConnection resourceManagerConnection;
// --------- TaskManagers --------
private final Map<ResourceID, Tuple2<TaskManagerLocation, TaskExecutorGateway>> registeredTaskManagers;
// ------------------------------------------------------------------------
public JobMaster(
RpcService rpcService,
ResourceID resourceId,
JobGraph jobGraph,
Configuration configuration,
HighAvailabilityServices highAvailabilityService,
HeartbeatServices heartbeatServices,
ScheduledExecutorService executor,
BlobLibraryCacheManager libraryCacheManager,
RestartStrategyFactory restartStrategyFactory,
Time rpcAskTimeout,
@Nullable JobManagerMetricGroup jobManagerMetricGroup,
OnCompletionActions jobCompletionActions,
FatalErrorHandler errorHandler,
ClassLoader userCodeLoader) throws Exception {
super(rpcService, AkkaRpcServiceUtils.createRandomName(JobMaster.JOB_MANAGER_NAME));
this.resourceId = checkNotNull(resourceId);
this.jobGraph = checkNotNull(jobGraph);
this.configuration = checkNotNull(configuration);
this.rpcTimeout = rpcAskTimeout;
this.highAvailabilityServices = checkNotNull(highAvailabilityService);
this.libraryCacheManager = checkNotNull(libraryCacheManager);
this.executor = checkNotNull(executor);
this.jobCompletionActions = checkNotNull(jobCompletionActions);
this.errorHandler = checkNotNull(errorHandler);
this.userCodeLoader = checkNotNull(userCodeLoader);
this.taskManagerHeartbeatManager = heartbeatServices.createHeartbeatManagerSender(
resourceId,
new TaskManagerHeartbeatListener(),
rpcService.getScheduledExecutor(),
log);
this.resourceManagerHeartbeatManager = heartbeatServices.createHeartbeatManager(
resourceId,
new ResourceManagerHeartbeatListener(),
rpcService.getScheduledExecutor(),
log);
final String jobName = jobGraph.getName();
final JobID jid = jobGraph.getJobID();
if (jobManagerMetricGroup != null) {
this.jobManagerMetricGroup = jobManagerMetricGroup;
this.jobMetricGroup = jobManagerMetricGroup.addJob(jobGraph);
} else {
this.jobManagerMetricGroup = new UnregisteredMetricsGroup();
this.jobMetricGroup = new UnregisteredMetricsGroup();
}
log.info("Initializing job {} ({}).", jobName, jid);
final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
jobGraph.getSerializedExecutionConfig()
.deserializeValue(userCodeLoader)
.getRestartStrategy();
final RestartStrategy restartStrategy = (restartStrategyConfiguration != null) ?
RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration) :
restartStrategyFactory.createRestartStrategy();
log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobName, jid);
CheckpointRecoveryFactory checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory();
resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
this.slotPool = new SlotPool(rpcService, jobGraph.getJobID());
this.slotPoolGateway = slotPool.getSelf();
this.executionGraph = ExecutionGraphBuilder.buildGraph(
null,
jobGraph,
configuration,
executor,
executor,
slotPool.getSlotProvider(),
userCodeLoader,
checkpointRecoveryFactory,
rpcAskTimeout,
restartStrategy,
jobMetricGroup,
-1,
log);
// register self as job status change listener
executionGraph.registerJobStatusListener(new JobManagerJobStatusListener());
this.registeredTaskManagers = new HashMap<>(4);
}
//----------------------------------------------------------------------------------------------
// 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 leaderSessionID The necessary leader id for running the job.
*/
public void start(final UUID leaderSessionID) throws Exception {
if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
// make sure we receive RPC and async calls
super.start();
log.info("JobManager started as leader {} for job {}", leaderSessionID, jobGraph.getJobID());
getSelf().startJobExecution();
}
else {
log.warn("Job already started with leader ID {}, ignoring this start request.", leaderSessionID);
}
}
/**
* Suspend the job and shutdown all other services including rpc.
*/
@Override
public void postStop() throws Exception {
taskManagerHeartbeatManager.stop();
resourceManagerHeartbeatManager.stop();
// make sure there is a graceful exit
suspendExecution(new Exception("JobManager is shutting down."));
super.postStop();
}
//----------------------------------------------------------------------------------------------
// RPC methods
//----------------------------------------------------------------------------------------------
//-- job starting and stopping -----------------------------------------------------------------
@RpcMethod
public void startJobExecution() {
// double check that the leader status did not change
if (leaderSessionID == null) {
log.info("Aborting job startup - JobManager lost leader status");
return;
}
log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID());
try {
// start the slot pool make sure the slot pool now accepts messages for this leader
log.debug("Staring SlotPool component");
slotPool.start(leaderSessionID, getAddress());
} catch (Exception e) {
log.error("Faild to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), e);
handleFatalError(new Exception("Could not start job execution: Failed to start the slot pool.", e));
}
try {
// job is ready to go, try to establish connection with resource manager
// - activate leader retrieval for the resource manager
// - on notification of the leader, the connection will be established and
// the slot pool will start requesting slots
resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener());
}
catch (Throwable t) {
log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
handleFatalError(new Exception(
"Could not start job execution: Failed to start leader service for Resource Manager", t));
return;
}
// start scheduling job in another thread
executor.execute(new Runnable() {
@Override
public void run() {
try {
executionGraph.scheduleForExecution();
}
catch (Throwable t) {
executionGraph.failGlobal(t);
}
}
});
}
/**
* 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(UUID)} method once we take the leadership back again.
*
* @param cause The reason of why this job been suspended.
*/
@RpcMethod
public void suspendExecution(final Throwable cause) {
if (leaderSessionID == null) {
log.debug("Job has already been suspended or shutdown.");
return;
}
// not leader any more - should not accept any leader messages any more
leaderSessionID = null;
try {
resourceManagerLeaderRetriever.stop();
} catch (Throwable t) {
log.warn("Failed to stop resource manager leader retriever when suspending.", t);
}
// tell the execution graph (JobManager is still processing messages here)
executionGraph.suspend(cause);
// receive no more messages until started again, should be called before we clear self leader id
((StartStoppable) getSelf()).stop();
// the slot pool stops receiving messages and clears its pooled slots
slotPoolGateway.suspend();
// disconnect from resource manager:
closeResourceManagerConnection(new Exception("Execution was suspended.", cause));
}
//----------------------------------------------------------------------------------------------
/**
* 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
*/
@RpcMethod
public Acknowledge updateTaskExecutionState(
final UUID leaderSessionID,
final TaskExecutionState taskExecutionState) throws Exception
{
checkNotNull(taskExecutionState, "taskExecutionState");
validateLeaderSessionId(leaderSessionID);
if (executionGraph.updateState(taskExecutionState)) {
return Acknowledge.get();
} else {
throw new ExecutionGraphException("The execution attempt " +
taskExecutionState.getID() + " was not found.");
}
}
@RpcMethod
public SerializedInputSplit requestNextInputSplit(
final UUID leaderSessionID,
final JobVertexID vertexID,
final ExecutionAttemptID executionAttempt) throws Exception
{
validateLeaderSessionId(leaderSessionID);
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
throw 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);
throw 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);
throw new Exception("No InputSplitAssigner for vertex ID " + vertexID);
}
final Slot 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 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);
throw reason;
}
}
@RpcMethod
public ExecutionState requestPartitionState(
final UUID leaderSessionID,
final IntermediateDataSetID intermediateResultId,
final ResultPartitionID resultPartitionId) throws Exception {
validateLeaderSessionId(leaderSessionID);
final Execution execution = executionGraph.getRegisteredExecutions().get(resultPartitionId.getProducerId());
if (execution != null) {
return 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() == resultPartitionId.getProducerId()) {
return producerExecution.getState();
} else {
throw new PartitionProducerDisposedException(resultPartitionId);
}
} else {
throw new IllegalArgumentException("Intermediate data set with ID "
+ intermediateResultId + " not found.");
}
}
}
@RpcMethod
public Acknowledge scheduleOrUpdateConsumers(
final UUID leaderSessionID,
final ResultPartitionID partitionID) throws Exception
{
validateLeaderSessionId(leaderSessionID);
executionGraph.scheduleOrUpdateConsumers(partitionID);
return Acknowledge.get();
}
@RpcMethod
public void disconnectTaskManager(final ResourceID resourceID, final Exception cause) {
taskManagerHeartbeatManager.unmonitorTarget(resourceID);
slotPoolGateway.releaseTaskManager(resourceID);
Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManagerConnection = registeredTaskManagers.remove(resourceID);
if (taskManagerConnection != null) {
taskManagerConnection.f1.disconnectJobManager(jobGraph.getJobID(), cause);
}
}
// TODO: This method needs a leader session ID
@RpcMethod
public void acknowledgeCheckpoint(
final JobID jobID,
final ExecutionAttemptID executionAttemptID,
final long checkpointId,
final CheckpointMetrics checkpointMetrics,
final SubtaskState checkpointState) throws CheckpointException {
final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
final AcknowledgeCheckpoint ackMessage =
new AcknowledgeCheckpoint(jobID, executionAttemptID, checkpointId, checkpointMetrics, checkpointState);
if (checkpointCoordinator != null) {
getRpcService().execute(new Runnable() {
@Override
public void run() {
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
@RpcMethod
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(new Runnable() {
@Override
public void run() {
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());
}
}
@RpcMethod
public KvStateLocation lookupKvStateLocation(final String registrationName) throws Exception {
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 location;
} else {
throw new UnknownKvStateLocation(registrationName);
}
}
@RpcMethod
public void notifyKvStateRegistered(
final JobVertexID jobVertexId,
final KeyGroupRange keyGroupRange,
final String registrationName,
final KvStateID kvStateId,
final KvStateServerAddress kvStateServerAddress)
{
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);
} catch (Exception e) {
log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
}
}
@RpcMethod
public void notifyKvStateUnregistered(
JobVertexID jobVertexId,
KeyGroupRange keyGroupRange,
String registrationName)
{
if (log.isDebugEnabled()) {
log.debug("Key value state unregistered for job {} under name {}.",
jobGraph.getJobID(), registrationName);
}
try {
executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
jobVertexId, keyGroupRange, registrationName);
} catch (Exception e) {
log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
}
}
@RpcMethod
public ClassloadingProps requestClassloadingProps() throws Exception {
return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
executionGraph.getRequiredJarFiles(),
executionGraph.getRequiredClasspaths());
}
@RpcMethod
public Future<Iterable<SlotOffer>> offerSlots(
final ResourceID taskManagerId,
final Iterable<SlotOffer> slots,
final UUID leaderId) throws Exception {
validateLeaderSessionId(leaderId);
Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManager = registeredTaskManagers.get(taskManagerId);
if (taskManager == null) {
throw new Exception("Unknown TaskManager " + taskManagerId);
}
final JobID jid = jobGraph.getJobID();
final TaskManagerLocation taskManagerLocation = taskManager.f0;
final TaskExecutorGateway taskExecutorGateway = taskManager.f1;
final ArrayList<Tuple2<AllocatedSlot, SlotOffer>> slotsAndOffers = new ArrayList<>();
final RpcTaskManagerGateway rpcTaskManagerGateway = new RpcTaskManagerGateway(taskExecutorGateway, leaderId);
for (SlotOffer slotOffer : slots) {
final AllocatedSlot slot = new AllocatedSlot(
slotOffer.getAllocationId(),
jid,
taskManagerLocation,
slotOffer.getSlotIndex(),
slotOffer.getResourceProfile(),
rpcTaskManagerGateway);
slotsAndOffers.add(new Tuple2<>(slot, slotOffer));
}
return slotPoolGateway.offerSlots(slotsAndOffers);
}
@RpcMethod
public void failSlot(final ResourceID taskManagerId,
final AllocationID allocationId,
final UUID leaderId,
final Exception cause) throws Exception
{
validateLeaderSessionId(leaderSessionID);
if (!registeredTaskManagers.containsKey(taskManagerId)) {
throw new Exception("Unknown TaskManager " + taskManagerId);
}
slotPoolGateway.failAllocation(allocationId, cause);
}
@RpcMethod
public Future<RegistrationResponse> registerTaskManager(
final String taskManagerRpcAddress,
final TaskManagerLocation taskManagerLocation,
final UUID leaderId) throws Exception
{
if (!JobMaster.this.leaderSessionID.equals(leaderId)) {
log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " +
"leader session ID {} did not equal the received leader session ID {}.",
taskManagerLocation.getResourceID(), taskManagerRpcAddress,
JobMaster.this.leaderSessionID, leaderId);
throw new Exception("Leader id not match, expected: " + JobMaster.this.leaderSessionID
+ ", actual: " + leaderId);
}
final ResourceID taskManagerId = taskManagerLocation.getResourceID();
if (registeredTaskManagers.containsKey(taskManagerId)) {
final RegistrationResponse response = new JMTMRegistrationSuccess(
resourceId, libraryCacheManager.getBlobServerPort());
return FlinkCompletableFuture.completed(response);
} else {
return getRpcService().execute(new Callable<TaskExecutorGateway>() {
@Override
public TaskExecutorGateway call() throws Exception {
return getRpcService().connect(taskManagerRpcAddress, TaskExecutorGateway.class)
.get(rpcTimeout.getSize(), rpcTimeout.getUnit());
}
}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
@Override
public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
if (throwable != null) {
return new RegistrationResponse.Decline(throwable.getMessage());
}
if (!JobMaster.this.leaderSessionID.equals(leaderId)) {
log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " +
"leader session ID {} did not equal the received leader session ID {}.",
taskManagerId, taskManagerRpcAddress,
JobMaster.this.leaderSessionID, leaderId);
return new RegistrationResponse.Decline("Invalid leader session id");
}
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, libraryCacheManager.getBlobServerPort());
}
}, getMainThreadExecutor());
}
}
@RpcMethod
public void disconnectResourceManager(
final UUID jobManagerLeaderId,
final UUID resourceManagerLeaderId,
final Exception cause) throws Exception {
validateLeaderSessionId(jobManagerLeaderId);
if (resourceManagerConnection != null
&& resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderId)) {
closeResourceManagerConnection(cause);
}
}
@RpcMethod
public void heartbeatFromTaskManager(final ResourceID resourceID) {
taskManagerHeartbeatManager.receiveHeartbeat(resourceID, null);
}
@RpcMethod
public void heartbeatFromResourceManager(final ResourceID resourceID) {
resourceManagerHeartbeatManager.requestHeartbeat(resourceID, null);
}
//----------------------------------------------------------------------------------------------
// Internal methods
//----------------------------------------------------------------------------------------------
private void handleFatalError(final Throwable cause) {
runAsync(new Runnable() {
@Override
public void run() {
log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
try {
shutDown();
} catch (Exception e) {
cause.addSuppressed(e);
}
errorHandler.onFatalError(cause);
}
});
}
private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
validateRunsInMainThread();
final JobID jobID = executionGraph.getJobID();
final String jobName = executionGraph.getJobName();
log.info("Status of job {} ({}) changed to {}.", jobID, jobName, newJobStatus, error);
if (newJobStatus.isGloballyTerminalState()) {
switch (newJobStatus) {
case FINISHED:
try {
// TODO get correct job duration
// job done, let's get the accumulators
Map<String, Object> accumulatorResults = executionGraph.getAccumulators();
JobExecutionResult result = new JobExecutionResult(jobID, 0L, accumulatorResults);
jobCompletionActions.jobFinished(result);
}
catch (Exception e) {
log.error("Cannot fetch final accumulators for job {} ({})", jobName, jobID, e);
final JobExecutionException exception = new JobExecutionException(jobID,
"Failed to retrieve accumulator results. " +
"The job is registered as 'FINISHED (successful), but this notification describes " +
"a failure, since the resulting accumulators could not be fetched.", e);
jobCompletionActions.jobFailed(exception);
}
break;
case CANCELED: {
final JobExecutionException exception = new JobExecutionException(
jobID, "Job was cancelled.", new Exception("The job was cancelled"));
jobCompletionActions.jobFailed(exception);
break;
}
case FAILED: {
final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
final JobExecutionException exception = new JobExecutionException(
jobID, "Job execution failed.", unpackedError);
jobCompletionActions.jobFailed(exception);
break;
}
default:
// this can happen only if the enum is buggy
throw new IllegalStateException(newJobStatus.toString());
}
}
}
private void notifyOfNewResourceManagerLeader(final String resourceManagerAddress, final UUID resourceManagerLeaderId) {
if (resourceManagerConnection != null) {
if (resourceManagerAddress != null) {
if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress())
&& resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId())) {
// both address and leader id are not changed, we can keep the old connection
return;
}
closeResourceManagerConnection(new Exception(
"ResourceManager leader changed to new address " + resourceManagerAddress));
log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
} else {
log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
resourceManagerConnection.getTargetAddress());
}
}
if (resourceManagerAddress != null) {
log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
resourceManagerConnection = new ResourceManagerConnection(
log,
jobGraph.getJobID(),
resourceId,
getAddress(),
leaderSessionID,
resourceManagerAddress,
resourceManagerLeaderId,
executor);
resourceManagerConnection.start();
}
}
private void establishResourceManagerConnection(final JobMasterRegistrationSuccess success) {
final UUID resourceManagerLeaderId = success.getResourceManagerLeaderId();
// verify the response with current connection
if (resourceManagerConnection != null
&& resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderId)) {
log.info("JobManager successfully registered at ResourceManager, leader id: {}.", resourceManagerLeaderId);
final ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
slotPoolGateway.connectToResourceManager(resourceManagerLeaderId, resourceManagerGateway);
resourceManagerHeartbeatManager.monitorTarget(success.getResourceManagerResourceId(), new HeartbeatTarget<Void>() {
@Override
public void receiveHeartbeat(ResourceID resourceID, Void payload) {
resourceManagerGateway.heartbeatFromJobManager(resourceID);
}
@Override
public void requestHeartbeat(ResourceID resourceID, Void payload) {
// request heartbeat will never be called on the job manager side
}
});
}
}
private void closeResourceManagerConnection(Exception cause) {
if (resourceManagerConnection != null) {
log.info("Close ResourceManager connection {}.", resourceManagerConnection.getResourceManagerResourceID(), cause);
resourceManagerHeartbeatManager.unmonitorTarget(resourceManagerConnection.getResourceManagerResourceID());
ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
resourceManagerGateway.disconnectJobManager(resourceManagerConnection.getJobID(), cause);
resourceManagerConnection.close();
resourceManagerConnection = null;
}
slotPoolGateway.disconnectResourceManager();
}
private void validateLeaderSessionId(UUID leaderSessionID) throws LeaderIdMismatchException {
if (this.leaderSessionID == null || !this.leaderSessionID.equals(leaderSessionID)) {
throw new LeaderIdMismatchException(this.leaderSessionID, leaderSessionID);
}
}
//----------------------------------------------------------------------------------------------
// Utility classes
//----------------------------------------------------------------------------------------------
private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
@Override
public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
runAsync(new Runnable() {
@Override
public void run() {
notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
}
});
}