forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
/
SavepointITCase.java
1902 lines (1608 loc) · 77.4 KB
/
SavepointITCase.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.test.checkpointing;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.ClusterOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
import org.apache.flink.core.execution.SavepointFormatType;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.FileSystemFactory;
import org.apache.flink.core.fs.local.LocalFileSystem;
import org.apache.flink.core.fs.local.LocalRecoverableWriter;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.checkpoint.CheckpointException;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.jobgraph.RestoreMode;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
import org.apache.flink.runtime.operators.testutils.ExpectedTestException;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.JobMessageParameters;
import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders;
import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
import org.apache.flink.runtime.testutils.CommonTestUtils;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.IterativeStream;
import org.apache.flink.streaming.api.environment.CheckpointConfig;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.flink.testutils.EntropyInjectingTestFileSystem;
import org.apache.flink.testutils.TestingUtils;
import org.apache.flink.testutils.executor.TestExecutorResource;
import org.apache.flink.testutils.junit.SharedObjects;
import org.apache.flink.testutils.junit.SharedReference;
import org.apache.flink.util.Collector;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.concurrent.FutureUtils;
import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
import org.hamcrest.Description;
import org.hamcrest.Matcher;
import org.hamcrest.TypeSafeDiagnosingMatcher;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.FileVisitOption;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Random;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.concurrent.CompletableFuture.allOf;
import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult;
import static org.apache.flink.util.ExceptionUtils.assertThrowable;
import static org.apache.flink.util.ExceptionUtils.assertThrowableWithMessage;
import static org.apache.flink.util.ExceptionUtils.findThrowable;
import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage;
import static org.hamcrest.CoreMatchers.either;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/** Integration test for triggering and resuming from savepoints. */
@SuppressWarnings("serial")
public class SavepointITCase extends TestLogger {
private static final Logger LOG = LoggerFactory.getLogger(SavepointITCase.class);
@ClassRule
public static final TestExecutorResource<ScheduledExecutorService> EXECUTOR_RESOURCE =
TestingUtils.defaultExecutorResource();
@Rule public final TemporaryFolder folder = new TemporaryFolder();
private File checkpointDir;
private File savepointDir;
@Before
public void setUp() throws Exception {
final File testRoot = folder.newFolder();
checkpointDir = new File(testRoot, "checkpoints");
savepointDir = new File(testRoot, "savepoints");
if (!checkpointDir.mkdir() || !savepointDir.mkdirs()) {
fail("Test setup failed: failed to create temporary directories.");
}
}
@Test
public void testStopWithSavepointForFlip27SourceWithDrain() throws Exception {
testStopWithSavepointForFlip27Source(true);
}
@Test
public void testStopWithSavepointForFlip27SourceWithoutDrain() throws Exception {
testStopWithSavepointForFlip27Source(false);
}
private void testStopWithSavepointForFlip27Source(boolean drain) throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers, numSlotsPerTaskManager, getFileBasedCheckpointsConfig());
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
BoundedPassThroughOperator<Long> operator =
new BoundedPassThroughOperator<>(ChainingStrategy.ALWAYS);
DataStream<Long> stream =
env.fromSequence(0, Long.MAX_VALUE)
.transform("pass-through", BasicTypeInfo.LONG_TYPE_INFO, operator);
stream.addSink(new DiscardingSink<>());
final JobGraph jobGraph = env.getStreamGraph().getJobGraph();
final JobID jobId = jobGraph.getJobID();
MiniClusterWithClientResource cluster = clusterFactory.get();
cluster.before();
ClusterClient<?> client = cluster.getClusterClient();
try {
BoundedPassThroughOperator.resetForTest(1, true);
client.submitJob(jobGraph).get();
BoundedPassThroughOperator.getProgressLatch().await();
waitForAllTaskRunning(cluster.getMiniCluster(), jobId, false);
client.stopWithSavepoint(jobId, drain, null, SavepointFormatType.CANONICAL).get();
if (drain) {
Assert.assertTrue(BoundedPassThroughOperator.inputEnded);
} else {
Assert.assertFalse(BoundedPassThroughOperator.inputEnded);
}
} finally {
cluster.after();
}
}
@Test
public void testStopWithSavepointWithDrainCallsFinishBeforeSnapshotState() throws Exception {
int sinkParallelism = 5;
MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberSlotsPerTaskManager(sinkParallelism + 1)
.build());
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
env.addSource(new InfiniteTestSource())
.setParallelism(1)
.name("Infinite Source")
.addSink(new FinishingSink<>())
// different parallelism to break chaining and add some concurrent tasks
.setParallelism(sinkParallelism);
final JobGraph jobGraph = env.getStreamGraph().getJobGraph();
cluster.before();
try {
ClusterClient<?> client = cluster.getClusterClient();
client.submitJob(jobGraph).get();
waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID());
client.stopWithSavepoint(
jobGraph.getJobID(),
true,
savepointDir.getAbsolutePath(),
SavepointFormatType.CANONICAL)
.get();
// there should be no exceptions and the finish should've been called in the
// FinishingSink
} finally {
cluster.after();
}
}
private static class FinishingSink<T> implements SinkFunction<T>, CheckpointedFunction {
private boolean finishCalled;
@Override
public void invoke(T value) throws Exception {
// ignore
}
@Override
public void finish() throws Exception {
this.finishCalled = true;
}
@Override
public void snapshotState(FunctionSnapshotContext context) throws Exception {
if (!finishCalled) {
fail("Finish is expected to be called before taking the savepoint with drain");
}
}
@Override
public void initializeState(FunctionInitializationContext context) throws Exception {}
}
@Test
public void testStopWithSavepointFailsOverToSavepoint() throws Throwable {
int sinkParallelism = 5;
MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberSlotsPerTaskManager(sinkParallelism + 1)
.build());
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 10));
env.setParallelism(1);
env.addSource(new InfiniteTestSource())
.name("Infinite Source")
.map(new FailingOnCompletedSavepointMapFunction(2))
.addSink(new DiscardingSink<>())
// different parallelism to break chaining and add some concurrent tasks
.setParallelism(sinkParallelism);
final JobGraph jobGraph = env.getStreamGraph().getJobGraph();
cluster.before();
try {
ClusterClient<?> client = cluster.getClusterClient();
client.submitJob(jobGraph).get();
waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID());
cluster.getMiniCluster().triggerCheckpoint(jobGraph.getJobID()).get();
final CompletableFuture<String> savepointCompleted =
client.stopWithSavepoint(
jobGraph.getJobID(),
true,
savepointDir.getAbsolutePath(),
SavepointFormatType.CANONICAL);
final Throwable savepointException =
assertThrows(ExecutionException.class, savepointCompleted::get).getCause();
assertThrowable(
savepointException,
throwable ->
throwable instanceof StopWithSavepointStoppingException
&& throwable
.getMessage()
.startsWith("A savepoint has been created at: "));
assertThat(
client.getJobStatus(jobGraph.getJobID()).get(),
either(is(JobStatus.FAILED)).or(is(JobStatus.FAILING)));
} finally {
cluster.after();
}
}
private static final class FailingOnCompletedSavepointMapFunction
extends RichMapFunction<Integer, Integer> implements CheckpointListener {
private final long savepointId;
private FailingOnCompletedSavepointMapFunction(long savepointId) {
this.savepointId = savepointId;
}
@Override
public Integer map(Integer value) throws Exception {
return value;
}
@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
if (checkpointId == savepointId) {
throw new ExpectedTestException();
}
}
}
/**
* Triggers a savepoint for a job that uses the FsStateBackend. We expect that all checkpoint
* files are written to a new savepoint directory.
*
* <ol>
* <li>Submit job, wait for some progress
* <li>Trigger savepoint and verify that savepoint has been created
* <li>Shut down the cluster, re-submit the job from the savepoint, verify that the initial
* state has been reset, and all tasks are running again
* <li>Cancel job, dispose the savepoint, and verify that everything has been cleaned up
* </ol>
*/
@Test
public void testTriggerSavepointAndResumeWithFileBasedCheckpoints() throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final int parallelism = numTaskManagers * numSlotsPerTaskManager;
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers, numSlotsPerTaskManager, getFileBasedCheckpointsConfig());
final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism);
verifySavepoint(parallelism, savepointPath);
restoreJobAndVerifyState(savepointPath, clusterFactory, parallelism);
}
@Test
public void testTriggerSavepointAndResumeWithClaim() throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final int parallelism = numTaskManagers * numSlotsPerTaskManager;
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers, numSlotsPerTaskManager, getFileBasedCheckpointsConfig());
final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism);
verifySavepoint(parallelism, savepointPath);
restoreJobAndVerifyState(
clusterFactory,
parallelism,
SavepointRestoreSettings.forPath(savepointPath, false, RestoreMode.CLAIM),
cluster -> {
cluster.after();
assertFalse(
"Savepoint not properly cleaned up.",
new File(new URI(savepointPath)).exists());
});
}
@Test
public void testTriggerSavepointAndResumeWithLegacyMode() throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final int parallelism = numTaskManagers * numSlotsPerTaskManager;
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers, numSlotsPerTaskManager, getFileBasedCheckpointsConfig());
final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism);
verifySavepoint(parallelism, savepointPath);
restoreJobAndVerifyState(
clusterFactory,
parallelism,
SavepointRestoreSettings.forPath(savepointPath, false, RestoreMode.LEGACY),
cluster -> {
cluster.after();
assertTrue(
"Savepoint unexpectedly cleaned up.",
new File(new URI(savepointPath)).exists());
});
}
@Rule public SharedObjects sharedObjects = SharedObjects.create();
@Test
@Ignore("Disabling this test because it regularly fails on AZP. See FLINK-25427.")
public void testTriggerSavepointAndResumeWithNoClaim() throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final int parallelism = numTaskManagers * numSlotsPerTaskManager;
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setStateBackend(new EmbeddedRocksDBStateBackend(true));
env.getCheckpointConfig()
.enableExternalizedCheckpoints(
CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
env.getCheckpointConfig().setCheckpointStorage(folder.newFolder().toURI());
env.setParallelism(parallelism);
final SharedReference<CountDownLatch> counter =
sharedObjects.add(new CountDownLatch(10_000));
env.fromSequence(1, Long.MAX_VALUE)
.keyBy(i -> i % parallelism)
.process(
new KeyedProcessFunction<Long, Long, Long>() {
private ListState<Long> last;
@Override
public void open(Configuration parameters) {
// we use list state here to create sst files of a significant size
// if sst files do not reach certain thresholds they are not stored
// in files, but as a byte stream in checkpoints metadata
last =
getRuntimeContext()
.getListState(
new ListStateDescriptor<>(
"last",
BasicTypeInfo.LONG_TYPE_INFO));
}
@Override
public void processElement(
Long value,
KeyedProcessFunction<Long, Long, Long>.Context ctx,
Collector<Long> out)
throws Exception {
last.add(value);
out.collect(value);
}
})
.addSink(
new SinkFunction<Long>() {
@Override
public void invoke(Long value) {
counter.consumeSync(CountDownLatch::countDown);
}
})
.setParallelism(1);
final JobGraph jobGraph = env.getStreamGraph().getJobGraph();
MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberTaskManagers(numTaskManagers)
.setNumberSlotsPerTaskManager(numSlotsPerTaskManager)
.build());
cluster.before();
try {
final JobID jobID1 = new JobID();
jobGraph.setJobID(jobID1);
cluster.getClusterClient().submitJob(jobGraph).get();
CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID1, false);
// wait for some records to be processed before taking the checkpoint
counter.get().await();
final String firstCheckpoint = cluster.getMiniCluster().triggerCheckpoint(jobID1).get();
cluster.getClusterClient().cancel(jobID1).get();
jobGraph.setSavepointRestoreSettings(
SavepointRestoreSettings.forPath(firstCheckpoint, false, RestoreMode.NO_CLAIM));
final JobID jobID2 = new JobID();
jobGraph.setJobID(jobID2);
cluster.getClusterClient().submitJob(jobGraph).get();
CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID2, false);
String secondCheckpoint = cluster.getMiniCluster().triggerCheckpoint(jobID2).get();
cluster.getClusterClient().cancel(jobID2).get();
// delete the checkpoint we restored from
FileUtils.deleteDirectory(Paths.get(new URI(firstCheckpoint)).getParent().toFile());
// we should be able to restore from the second checkpoint even though it has been built
// on top of the first checkpoint
jobGraph.setSavepointRestoreSettings(
SavepointRestoreSettings.forPath(
secondCheckpoint, false, RestoreMode.NO_CLAIM));
final JobID jobID3 = new JobID();
jobGraph.setJobID(jobID3);
cluster.getClusterClient().submitJob(jobGraph).get();
CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID3, false);
} finally {
cluster.after();
}
}
@Test
public void testTriggerSavepointAndResumeWithFileBasedCheckpointsAndRelocateBasePath()
throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final int parallelism = numTaskManagers * numSlotsPerTaskManager;
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers, numSlotsPerTaskManager, getFileBasedCheckpointsConfig());
final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism);
final org.apache.flink.core.fs.Path oldPath =
new org.apache.flink.core.fs.Path(savepointPath);
final org.apache.flink.core.fs.Path newPath =
new org.apache.flink.core.fs.Path(folder.newFolder().toURI().toString());
(new org.apache.flink.core.fs.Path(savepointPath).getFileSystem()).rename(oldPath, newPath);
verifySavepoint(parallelism, newPath.toUri().toString());
restoreJobAndVerifyState(newPath.toUri().toString(), clusterFactory, parallelism);
}
@Test
public void testShouldAddEntropyToSavepointPath() throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
final int parallelism = numTaskManagers * numSlotsPerTaskManager;
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers,
numSlotsPerTaskManager,
getCheckpointingWithEntropyConfig());
final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism);
assertThat(savepointDir, hasEntropyInFileStateHandlePaths());
restoreJobAndVerifyState(
clusterFactory,
parallelism,
SavepointRestoreSettings.forPath(savepointPath),
cluster -> {
final URI localURI = new URI(savepointPath.replace("test-entropy:/", "file:/"));
assertTrue("Savepoint has not been created", new File(localURI).exists());
cluster.getClusterClient().disposeSavepoint(savepointPath).get();
assertFalse("Savepoint not properly cleaned up.", new File(localURI).exists());
});
}
private Configuration getCheckpointingWithEntropyConfig() {
final String savepointPathWithEntropyPlaceholder =
new File(savepointDir, EntropyInjectingTestFileSystem.ENTROPY_INJECTION_KEY)
.getPath();
final Configuration config =
getFileBasedCheckpointsConfig(
"test-entropy://" + savepointPathWithEntropyPlaceholder);
config.setString("s3.entropy.key", EntropyInjectingTestFileSystem.ENTROPY_INJECTION_KEY);
return config;
}
private String submitJobAndTakeSavepoint(
MiniClusterResourceFactory clusterFactory, int parallelism) throws Exception {
final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000);
final JobID jobId = jobGraph.getJobID();
StatefulCounter.resetForTest(parallelism);
MiniClusterWithClientResource cluster = clusterFactory.get();
cluster.before();
ClusterClient<?> client = cluster.getClusterClient();
try {
client.submitJob(jobGraph).get();
waitForAllTaskRunning(cluster.getMiniCluster(), jobId, false);
StatefulCounter.getProgressLatch().await();
return client.cancelWithSavepoint(jobId, null, SavepointFormatType.CANONICAL).get();
} finally {
cluster.after();
StatefulCounter.resetForTest(parallelism);
}
}
private void verifySavepoint(final int parallelism, final String savepointPath)
throws URISyntaxException {
// Only one savepoint should exist
File savepointDir = new File(new URI(savepointPath));
assertTrue("Savepoint directory does not exist.", savepointDir.exists());
assertTrue(
"Savepoint did not create self-contained directory.", savepointDir.isDirectory());
File[] savepointFiles = savepointDir.listFiles();
if (savepointFiles != null) {
// Expect one metadata file and one checkpoint file per stateful
// parallel subtask
String errMsg =
"Did not write expected number of savepoint/checkpoint files to directory: "
+ Arrays.toString(savepointFiles);
assertEquals(errMsg, 1 + parallelism, savepointFiles.length);
} else {
fail(String.format("Returned savepoint path (%s) is not valid.", savepointPath));
}
}
private void restoreJobAndVerifyState(
String savepointPath, MiniClusterResourceFactory clusterFactory, int parallelism)
throws Exception {
restoreJobAndVerifyState(
clusterFactory,
parallelism,
SavepointRestoreSettings.forPath(savepointPath, false),
cluster -> {
cluster.getClusterClient().disposeSavepoint(savepointPath).get();
assertFalse(
"Savepoint not properly cleaned up.",
new File(new URI(savepointPath)).exists());
});
}
@FunctionalInterface
interface PostCancelChecker {
void check(MiniClusterWithClientResource cluster) throws Exception;
}
private void restoreJobAndVerifyState(
MiniClusterResourceFactory clusterFactory,
int parallelism,
SavepointRestoreSettings savepointRestoreSettings,
PostCancelChecker postCancelChecks)
throws Exception {
final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000);
jobGraph.setSavepointRestoreSettings(savepointRestoreSettings);
final JobID jobId = jobGraph.getJobID();
StatefulCounter.resetForTest(parallelism);
MiniClusterWithClientResource cluster = clusterFactory.get();
cluster.before();
ClusterClient<?> client = cluster.getClusterClient();
try {
client.submitJob(jobGraph).get();
// Await state is restored
StatefulCounter.getRestoreLatch().await();
// Await some progress after restore
StatefulCounter.getProgressLatch().await();
client.cancel(jobId).get();
FutureUtils.retrySuccessfulWithDelay(
() -> client.getJobStatus(jobId),
Time.milliseconds(50),
Deadline.now().plus(Duration.ofSeconds(30)),
status -> status == JobStatus.CANCELED,
new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()));
postCancelChecks.check(cluster);
} finally {
cluster.after();
StatefulCounter.resetForTest(parallelism);
}
}
@Test
public void testTriggerSavepointForNonExistingJob() throws Exception {
// Config
final int numTaskManagers = 1;
final int numSlotsPerTaskManager = 1;
final Configuration config = new Configuration();
config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
final MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setConfiguration(config)
.setNumberTaskManagers(numTaskManagers)
.setNumberSlotsPerTaskManager(numSlotsPerTaskManager)
.build());
cluster.before();
final ClusterClient<?> client = cluster.getClusterClient();
final JobID jobID = new JobID();
try {
client.triggerSavepoint(jobID, null, SavepointFormatType.CANONICAL).get();
fail();
} catch (ExecutionException e) {
assertThrowable(e, FlinkJobNotFoundException.class);
assertThrowableWithMessage(e, jobID.toString());
} finally {
cluster.after();
}
}
@Test
public void testTriggerSavepointWithCheckpointingDisabled() throws Exception {
// Config
final int numTaskManagers = 1;
final int numSlotsPerTaskManager = 1;
final Configuration config = new Configuration();
final MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setConfiguration(config)
.setNumberTaskManagers(numTaskManagers)
.setNumberSlotsPerTaskManager(numSlotsPerTaskManager)
.build());
cluster.before();
final ClusterClient<?> client = cluster.getClusterClient();
final JobVertex vertex = new JobVertex("Blocking vertex");
vertex.setInvokableClass(BlockingNoOpInvokable.class);
vertex.setParallelism(1);
final JobGraph graph = JobGraphTestUtils.streamingJobGraph(vertex);
try {
client.submitJob(graph).get();
// triggerSavepoint is only available after all tasks are running
waitForAllTaskRunning(cluster.getMiniCluster(), graph.getJobID(), false);
client.triggerSavepoint(graph.getJobID(), null, SavepointFormatType.CANONICAL).get();
fail();
} catch (ExecutionException e) {
assertThrowable(e, IllegalStateException.class);
assertThrowableWithMessage(e, graph.getJobID().toString());
assertThrowableWithMessage(e, "is not a streaming job");
} finally {
cluster.after();
}
}
@Test
public void testTriggerSavepointWithoutCheckpointBaseLocations() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.getCheckpointConfig().disableCheckpointing();
env.setParallelism(1);
env.addSource(new IntegerStreamSource()).addSink(new DiscardingSink<>());
JobGraph jobGraph = env.getStreamGraph().getJobGraph();
Configuration config = getFileBasedCheckpointsConfig();
config.addAll(jobGraph.getJobConfiguration());
MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setConfiguration(config)
.setNumberTaskManagers(1)
.setNumberSlotsPerTaskManager(1)
.build());
cluster.before();
ClusterClient<?> client = cluster.getClusterClient();
String savepointPath = null;
try {
client.submitJob(jobGraph).get();
waitForAllTaskRunning(cluster.getMiniCluster(), jobGraph.getJobID(), false);
savepointPath =
client.triggerSavepoint(
jobGraph.getJobID(), null, SavepointFormatType.CANONICAL)
.get();
assertNotNull(savepointPath);
client.cancel(jobGraph.getJobID()).get();
// checkpoint directory should not be initialized
assertEquals(0, Objects.requireNonNull(checkpointDir.listFiles()).length);
} finally {
if (null != savepointPath) {
client.disposeSavepoint(savepointPath);
}
cluster.after();
}
}
static class BoundedPassThroughOperator<T> extends AbstractStreamOperator<T>
implements OneInputStreamOperator<T, T>, BoundedOneInput {
static volatile CountDownLatch progressLatch;
static volatile CountDownLatch snapshotAllowedLatch;
static volatile CountDownLatch snapshotStartedLatch;
static volatile boolean inputEnded;
private transient boolean processed;
BoundedPassThroughOperator(ChainingStrategy chainingStrategy) {
this.chainingStrategy = chainingStrategy;
}
private static void allowSnapshots() {
snapshotAllowedLatch.countDown();
}
public static void awaitSnapshotStarted() throws InterruptedException {
snapshotStartedLatch.await();
}
@Override
public void endInput() throws Exception {
inputEnded = true;
}
@Override
public void processElement(StreamRecord<T> element) throws Exception {
output.collect(element);
if (!processed) {
processed = true;
progressLatch.countDown();
}
}
@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
snapshotStartedLatch.countDown();
snapshotAllowedLatch.await();
super.snapshotState(context);
}
// --------------------------------------------------------------------
static CountDownLatch getProgressLatch() {
return progressLatch;
}
static void resetForTest(int parallelism, boolean allowSnapshots) {
progressLatch = new CountDownLatch(parallelism);
snapshotAllowedLatch = new CountDownLatch(allowSnapshots ? 0 : 1);
snapshotStartedLatch = new CountDownLatch(parallelism);
inputEnded = false;
}
}
@Test
public void testStopSavepointWithBoundedInput() throws Exception {
final int numTaskManagers = 2;
final int numSlotsPerTaskManager = 2;
for (ChainingStrategy chainingStrategy : ChainingStrategy.values()) {
final MiniClusterResourceFactory clusterFactory =
new MiniClusterResourceFactory(
numTaskManagers,
numSlotsPerTaskManager,
getFileBasedCheckpointsConfig());
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
BoundedPassThroughOperator<Integer> operator =
new BoundedPassThroughOperator<>(chainingStrategy);
DataStream<Integer> stream =
env.addSource(new InfiniteTestSource())
.transform("pass-through", BasicTypeInfo.INT_TYPE_INFO, operator);
stream.addSink(new DiscardingSink<>());
final JobGraph jobGraph = env.getStreamGraph().getJobGraph();
final JobID jobId = jobGraph.getJobID();
MiniClusterWithClientResource cluster = clusterFactory.get();
cluster.before();
ClusterClient<?> client = cluster.getClusterClient();
try {
BoundedPassThroughOperator.resetForTest(1, true);
client.submitJob(jobGraph).get();
BoundedPassThroughOperator.getProgressLatch().await();
waitForAllTaskRunning(cluster.getMiniCluster(), jobId, false);
client.stopWithSavepoint(jobId, false, null, SavepointFormatType.CANONICAL).get();
Assert.assertFalse(
"input ended with chainingStrategy " + chainingStrategy,
BoundedPassThroughOperator.inputEnded);
} finally {
cluster.after();
}
}
}
@Test
public void testSubmitWithUnknownSavepointPath() throws Exception {
// Config
int numTaskManagers = 1;
int numSlotsPerTaskManager = 1;
int parallelism = numTaskManagers * numSlotsPerTaskManager;
final Configuration config = new Configuration();
config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString());
MiniClusterWithClientResource cluster =
new MiniClusterWithClientResource(
new MiniClusterResourceConfiguration.Builder()
.setConfiguration(config)
.setNumberTaskManagers(numTaskManagers)
.setNumberSlotsPerTaskManager(numSlotsPerTaskManager)
.build());
cluster.before();
ClusterClient<?> client = cluster.getClusterClient();
try {
// High value to ensure timeouts if restarted.
int numberOfRetries = 1000;
// Submit the job
// Long delay to ensure that the test times out if the job
// manager tries to restart the job.
final JobGraph jobGraph = createJobGraph(parallelism, numberOfRetries, 3600000);
// Set non-existing savepoint path
jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath("unknown path"));
assertEquals("unknown path", jobGraph.getSavepointRestoreSettings().getRestorePath());
LOG.info("Submitting job " + jobGraph.getJobID() + " in detached mode.");
try {
submitJobAndWaitForResult(client, jobGraph, getClass().getClassLoader());
} catch (Exception e) {
Optional<JobExecutionException> expectedJobExecutionException =
findThrowable(e, JobExecutionException.class);
Optional<FileNotFoundException> expectedFileNotFoundException =
findThrowable(e, FileNotFoundException.class);
if (!(expectedJobExecutionException.isPresent()
&& expectedFileNotFoundException.isPresent())) {
throw e;
}
}
} finally {
cluster.after();