rkhachatryan commented on a change in pull request #14662: URL: https://github.com/apache/flink/pull/14662#discussion_r564382036
########## File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/CheckpointFailureManagerITCase.java ########## @@ -0,0 +1,214 @@ +/* + * 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.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.CheckpointFailureManager; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.state.AbstractSnapshotStrategy; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.DefaultOperatorStateBackend; +import org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestUtils; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Optional; +import java.util.concurrent.FutureTask; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.mockito.Mockito.mock; + +/** Tests to verify end-to-end logic of checkpoint failure manager. */ +public class CheckpointFailureManagerITCase extends TestLogger { + private static MiniClusterWithClientResource cluster; + + @Before + public void setup() throws Exception { + Configuration configuration = new Configuration(); + + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration) + .build()); + cluster.before(); + } + + @AfterClass + public static void shutDownExistingCluster() { + if (cluster != null) { + cluster.after(); + cluster = null; + } + } + + @Test(timeout = 10000) + public void testAsyncCheckpointFailureTriggerJobFailed() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setStateBackend(new AsyncFailureStateBackend()); + env.addSource(new StringGeneratingSourceFunction()).addSink(new DiscardingSink<>()); + JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph()); + try { + // assert that the job only execute checkpoint once and only failed once. + TestUtils.submitJobAndWaitForResult( + cluster.getClusterClient(), jobGraph, getClass().getClassLoader()); + } catch (JobExecutionException jobException) { + Optional<FlinkRuntimeException> throwable = + ExceptionUtils.findThrowable(jobException, FlinkRuntimeException.class); + Assert.assertTrue(throwable.isPresent()); + Assert.assertEquals( + CheckpointFailureManager.EXCEEDED_CHECKPOINT_TOLERABLE_FAILURE_MESSAGE, Review comment: I think this is a bit fragile. How about testing this on unit level (`CheckpointFailureManagerTest`): ``` @Test public void testAsyncExceptionIsFailure() { TestFailJobCallback callback = new TestFailJobCallback(); new CheckpointFailureManager(1, callback) .handleTaskLevelCheckpointException( new CheckpointException(CHECKPOINT_ASYNC_EXCEPTION), 1L, new ExecutionAttemptID()); assertEquals(1, callback.invokeCounter); } ``` Furthermore, the new IT Case is quite complex, I'm not sure whether it worth it (given we have a unit test). WDYT? ---------------------------------------------------------------- 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: [email protected]
