echauchot commented on a change in pull request #13040: URL: https://github.com/apache/flink/pull/13040#discussion_r494178724
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java ########## @@ -283,6 +286,54 @@ public void testConcurrentCheckpointOperations() throws Exception { recoveredTestCheckpoint.awaitDiscard(); } + /** + * FLINK-17073 tests that there is no request triggered when there are too many checkpoints + * waiting to clean and that it resumes when the number of waiting checkpoints as gone below + * the threshold. + * + */ + @Test + public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Exception{ + ManualClock clock = new ManualClock(); + clock.advanceTime(1, TimeUnit.DAYS); + int maxCleaningCheckpoints = 1; + CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + CheckpointRequestDecider checkpointRequestDecider = new CheckpointRequestDecider(maxCleaningCheckpoints, unused ->{}, clock, 1, new AtomicInteger(0)::get, checkpointsCleaner::getNumberOfCheckpointsToClean); + + final int maxCheckpointsToRetain = 1; + Executors.PausableThreadPoolExecutor executor = Executors.pausableExecutor(); + ZooKeeperCompletedCheckpointStore checkpointStore = createCompletedCheckpoints(maxCheckpointsToRetain, executor); + + //pause the executor to pause checkpoints cleaning, to allow assertions + executor.pause(); + + int nbCheckpointsToInject = 3; + for (int i = 1; i <= nbCheckpointsToInject; i++) { + // add checkpoints to clean + TestCompletedCheckpoint completedCheckpoint = new TestCompletedCheckpoint(new JobID(), i, + i, Collections.emptyMap(), CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + checkpointsCleaner::cleanCheckpoint); + checkpointStore.addCheckpoint(completedCheckpoint); + } + + Thread.sleep(100L); // give time to submit checkpoints for cleaning + + int nbCheckpointsSubmittedForCleaningByCheckpointStore = nbCheckpointsToInject - maxCheckpointsToRetain; + assertEquals(nbCheckpointsSubmittedForCleaningByCheckpointStore, checkpointsCleaner.getNumberOfCheckpointsToClean()); Review comment: For 1. you're right, `nbCheckpointsSubmittedForCleaning == 2` so `CountDownLatch(1)` is not correct. +1 on `triggerAll` For 2. true For 3. it is just an extend with 2 overrides it does not seem a big deal to me. Moreover, this new class is only in scope test. Still I don't like looping on the condition that we are actually testing (the count of cleaning CP). IMHO, failing with a timeout if the tested count is incorrect is misleading and makes the build longer so IMHO it is not acceptable. I'll try to find a better way to sync the executor and the test rather than the CountDownLatch. PS: I resolved the discussion by mistake. I just unresolved it. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java ########## @@ -283,6 +286,54 @@ public void testConcurrentCheckpointOperations() throws Exception { recoveredTestCheckpoint.awaitDiscard(); } + /** + * FLINK-17073 tests that there is no request triggered when there are too many checkpoints + * waiting to clean and that it resumes when the number of waiting checkpoints as gone below + * the threshold. + * + */ + @Test + public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Exception{ + ManualClock clock = new ManualClock(); + clock.advanceTime(1, TimeUnit.DAYS); + int maxCleaningCheckpoints = 1; + CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + CheckpointRequestDecider checkpointRequestDecider = new CheckpointRequestDecider(maxCleaningCheckpoints, unused ->{}, clock, 1, new AtomicInteger(0)::get, checkpointsCleaner::getNumberOfCheckpointsToClean); + + final int maxCheckpointsToRetain = 1; + Executors.PausableThreadPoolExecutor executor = Executors.pausableExecutor(); + ZooKeeperCompletedCheckpointStore checkpointStore = createCompletedCheckpoints(maxCheckpointsToRetain, executor); + + //pause the executor to pause checkpoints cleaning, to allow assertions + executor.pause(); + + int nbCheckpointsToInject = 3; + for (int i = 1; i <= nbCheckpointsToInject; i++) { + // add checkpoints to clean + TestCompletedCheckpoint completedCheckpoint = new TestCompletedCheckpoint(new JobID(), i, + i, Collections.emptyMap(), CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + checkpointsCleaner::cleanCheckpoint); + checkpointStore.addCheckpoint(completedCheckpoint); + } + + Thread.sleep(100L); // give time to submit checkpoints for cleaning + + int nbCheckpointsSubmittedForCleaningByCheckpointStore = nbCheckpointsToInject - maxCheckpointsToRetain; + assertEquals(nbCheckpointsSubmittedForCleaningByCheckpointStore, checkpointsCleaner.getNumberOfCheckpointsToClean()); Review comment: For 1. you're right, my bad, `nbCheckpointsSubmittedForCleaning == 2` so `CountDownLatch(1)` is not correct. +1 on `triggerAll` For 2. true For 3. it is just an extend with 2 overrides it does not seem a big deal to me. Moreover, this new class is only in scope test. Still I don't like looping on the condition that we are actually testing (the count of cleaning CP). IMHO, failing with a timeout if the tested count is incorrect is misleading and makes the build longer so IMHO it is not acceptable. I'll try to find a better way to sync the executor and the test rather than the CountDownLatch. PS: I resolved the discussion by mistake. I just unresolved it. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java ########## @@ -283,6 +286,54 @@ public void testConcurrentCheckpointOperations() throws Exception { recoveredTestCheckpoint.awaitDiscard(); } + /** + * FLINK-17073 tests that there is no request triggered when there are too many checkpoints + * waiting to clean and that it resumes when the number of waiting checkpoints as gone below + * the threshold. + * + */ + @Test + public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Exception{ + ManualClock clock = new ManualClock(); + clock.advanceTime(1, TimeUnit.DAYS); + int maxCleaningCheckpoints = 1; + CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + CheckpointRequestDecider checkpointRequestDecider = new CheckpointRequestDecider(maxCleaningCheckpoints, unused ->{}, clock, 1, new AtomicInteger(0)::get, checkpointsCleaner::getNumberOfCheckpointsToClean); + + final int maxCheckpointsToRetain = 1; + Executors.PausableThreadPoolExecutor executor = Executors.pausableExecutor(); + ZooKeeperCompletedCheckpointStore checkpointStore = createCompletedCheckpoints(maxCheckpointsToRetain, executor); + + //pause the executor to pause checkpoints cleaning, to allow assertions + executor.pause(); + + int nbCheckpointsToInject = 3; + for (int i = 1; i <= nbCheckpointsToInject; i++) { + // add checkpoints to clean + TestCompletedCheckpoint completedCheckpoint = new TestCompletedCheckpoint(new JobID(), i, + i, Collections.emptyMap(), CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + checkpointsCleaner::cleanCheckpoint); + checkpointStore.addCheckpoint(completedCheckpoint); + } + + Thread.sleep(100L); // give time to submit checkpoints for cleaning + + int nbCheckpointsSubmittedForCleaningByCheckpointStore = nbCheckpointsToInject - maxCheckpointsToRetain; + assertEquals(nbCheckpointsSubmittedForCleaningByCheckpointStore, checkpointsCleaner.getNumberOfCheckpointsToClean()); Review comment: @rkhachatryan I used waiting loops instead of `countDownLatch` to synchronise the test and the executor. And I simply used the executor queue size as the condition for the waiting loop. That way we no more have the condition I did not like and we no more have the countDownLatch and the new executor class you did not like. Do you find this solution acceptable ? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java ########## @@ -283,6 +286,54 @@ public void testConcurrentCheckpointOperations() throws Exception { recoveredTestCheckpoint.awaitDiscard(); } + /** + * FLINK-17073 tests that there is no request triggered when there are too many checkpoints + * waiting to clean and that it resumes when the number of waiting checkpoints as gone below + * the threshold. + * + */ + @Test + public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Exception{ + ManualClock clock = new ManualClock(); + clock.advanceTime(1, TimeUnit.DAYS); + int maxCleaningCheckpoints = 1; + CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + CheckpointRequestDecider checkpointRequestDecider = new CheckpointRequestDecider(maxCleaningCheckpoints, unused ->{}, clock, 1, new AtomicInteger(0)::get, checkpointsCleaner::getNumberOfCheckpointsToClean); + + final int maxCheckpointsToRetain = 1; + Executors.PausableThreadPoolExecutor executor = Executors.pausableExecutor(); + ZooKeeperCompletedCheckpointStore checkpointStore = createCompletedCheckpoints(maxCheckpointsToRetain, executor); + + //pause the executor to pause checkpoints cleaning, to allow assertions + executor.pause(); + + int nbCheckpointsToInject = 3; + for (int i = 1; i <= nbCheckpointsToInject; i++) { + // add checkpoints to clean + TestCompletedCheckpoint completedCheckpoint = new TestCompletedCheckpoint(new JobID(), i, + i, Collections.emptyMap(), CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + checkpointsCleaner::cleanCheckpoint); + checkpointStore.addCheckpoint(completedCheckpoint); + } + + Thread.sleep(100L); // give time to submit checkpoints for cleaning + + int nbCheckpointsSubmittedForCleaningByCheckpointStore = nbCheckpointsToInject - maxCheckpointsToRetain; + assertEquals(nbCheckpointsSubmittedForCleaningByCheckpointStore, checkpointsCleaner.getNumberOfCheckpointsToClean()); Review comment: Sure the executor can be used for other things that checkpointing in a regular pipeline for the general case. But in the case of this test there is no real pipeline, no `CheckpointCoordinator` just a `ZookeeperCheckpointStore` that gets added some checkpoints. `ZookeeperCheckpointStore` uses the executor only for checkpoints cleaning. In such conditions, I'm not sure that the assumption that there is no other runnables than checkpoint cleanup runnables for the executor does not seem that weak to me. Do you see a case in which this test environment (only a ZookeeperCPStore created) can generate other runnables for the executor ? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java ########## @@ -283,6 +286,54 @@ public void testConcurrentCheckpointOperations() throws Exception { recoveredTestCheckpoint.awaitDiscard(); } + /** + * FLINK-17073 tests that there is no request triggered when there are too many checkpoints + * waiting to clean and that it resumes when the number of waiting checkpoints as gone below + * the threshold. + * + */ + @Test + public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Exception{ + ManualClock clock = new ManualClock(); + clock.advanceTime(1, TimeUnit.DAYS); + int maxCleaningCheckpoints = 1; + CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + CheckpointRequestDecider checkpointRequestDecider = new CheckpointRequestDecider(maxCleaningCheckpoints, unused ->{}, clock, 1, new AtomicInteger(0)::get, checkpointsCleaner::getNumberOfCheckpointsToClean); + + final int maxCheckpointsToRetain = 1; + Executors.PausableThreadPoolExecutor executor = Executors.pausableExecutor(); + ZooKeeperCompletedCheckpointStore checkpointStore = createCompletedCheckpoints(maxCheckpointsToRetain, executor); + + //pause the executor to pause checkpoints cleaning, to allow assertions + executor.pause(); + + int nbCheckpointsToInject = 3; + for (int i = 1; i <= nbCheckpointsToInject; i++) { + // add checkpoints to clean + TestCompletedCheckpoint completedCheckpoint = new TestCompletedCheckpoint(new JobID(), i, + i, Collections.emptyMap(), CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + checkpointsCleaner::cleanCheckpoint); + checkpointStore.addCheckpoint(completedCheckpoint); + } + + Thread.sleep(100L); // give time to submit checkpoints for cleaning + + int nbCheckpointsSubmittedForCleaningByCheckpointStore = nbCheckpointsToInject - maxCheckpointsToRetain; + assertEquals(nbCheckpointsSubmittedForCleaningByCheckpointStore, checkpointsCleaner.getNumberOfCheckpointsToClean()); Review comment: Sure the executor can be used for other things that checkpointing in a regular pipeline for the general case. But in the case of this test there is no real pipeline, no `CheckpointCoordinator` just a `ZookeeperCheckpointStore` that gets added some checkpoints. `ZookeeperCheckpointStore` uses the executor only for checkpoints cleaning. In such conditions, I think the assumption that there is no other runnables than checkpoint cleanup runnables for the executor seem to hold. Do you see a case in which this test environment (only a ZookeeperCPStore created) can generate other runnables for the executor ? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org