tillrohrmann commented on a change in pull request #14683: URL: https://github.com/apache/flink/pull/14683#discussion_r559714441
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ########## @@ -1278,7 +1278,11 @@ public void run() { } void scheduleTriggerRequest() { - timer.execute(this::executeQueuedRequest); + if (isShutdown()) { + LOG.debug("Skip scheduling trigger request because is shutting down"); Review comment: ```suggestion LOG.debug("Skip scheduling trigger request because the CheckpointCoordinator is shut down"); ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java ########## @@ -62,8 +62,15 @@ public void cleanCheckpoint( } } } finally { - numberOfCheckpointsToClean.decrementAndGet(); - postCleanAction.run(); + try { + numberOfCheckpointsToClean.decrementAndGet(); + postCleanAction.run(); + } catch (Exception e) { + LOG.error( + "Error while cleaning up checkpoint {}", + checkpoint.getCheckpointID(), + e); Review comment: I am not entirely sure whether I would swallow the exceptions here. I think the contract should be that here mustn't occur any exceptions. If they do, then this warrants for a hard exit of the process. With a log, even if it is on error, we would probably not have found this bug very soon. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ########## @@ -1278,7 +1278,11 @@ public void run() { } void scheduleTriggerRequest() { - timer.execute(this::executeQueuedRequest); + if (isShutdown()) { + LOG.debug("Skip scheduling trigger request because is shutting down"); + } else { + timer.execute(this::executeQueuedRequest); + } } Review comment: I think we are missing a test for this behaviour here. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCleanerTest.java ########## @@ -0,0 +1,86 @@ +/* + * 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.api.common.JobID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION; +import static org.apache.flink.util.Preconditions.checkState; +import static org.junit.Assert.assertTrue; + +public class CheckpointCleanerTest { + + @Test + public void testTolerateFailureInPostCleanupSubmit() throws InterruptedException { + ExecutorService executor = java.util.concurrent.Executors.newSingleThreadExecutor(); + CompletedCheckpoint checkpoint = createCheckpoint(); + TestDiscardCallback discardCallback = new TestDiscardCallback(); + checkpoint.setDiscardCallback(discardCallback); + new CheckpointsCleaner().cleanCheckpoint(checkpoint, true, executor::shutdownNow, executor); + checkState(executor.awaitTermination(10, SECONDS)); + assertTrue(discardCallback.isDiscarded()); + } + + @Test + public void testTolerateFailureInPostCleanup() { + CompletedCheckpoint checkpoint = createCheckpoint(); + TestDiscardCallback discardCallback = new TestDiscardCallback(); + checkpoint.setDiscardCallback(discardCallback); + new CheckpointsCleaner() + .cleanCheckpoint( + checkpoint, + true, + () -> { + throw new RuntimeException(); Review comment: Do we really want to support this? I am not sure tbh. Afaik this is framework code we have under our control. Hence, we could say that there mustn't be an exception occurring. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCleanerTest.java ########## @@ -0,0 +1,86 @@ +/* + * 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.api.common.JobID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION; +import static org.apache.flink.util.Preconditions.checkState; +import static org.junit.Assert.assertTrue; + +public class CheckpointCleanerTest { Review comment: ```suggestion public class CheckpointCleanerTest extends TestLogger { ``` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCleanerTest.java ########## @@ -0,0 +1,86 @@ +/* + * 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.api.common.JobID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION; +import static org.apache.flink.util.Preconditions.checkState; +import static org.junit.Assert.assertTrue; + +public class CheckpointCleanerTest { + + @Test + public void testTolerateFailureInPostCleanupSubmit() throws InterruptedException { Review comment: I don't fully understand the test name. What are we testing here? ---------------------------------------------------------------- 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