tillrohrmann commented on a change in pull request #17693:
URL: https://github.com/apache/flink/pull/17693#discussion_r755324765
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java
##########
@@ -1002,6 +1007,72 @@ public void
testComputeVertexParallelismStoreForExecutionInDefaultMode() {
}
}
+ @Test
+ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws
Exception {
+ final JobGraph jobGraph = createJobGraph();
+ enableCheckpointing(jobGraph);
+ final CountDownLatch checkpointServicesShutdownBlocked = new
CountDownLatch(1);
+ final CountDownLatch cleanerClosed = new CountDownLatch(1);
+ final ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ final CompletedCheckpointStore completedCheckpointStore =
+ new StandaloneCompletedCheckpointStore(1) {
+
+ @Override
+ public void shutdown(JobStatus jobStatus,
CheckpointsCleaner checkpointsCleaner)
+ throws Exception {
+ checkpointServicesShutdownBlocked.await();
+ super.shutdown(jobStatus, checkpointsCleaner);
+ }
+ };
+ final CheckpointIDCounter checkpointIDCounter =
+ new StandaloneCheckpointIDCounter() {
+
+ @Override
+ public void shutdown(JobStatus jobStatus) throws Exception
{
+ checkpointServicesShutdownBlocked.await();
+ super.shutdown(jobStatus);
+ }
+ };
Review comment:
You might be able to deduplicate these classes with the test for the
`DefaultScheduler`.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -1479,6 +1485,72 @@ public void testLateRegisteredPartitionsWillBeReleased()
{
assertThat(shuffleMaster.getExternallyReleasedPartitions(),
hasSize(1));
}
+ @Test
+ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws
Exception {
+ final JobGraph jobGraph = singleJobVertexJobGraph(1);
+ enableCheckpointing(jobGraph);
+ final CountDownLatch checkpointServicesShutdownBlocked = new
CountDownLatch(1);
+ final CountDownLatch cleanerClosed = new CountDownLatch(1);
Review comment:
A slightly simpler variant of the `CountDownLatch` is the `OneShotLatch`.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -598,12 +599,10 @@ private void registerJobMetrics() {
final FlinkException cause = new FlinkException("Scheduler is being
stopped.");
final CompletableFuture<Void> checkpointServicesShutdownFuture =
- CompletableFuture.allOf(
- executionGraph
- .getTerminationFuture()
- .thenAcceptAsync(
- this::shutDownCheckpointServices,
getMainThreadExecutor()),
- checkpointsCleaner.closeAsync());
+ executionGraph
+ .getTerminationFuture()
+ .thenAcceptAsync(this::shutDownCheckpointServices,
getMainThreadExecutor())
+ .thenCompose(ignored ->
checkpointsCleaner.closeAsync());
Review comment:
I would probably use `FutureUtils.composeAfterwards` because this will
run `checkpointsCleaner.closeAsync()` also in case that the previous future
fails and forward the exception afterwards.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
##########
@@ -1479,6 +1485,72 @@ public void testLateRegisteredPartitionsWillBeReleased()
{
assertThat(shuffleMaster.getExternallyReleasedPartitions(),
hasSize(1));
}
+ @Test
+ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws
Exception {
+ final JobGraph jobGraph = singleJobVertexJobGraph(1);
+ enableCheckpointing(jobGraph);
+ final CountDownLatch checkpointServicesShutdownBlocked = new
CountDownLatch(1);
+ final CountDownLatch cleanerClosed = new CountDownLatch(1);
+ final ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ final CompletedCheckpointStore completedCheckpointStore =
+ new StandaloneCompletedCheckpointStore(1) {
+
+ @Override
+ public void shutdown(JobStatus jobStatus,
CheckpointsCleaner checkpointsCleaner)
+ throws Exception {
+ checkpointServicesShutdownBlocked.await();
+ super.shutdown(jobStatus, checkpointsCleaner);
+ }
+ };
+ final CheckpointIDCounter checkpointIDCounter =
+ new StandaloneCheckpointIDCounter() {
+
+ @Override
+ public void shutdown(JobStatus jobStatus) throws Exception
{
+ checkpointServicesShutdownBlocked.await();
+ super.shutdown(jobStatus);
+ }
+ };
+ final DefaultScheduler scheduler =
+ SchedulerTestingUtils.newSchedulerBuilder(
+ jobGraph,
+
ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(
+ executorService))
+ .setCheckpointRecoveryFactory(
+ new TestingCheckpointRecoveryFactory(
+ completedCheckpointStore,
checkpointIDCounter))
+ .setCheckpointCleaner(
+ new CheckpointsCleaner() {
+
+ @Override
+ public synchronized
CompletableFuture<Void> closeAsync() {
+ cleanerClosed.countDown();
+ return super.closeAsync();
+ }
+ })
+ .build();
+
+ final CompletableFuture<Void> schedulerClosed = new
CompletableFuture<>();
+ final CountDownLatch schedulerClosing = new CountDownLatch(1);
+
+ executorService.submit(
+ () -> {
+ scheduler.closeAsync().thenRun(() ->
schedulerClosed.complete(null));
+ schedulerClosing.countDown();
+ });
+
+ // Wait for scheduler to start closing.
+ schedulerClosing.await();
+ assertFalse(
+ "CheckpointCleaner should not close before checkpoint
services.",
+ cleanerClosed.await(10, TimeUnit.MILLISECONDS));
+ checkpointServicesShutdownBlocked.countDown();
Review comment:
before counting down, you could check whether `cleanerClosed` has been
counted down or not.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java
##########
@@ -1002,6 +1007,72 @@ public void
testComputeVertexParallelismStoreForExecutionInDefaultMode() {
}
}
+ @Test
+ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws
Exception {
+ final JobGraph jobGraph = createJobGraph();
+ enableCheckpointing(jobGraph);
+ final CountDownLatch checkpointServicesShutdownBlocked = new
CountDownLatch(1);
+ final CountDownLatch cleanerClosed = new CountDownLatch(1);
+ final ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ final CompletedCheckpointStore completedCheckpointStore =
+ new StandaloneCompletedCheckpointStore(1) {
+
+ @Override
+ public void shutdown(JobStatus jobStatus,
CheckpointsCleaner checkpointsCleaner)
+ throws Exception {
+ checkpointServicesShutdownBlocked.await();
+ super.shutdown(jobStatus, checkpointsCleaner);
+ }
+ };
+ final CheckpointIDCounter checkpointIDCounter =
+ new StandaloneCheckpointIDCounter() {
+
+ @Override
+ public void shutdown(JobStatus jobStatus) throws Exception
{
+ checkpointServicesShutdownBlocked.await();
+ super.shutdown(jobStatus);
+ }
+ };
Review comment:
You might be able to generalize this test so that you only provide a
`SchedulerNG` instance. That way you wouldn't have to write this test twice.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]