/
CheckpointCoordinator.java
1409 lines (1171 loc) · 52.7 KB
/
CheckpointCoordinator.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.checkpoint;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.checkpoint.hooks.MasterHooks;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.ExecutionVertex;
import org.apache.flink.runtime.executiongraph.JobStatusListener;
import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
import org.apache.flink.runtime.state.CheckpointStorageCoordinatorView;
import org.apache.flink.runtime.state.CheckpointStorageLocation;
import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
import org.apache.flink.runtime.state.SharedStateRegistry;
import org.apache.flink.runtime.state.SharedStateRegistryFactory;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* The checkpoint coordinator coordinates the distributed snapshots of operators and state.
* It triggers the checkpoint by sending the messages to the relevant tasks and collects the
* checkpoint acknowledgements. It also collects and maintains the overview of the state handles
* reported by the tasks that acknowledge the checkpoint.
*/
public class CheckpointCoordinator {
private static final Logger LOG = LoggerFactory.getLogger(CheckpointCoordinator.class);
/** The number of recent checkpoints whose IDs are remembered. */
private static final int NUM_GHOST_CHECKPOINT_IDS = 16;
// ------------------------------------------------------------------------
/** Coordinator-wide lock to safeguard the checkpoint updates. */
private final Object lock = new Object();
/** Lock specially to make sure that trigger requests do not overtake each other.
* This is not done with the coordinator-wide lock, because as part of triggering,
* blocking operations may happen (distributed atomic counters).
* Using a dedicated lock, we avoid blocking the processing of 'acknowledge/decline'
* messages during that phase. */
private final Object triggerLock = new Object();
/** The job whose checkpoint this coordinator coordinates. */
private final JobID job;
/** Default checkpoint properties. **/
private final CheckpointProperties checkpointProperties;
/** The executor used for asynchronous calls, like potentially blocking I/O. */
private final Executor executor;
/** Tasks who need to be sent a message when a checkpoint is started. */
private final ExecutionVertex[] tasksToTrigger;
/** Tasks who need to acknowledge a checkpoint before it succeeds. */
private final ExecutionVertex[] tasksToWaitFor;
/** Tasks who need to be sent a message when a checkpoint is confirmed. */
private final ExecutionVertex[] tasksToCommitTo;
/** Map from checkpoint ID to the pending checkpoint. */
private final Map<Long, PendingCheckpoint> pendingCheckpoints;
/** Completed checkpoints. Implementations can be blocking. Make sure calls to methods
* accessing this don't block the job manager actor and run asynchronously. */
private final CompletedCheckpointStore completedCheckpointStore;
/** The root checkpoint state backend, which is responsible for initializing the
* checkpoint, storing the metadata, and cleaning up the checkpoint. */
private final CheckpointStorageCoordinatorView checkpointStorage;
/** A list of recent checkpoint IDs, to identify late messages (vs invalid ones). */
private final ArrayDeque<Long> recentPendingCheckpoints;
/** Checkpoint ID counter to ensure ascending IDs. In case of job manager failures, these
* need to be ascending across job managers. */
private final CheckpointIDCounter checkpointIdCounter;
/** The base checkpoint interval. Actual trigger time may be affected by the
* max concurrent checkpoints and minimum-pause values */
private final long baseInterval;
/** The max time (in ms) that a checkpoint may take. */
private final long checkpointTimeout;
/** The min time(in ns) to delay after a checkpoint could be triggered. Allows to
* enforce minimum processing time between checkpoint attempts */
private final long minPauseBetweenCheckpointsNanos;
/** The maximum number of checkpoints that may be in progress at the same time. */
private final int maxConcurrentCheckpointAttempts;
/** The timer that handles the checkpoint timeouts and triggers periodic checkpoints. */
private final ScheduledThreadPoolExecutor timer;
/** The master checkpoint hooks executed by this checkpoint coordinator. */
private final HashMap<String, MasterTriggerRestoreHook<?>> masterHooks;
/** Actor that receives status updates from the execution graph this coordinator works for. */
private JobStatusListener jobStatusListener;
/** The number of consecutive failed trigger attempts. */
private final AtomicInteger numUnsuccessfulCheckpointsTriggers = new AtomicInteger(0);
/** A handle to the current periodic trigger, to cancel it when necessary. */
private ScheduledFuture<?> currentPeriodicTrigger;
/** The timestamp (via {@link System#nanoTime()}) when the last checkpoint completed. */
private long lastCheckpointCompletionNanos;
/** Flag whether a triggered checkpoint should immediately schedule the next checkpoint.
* Non-volatile, because only accessed in synchronized scope */
private boolean periodicScheduling;
/** Flag whether a trigger request could not be handled immediately. Non-volatile, because only
* accessed in synchronized scope */
private boolean triggerRequestQueued;
/** Flag marking the coordinator as shut down (not accepting any messages any more). */
private volatile boolean shutdown;
/** Optional tracker for checkpoint statistics. */
@Nullable
private CheckpointStatsTracker statsTracker;
/** A factory for SharedStateRegistry objects. */
private final SharedStateRegistryFactory sharedStateRegistryFactory;
/** Registry that tracks state which is shared across (incremental) checkpoints. */
private SharedStateRegistry sharedStateRegistry;
private boolean isPreferCheckpointForRecovery;
private final CheckpointFailureManager failureManager;
// --------------------------------------------------------------------------------------------
public CheckpointCoordinator(
JobID job,
CheckpointCoordinatorConfiguration chkConfig,
ExecutionVertex[] tasksToTrigger,
ExecutionVertex[] tasksToWaitFor,
ExecutionVertex[] tasksToCommitTo,
CheckpointIDCounter checkpointIDCounter,
CompletedCheckpointStore completedCheckpointStore,
StateBackend checkpointStateBackend,
Executor executor,
SharedStateRegistryFactory sharedStateRegistryFactory,
CheckpointFailureManager failureManager) {
// sanity checks
checkNotNull(checkpointStateBackend);
// max "in between duration" can be one year - this is to prevent numeric overflows
long minPauseBetweenCheckpoints = chkConfig.getMinPauseBetweenCheckpoints();
if (minPauseBetweenCheckpoints > 365L * 24 * 60 * 60 * 1_000) {
minPauseBetweenCheckpoints = 365L * 24 * 60 * 60 * 1_000;
}
// it does not make sense to schedule checkpoints more often then the desired
// time between checkpoints
long baseInterval = chkConfig.getCheckpointInterval();
if (baseInterval < minPauseBetweenCheckpoints) {
baseInterval = minPauseBetweenCheckpoints;
}
this.job = checkNotNull(job);
this.baseInterval = baseInterval;
this.checkpointTimeout = chkConfig.getCheckpointTimeout();
this.minPauseBetweenCheckpointsNanos = minPauseBetweenCheckpoints * 1_000_000;
this.maxConcurrentCheckpointAttempts = chkConfig.getMaxConcurrentCheckpoints();
this.tasksToTrigger = checkNotNull(tasksToTrigger);
this.tasksToWaitFor = checkNotNull(tasksToWaitFor);
this.tasksToCommitTo = checkNotNull(tasksToCommitTo);
this.pendingCheckpoints = new LinkedHashMap<>();
this.checkpointIdCounter = checkNotNull(checkpointIDCounter);
this.completedCheckpointStore = checkNotNull(completedCheckpointStore);
this.executor = checkNotNull(executor);
this.sharedStateRegistryFactory = checkNotNull(sharedStateRegistryFactory);
this.sharedStateRegistry = sharedStateRegistryFactory.create(executor);
this.isPreferCheckpointForRecovery = chkConfig.isPreferCheckpointForRecovery();
this.failureManager = checkNotNull(failureManager);
this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS);
this.masterHooks = new HashMap<>();
this.timer = new ScheduledThreadPoolExecutor(1,
new DispatcherThreadFactory(Thread.currentThread().getThreadGroup(), "Checkpoint Timer"));
// make sure the timer internally cleans up and does not hold onto stale scheduled tasks
this.timer.setRemoveOnCancelPolicy(true);
this.timer.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
this.timer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
this.checkpointProperties = CheckpointProperties.forCheckpoint(chkConfig.getCheckpointRetentionPolicy());
try {
this.checkpointStorage = checkpointStateBackend.createCheckpointStorage(job);
} catch (IOException e) {
throw new FlinkRuntimeException("Failed to create checkpoint storage at checkpoint coordinator side.", e);
}
try {
// Make sure the checkpoint ID enumerator is running. Possibly
// issues a blocking call to ZooKeeper.
checkpointIDCounter.start();
} catch (Throwable t) {
throw new RuntimeException("Failed to start checkpoint ID counter: " + t.getMessage(), t);
}
}
// --------------------------------------------------------------------------------------------
// Configuration
// --------------------------------------------------------------------------------------------
/**
* Adds the given master hook to the checkpoint coordinator. This method does nothing, if
* the checkpoint coordinator already contained a hook with the same ID (as defined via
* {@link MasterTriggerRestoreHook#getIdentifier()}).
*
* @param hook The hook to add.
* @return True, if the hook was added, false if the checkpoint coordinator already
* contained a hook with the same ID.
*/
public boolean addMasterHook(MasterTriggerRestoreHook<?> hook) {
checkNotNull(hook);
final String id = hook.getIdentifier();
checkArgument(!StringUtils.isNullOrWhitespaceOnly(id), "The hook has a null or empty id");
synchronized (lock) {
if (!masterHooks.containsKey(id)) {
masterHooks.put(id, hook);
return true;
}
else {
return false;
}
}
}
/**
* Gets the number of currently register master hooks.
*/
public int getNumberOfRegisteredMasterHooks() {
synchronized (lock) {
return masterHooks.size();
}
}
/**
* Sets the checkpoint stats tracker.
*
* @param statsTracker The checkpoint stats tracker.
*/
public void setCheckpointStatsTracker(@Nullable CheckpointStatsTracker statsTracker) {
this.statsTracker = statsTracker;
}
// --------------------------------------------------------------------------------------------
// Clean shutdown
// --------------------------------------------------------------------------------------------
/**
* Shuts down the checkpoint coordinator.
*
* <p>After this method has been called, the coordinator does not accept
* and further messages and cannot trigger any further checkpoints.
*/
public void shutdown(JobStatus jobStatus) throws Exception {
synchronized (lock) {
if (!shutdown) {
shutdown = true;
LOG.info("Stopping checkpoint coordinator for job {}.", job);
periodicScheduling = false;
triggerRequestQueued = false;
// shut down the hooks
MasterHooks.close(masterHooks.values(), LOG);
masterHooks.clear();
// shut down the thread that handles the timeouts and pending triggers
timer.shutdownNow();
// clear and discard all pending checkpoints
for (PendingCheckpoint pending : pendingCheckpoints.values()) {
failPendingCheckpoint(pending, CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN);
}
pendingCheckpoints.clear();
completedCheckpointStore.shutdown(jobStatus);
checkpointIdCounter.shutdown(jobStatus);
}
}
}
public boolean isShutdown() {
return shutdown;
}
// --------------------------------------------------------------------------------------------
// Triggering Checkpoints and Savepoints
// --------------------------------------------------------------------------------------------
/**
* Triggers a savepoint with the given savepoint directory as a target.
*
* @param timestamp The timestamp for the savepoint.
* @param targetLocation Target location for the savepoint, optional. If null, the
* state backend's configured default will be used.
* @return A future to the completed checkpoint
* @throws IllegalStateException If no savepoint directory has been
* specified and no default savepoint directory has been
* configured
*/
public CompletableFuture<CompletedCheckpoint> triggerSavepoint(
final long timestamp,
@Nullable final String targetLocation) {
final CheckpointProperties properties = CheckpointProperties.forSavepoint();
return triggerSavepointInternal(timestamp, properties, false, targetLocation);
}
/**
* Triggers a synchronous savepoint with the given savepoint directory as a target.
*
* @param timestamp The timestamp for the savepoint.
* @param advanceToEndOfEventTime Flag indicating if the source should inject a {@code MAX_WATERMARK} in the pipeline
* to fire any registered event-time timers.
* @param targetLocation Target location for the savepoint, optional. If null, the
* state backend's configured default will be used.
* @return A future to the completed checkpoint
* @throws IllegalStateException If no savepoint directory has been
* specified and no default savepoint directory has been
* configured
*/
public CompletableFuture<CompletedCheckpoint> triggerSynchronousSavepoint(
final long timestamp,
final boolean advanceToEndOfEventTime,
@Nullable final String targetLocation) {
final CheckpointProperties properties = CheckpointProperties.forSyncSavepoint();
return triggerSavepointInternal(timestamp, properties, advanceToEndOfEventTime, targetLocation);
}
private CompletableFuture<CompletedCheckpoint> triggerSavepointInternal(
final long timestamp,
final CheckpointProperties checkpointProperties,
final boolean advanceToEndOfEventTime,
@Nullable final String targetLocation) {
checkNotNull(checkpointProperties);
try {
PendingCheckpoint pendingCheckpoint = triggerCheckpoint(
timestamp,
checkpointProperties,
targetLocation,
false,
advanceToEndOfEventTime);
return pendingCheckpoint.getCompletionFuture();
} catch (CheckpointException e) {
Throwable cause = new CheckpointException("Failed to trigger savepoint.", e.getCheckpointFailureReason());
return FutureUtils.completedExceptionally(cause);
}
}
/**
* Triggers a new standard checkpoint and uses the given timestamp as the checkpoint
* timestamp.
*
* @param timestamp The timestamp for the checkpoint.
* @param isPeriodic Flag indicating whether this triggered checkpoint is
* periodic. If this flag is true, but the periodic scheduler is disabled,
* the checkpoint will be declined.
* @return <code>true</code> if triggering the checkpoint succeeded.
*/
public boolean triggerCheckpoint(long timestamp, boolean isPeriodic) {
try {
triggerCheckpoint(timestamp, checkpointProperties, null, isPeriodic, false);
return true;
} catch (CheckpointException e) {
long latestGeneratedCheckpointId = getCheckpointIdCounter().get();
// here we can not get the failed pending checkpoint's id,
// so we pass the negative latest generated checkpoint id as a special flag
failureManager.handleCheckpointException(e, -1 * latestGeneratedCheckpointId);
return false;
}
}
@VisibleForTesting
public PendingCheckpoint triggerCheckpoint(
long timestamp,
CheckpointProperties props,
@Nullable String externalSavepointLocation,
boolean isPeriodic,
boolean advanceToEndOfTime) throws CheckpointException {
if (advanceToEndOfTime && !(props.isSynchronous() && props.isSavepoint())) {
throw new IllegalArgumentException("Only synchronous savepoints are allowed to advance the watermark to MAX.");
}
// make some eager pre-checks
synchronized (lock) {
// abort if the coordinator has been shutdown in the meantime
if (shutdown) {
throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN);
}
// Don't allow periodic checkpoint if scheduling has been disabled
if (isPeriodic && !periodicScheduling) {
throw new CheckpointException(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN);
}
// validate whether the checkpoint can be triggered, with respect to the limit of
// concurrent checkpoints, and the minimum time between checkpoints.
// these checks are not relevant for savepoints
if (!props.forceCheckpoint()) {
// sanity check: there should never be more than one trigger request queued
if (triggerRequestQueued) {
LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job);
throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED);
}
// if too many checkpoints are currently in progress, we need to mark that a request is queued
if (pendingCheckpoints.size() >= maxConcurrentCheckpointAttempts) {
triggerRequestQueued = true;
if (currentPeriodicTrigger != null) {
currentPeriodicTrigger.cancel(false);
currentPeriodicTrigger = null;
}
throw new CheckpointException(CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS);
}
// make sure the minimum interval between checkpoints has passed
final long earliestNext = lastCheckpointCompletionNanos + minPauseBetweenCheckpointsNanos;
final long durationTillNextMillis = (earliestNext - System.nanoTime()) / 1_000_000;
if (durationTillNextMillis > 0) {
if (currentPeriodicTrigger != null) {
currentPeriodicTrigger.cancel(false);
currentPeriodicTrigger = null;
}
// Reassign the new trigger to the currentPeriodicTrigger
currentPeriodicTrigger = timer.scheduleAtFixedRate(
new ScheduledTrigger(),
durationTillNextMillis, baseInterval, TimeUnit.MILLISECONDS);
throw new CheckpointException(CheckpointFailureReason.MINIMUM_TIME_BETWEEN_CHECKPOINTS);
}
}
}
// check if all tasks that we need to trigger are running.
// if not, abort the checkpoint
Execution[] executions = new Execution[tasksToTrigger.length];
for (int i = 0; i < tasksToTrigger.length; i++) {
Execution ee = tasksToTrigger[i].getCurrentExecutionAttempt();
if (ee == null) {
LOG.info("Checkpoint triggering task {} of job {} is not being executed at the moment. Aborting checkpoint.",
tasksToTrigger[i].getTaskNameWithSubtaskIndex(),
job);
throw new CheckpointException(CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
} else if (ee.getState() == ExecutionState.RUNNING) {
executions[i] = ee;
} else {
LOG.info("Checkpoint triggering task {} of job {} is not in state {} but {} instead. Aborting checkpoint.",
tasksToTrigger[i].getTaskNameWithSubtaskIndex(),
job,
ExecutionState.RUNNING,
ee.getState());
throw new CheckpointException(CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
}
}
// next, check if all tasks that need to acknowledge the checkpoint are running.
// if not, abort the checkpoint
Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new HashMap<>(tasksToWaitFor.length);
for (ExecutionVertex ev : tasksToWaitFor) {
Execution ee = ev.getCurrentExecutionAttempt();
if (ee != null) {
ackTasks.put(ee.getAttemptId(), ev);
} else {
LOG.info("Checkpoint acknowledging task {} of job {} is not being executed at the moment. Aborting checkpoint.",
ev.getTaskNameWithSubtaskIndex(),
job);
throw new CheckpointException(CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
}
}
// we will actually trigger this checkpoint!
// we lock with a special lock to make sure that trigger requests do not overtake each other.
// this is not done with the coordinator-wide lock, because the 'checkpointIdCounter'
// may issue blocking operations. Using a different lock than the coordinator-wide lock,
// we avoid blocking the processing of 'acknowledge/decline' messages during that time.
synchronized (triggerLock) {
final CheckpointStorageLocation checkpointStorageLocation;
final long checkpointID;
try {
// this must happen outside the coordinator-wide lock, because it communicates
// with external services (in HA mode) and may block for a while.
checkpointID = checkpointIdCounter.getAndIncrement();
checkpointStorageLocation = props.isSavepoint() ?
checkpointStorage.initializeLocationForSavepoint(checkpointID, externalSavepointLocation) :
checkpointStorage.initializeLocationForCheckpoint(checkpointID);
}
catch (Throwable t) {
int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet();
LOG.warn("Failed to trigger checkpoint for job {} ({} consecutive failed attempts so far).",
job,
numUnsuccessful,
t);
throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t);
}
final PendingCheckpoint checkpoint = new PendingCheckpoint(
job,
checkpointID,
timestamp,
ackTasks,
props,
checkpointStorageLocation,
executor);
if (statsTracker != null) {
PendingCheckpointStats callback = statsTracker.reportPendingCheckpoint(
checkpointID,
timestamp,
props);
checkpoint.setStatsCallback(callback);
}
// schedule the timer that will clean up the expired checkpoints
final Runnable canceller = () -> {
synchronized (lock) {
// only do the work if the checkpoint is not discarded anyways
// note that checkpoint completion discards the pending checkpoint object
if (!checkpoint.isDiscarded()) {
LOG.info("Checkpoint {} of job {} expired before completing.", checkpointID, job);
failPendingCheckpoint(checkpoint, CheckpointFailureReason.CHECKPOINT_EXPIRED);
pendingCheckpoints.remove(checkpointID);
rememberRecentCheckpointId(checkpointID);
triggerQueuedRequests();
}
}
};
try {
// re-acquire the coordinator-wide lock
synchronized (lock) {
// since we released the lock in the meantime, we need to re-check
// that the conditions still hold.
if (shutdown) {
throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN);
}
else if (!props.forceCheckpoint()) {
if (triggerRequestQueued) {
LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job);
throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED);
}
if (pendingCheckpoints.size() >= maxConcurrentCheckpointAttempts) {
triggerRequestQueued = true;
if (currentPeriodicTrigger != null) {
currentPeriodicTrigger.cancel(false);
currentPeriodicTrigger = null;
}
throw new CheckpointException(CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS);
}
// make sure the minimum interval between checkpoints has passed
final long earliestNext = lastCheckpointCompletionNanos + minPauseBetweenCheckpointsNanos;
final long durationTillNextMillis = (earliestNext - System.nanoTime()) / 1_000_000;
if (durationTillNextMillis > 0) {
if (currentPeriodicTrigger != null) {
currentPeriodicTrigger.cancel(false);
currentPeriodicTrigger = null;
}
// Reassign the new trigger to the currentPeriodicTrigger
currentPeriodicTrigger = timer.scheduleAtFixedRate(
new ScheduledTrigger(),
durationTillNextMillis, baseInterval, TimeUnit.MILLISECONDS);
throw new CheckpointException(CheckpointFailureReason.MINIMUM_TIME_BETWEEN_CHECKPOINTS);
}
}
LOG.info("Triggering checkpoint {} @ {} for job {}.", checkpointID, timestamp, job);
pendingCheckpoints.put(checkpointID, checkpoint);
ScheduledFuture<?> cancellerHandle = timer.schedule(
canceller,
checkpointTimeout, TimeUnit.MILLISECONDS);
if (!checkpoint.setCancellerHandle(cancellerHandle)) {
// checkpoint is already disposed!
cancellerHandle.cancel(false);
}
// trigger the master hooks for the checkpoint
final List<MasterState> masterStates = MasterHooks.triggerMasterHooks(masterHooks.values(),
checkpointID, timestamp, executor, Time.milliseconds(checkpointTimeout));
for (MasterState s : masterStates) {
checkpoint.addMasterState(s);
}
}
// end of lock scope
final CheckpointOptions checkpointOptions = new CheckpointOptions(
props.getCheckpointType(),
checkpointStorageLocation.getLocationReference());
// send the messages to the tasks that trigger their checkpoint
for (Execution execution: executions) {
if (props.isSynchronous()) {
execution.triggerSynchronousSavepoint(checkpointID, timestamp, checkpointOptions, advanceToEndOfTime);
} else {
execution.triggerCheckpoint(checkpointID, timestamp, checkpointOptions);
}
}
numUnsuccessfulCheckpointsTriggers.set(0);
return checkpoint;
}
catch (Throwable t) {
// guard the map against concurrent modifications
synchronized (lock) {
pendingCheckpoints.remove(checkpointID);
}
int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet();
LOG.warn("Failed to trigger checkpoint {} for job {}. ({} consecutive failed attempts so far)",
checkpointID, job, numUnsuccessful, t);
if (!checkpoint.isDiscarded()) {
failPendingCheckpoint(checkpoint, CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE, t);
}
try {
checkpointStorageLocation.disposeOnFailure();
}
catch (Throwable t2) {
LOG.warn("Cannot dispose failed checkpoint storage location {}", checkpointStorageLocation, t2);
}
throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t);
}
} // end trigger lock
}
// --------------------------------------------------------------------------------------------
// Handling checkpoints and messages
// --------------------------------------------------------------------------------------------
/**
* Receives a {@link DeclineCheckpoint} message for a pending checkpoint.
*
* @param message Checkpoint decline from the task manager
* @param taskManagerLocationInfo The location info of the decline checkpoint message's sender
*/
public void receiveDeclineMessage(DeclineCheckpoint message, String taskManagerLocationInfo) {
if (shutdown || message == null) {
return;
}
if (!job.equals(message.getJob())) {
throw new IllegalArgumentException("Received DeclineCheckpoint message for job " +
message.getJob() + " from " + taskManagerLocationInfo + " while this coordinator handles job " + job);
}
final long checkpointId = message.getCheckpointId();
final String reason = (message.getReason() != null ? message.getReason().getMessage() : "");
PendingCheckpoint checkpoint;
synchronized (lock) {
// we need to check inside the lock for being shutdown as well, otherwise we
// get races and invalid error log messages
if (shutdown) {
return;
}
checkpoint = pendingCheckpoints.remove(checkpointId);
if (checkpoint != null && !checkpoint.isDiscarded()) {
LOG.info("Decline checkpoint {} by task {} of job {} at {}.",
checkpointId,
message.getTaskExecutionId(),
job,
taskManagerLocationInfo);
discardCheckpoint(checkpoint, message.getReason());
}
else if (checkpoint != null) {
// this should not happen
throw new IllegalStateException(
"Received message for discarded but non-removed checkpoint " + checkpointId);
}
else if (LOG.isDebugEnabled()) {
if (recentPendingCheckpoints.contains(checkpointId)) {
// message is for an unknown checkpoint, or comes too late (checkpoint disposed)
LOG.debug("Received another decline message for now expired checkpoint attempt {} from task {} of job {} at {} : {}",
checkpointId, message.getTaskExecutionId(), job, taskManagerLocationInfo, reason);
} else {
// message is for an unknown checkpoint. might be so old that we don't even remember it any more
LOG.debug("Received decline message for unknown (too old?) checkpoint attempt {} from task {} of job {} at {} : {}",
checkpointId, message.getTaskExecutionId(), job, taskManagerLocationInfo, reason);
}
}
}
}
/**
* Receives an AcknowledgeCheckpoint message and returns whether the
* message was associated with a pending checkpoint.
*
* @param message Checkpoint ack from the task manager
*
* @param taskManagerLocationInfo The location of the acknowledge checkpoint message's sender
* @return Flag indicating whether the ack'd checkpoint was associated
* with a pending checkpoint.
*
* @throws CheckpointException If the checkpoint cannot be added to the completed checkpoint store.
*/
public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message, String taskManagerLocationInfo) throws CheckpointException {
if (shutdown || message == null) {
return false;
}
if (!job.equals(message.getJob())) {
LOG.error("Received wrong AcknowledgeCheckpoint message for job {} from {} : {}", job, taskManagerLocationInfo, message);
return false;
}
final long checkpointId = message.getCheckpointId();
synchronized (lock) {
// we need to check inside the lock for being shutdown as well, otherwise we
// get races and invalid error log messages
if (shutdown) {
return false;
}
final PendingCheckpoint checkpoint = pendingCheckpoints.get(checkpointId);
if (checkpoint != null && !checkpoint.isDiscarded()) {
switch (checkpoint.acknowledgeTask(message.getTaskExecutionId(), message.getSubtaskState(), message.getCheckpointMetrics())) {
case SUCCESS:
LOG.debug("Received acknowledge message for checkpoint {} from task {} of job {} at {}.",
checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo);
if (checkpoint.isFullyAcknowledged()) {
completePendingCheckpoint(checkpoint);
}
break;
case DUPLICATE:
LOG.debug("Received a duplicate acknowledge message for checkpoint {}, task {}, job {}, location {}.",
message.getCheckpointId(), message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo);
break;
case UNKNOWN:
LOG.warn("Could not acknowledge the checkpoint {} for task {} of job {} at {}, " +
"because the task's execution attempt id was unknown. Discarding " +
"the state handle to avoid lingering state.", message.getCheckpointId(),
message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo);
discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState());
break;
case DISCARDED:
LOG.warn("Could not acknowledge the checkpoint {} for task {} of job {} at {}, " +
"because the pending checkpoint had been discarded. Discarding the " +
"state handle tp avoid lingering state.",
message.getCheckpointId(), message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo);
discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState());
}
return true;
}
else if (checkpoint != null) {
// this should not happen
throw new IllegalStateException(
"Received message for discarded but non-removed checkpoint " + checkpointId);
}
else {
boolean wasPendingCheckpoint;
// message is for an unknown checkpoint, or comes too late (checkpoint disposed)
if (recentPendingCheckpoints.contains(checkpointId)) {
wasPendingCheckpoint = true;
LOG.warn("Received late message for now expired checkpoint attempt {} from task " +
"{} of job {} at {}.", checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo);
}
else {
LOG.debug("Received message for an unknown checkpoint {} from task {} of job {} at {}.",
checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo);
wasPendingCheckpoint = false;
}
// try to discard the state so that we don't have lingering state lying around
discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState());
return wasPendingCheckpoint;
}
}
}
/**
* Try to complete the given pending checkpoint.
*
* <p>Important: This method should only be called in the checkpoint lock scope.
*
* @param pendingCheckpoint to complete
* @throws CheckpointException if the completion failed
*/
private void completePendingCheckpoint(PendingCheckpoint pendingCheckpoint) throws CheckpointException {
final long checkpointId = pendingCheckpoint.getCheckpointId();
final CompletedCheckpoint completedCheckpoint;
// As a first step to complete the checkpoint, we register its state with the registry
Map<OperatorID, OperatorState> operatorStates = pendingCheckpoint.getOperatorStates();
sharedStateRegistry.registerAll(operatorStates.values());
try {
try {
completedCheckpoint = pendingCheckpoint.finalizeCheckpoint();
failureManager.handleCheckpointSuccess(pendingCheckpoint.getCheckpointId());
}
catch (Exception e1) {
// abort the current pending checkpoint if we fails to finalize the pending checkpoint.
if (!pendingCheckpoint.isDiscarded()) {
failPendingCheckpoint(pendingCheckpoint, CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, e1);
}
throw new CheckpointException("Could not finalize the pending checkpoint " + checkpointId + '.',
CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, e1);
}
// the pending checkpoint must be discarded after the finalization
Preconditions.checkState(pendingCheckpoint.isDiscarded() && completedCheckpoint != null);
try {
completedCheckpointStore.addCheckpoint(completedCheckpoint);
} catch (Exception exception) {
// we failed to store the completed checkpoint. Let's clean up
executor.execute(new Runnable() {
@Override
public void run() {
try {
completedCheckpoint.discardOnFailedStoring();
} catch (Throwable t) {
LOG.warn("Could not properly discard completed checkpoint {}.", completedCheckpoint.getCheckpointID(), t);
}
}
});
throw new CheckpointException("Could not complete the pending checkpoint " + checkpointId + '.',
CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, exception);
}
} finally {
pendingCheckpoints.remove(checkpointId);
triggerQueuedRequests();
}
rememberRecentCheckpointId(checkpointId);
// drop those pending checkpoints that are at prior to the completed one
dropSubsumedCheckpoints(checkpointId);
// record the time when this was completed, to calculate
// the 'min delay between checkpoints'
lastCheckpointCompletionNanos = System.nanoTime();
LOG.info("Completed checkpoint {} for job {} ({} bytes in {} ms).", checkpointId, job,
completedCheckpoint.getStateSize(), completedCheckpoint.getDuration());
if (LOG.isDebugEnabled()) {
StringBuilder builder = new StringBuilder();
builder.append("Checkpoint state: ");
for (OperatorState state : completedCheckpoint.getOperatorStates().values()) {
builder.append(state);
builder.append(", ");
}
// Remove last two chars ", "
builder.setLength(builder.length() - 2);
LOG.debug(builder.toString());
}
// send the "notify complete" call to all vertices
final long timestamp = completedCheckpoint.getTimestamp();
for (ExecutionVertex ev : tasksToCommitTo) {
Execution ee = ev.getCurrentExecutionAttempt();
if (ee != null) {
ee.notifyCheckpointComplete(checkpointId, timestamp);
}
}
}
/**
* Fails all pending checkpoints which have not been acknowledged by the given execution
* attempt id.
*
* @param executionAttemptId for which to discard unacknowledged pending checkpoints
* @param cause of the failure
*/
public void failUnacknowledgedPendingCheckpointsFor(ExecutionAttemptID executionAttemptId, Throwable cause) {
synchronized (lock) {
Iterator<PendingCheckpoint> pendingCheckpointIterator = pendingCheckpoints.values().iterator();
while (pendingCheckpointIterator.hasNext()) {
final PendingCheckpoint pendingCheckpoint = pendingCheckpointIterator.next();
if (!pendingCheckpoint.isAcknowledgedBy(executionAttemptId)) {
pendingCheckpointIterator.remove();
discardCheckpoint(pendingCheckpoint, cause);
}
}
}
}
private void rememberRecentCheckpointId(long id) {
if (recentPendingCheckpoints.size() >= NUM_GHOST_CHECKPOINT_IDS) {
recentPendingCheckpoints.removeFirst();
}
recentPendingCheckpoints.addLast(id);
}
private void dropSubsumedCheckpoints(long checkpointId) {
Iterator<Map.Entry<Long, PendingCheckpoint>> entries = pendingCheckpoints.entrySet().iterator();