wangyang0918 commented on a change in pull request #13864: URL: https://github.com/apache/flink/pull/13864#discussion_r517111520
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java ########## @@ -0,0 +1,306 @@ +/* + * 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.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.runtime.checkpoint.TestingRetrievableStateStorageHelper; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.state.RetrievableStateHandle; +import org.apache.flink.runtime.statehandle.StateHandleStore; +import org.apache.flink.runtime.statehandle.TestingStateHandleStore; +import org.apache.flink.util.AbstractID; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.statehandle.StateHandleStore.NON_EXIST_RESOURCE_VERSION; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link DefaultJobGraphStore} with {@link TestingJobGraphStoreWatcher}, {@link TestingStateHandleStore} + * and {@link TestingJobGraphListener}. + */ +public class DefaultJobGraphStoreTest extends TestLogger { + + private final JobGraph testingJobGraph = new JobGraph(); + private final long timeout = 3 * 1000; + + private TestingStateHandleStore.Builder<JobGraph> builder; + private TestingRetrievableStateStorageHelper<JobGraph> jobGraphStorageHelper; + private TestingJobGraphStoreWatcher testingJobGraphStoreWatcher = new TestingJobGraphStoreWatcher(); + private TestingJobGraphListener testingJobGraphListener = new TestingJobGraphListener(); + + @Before + public void setup() { + builder = TestingStateHandleStore.builder(); + jobGraphStorageHelper = new TestingRetrievableStateStorageHelper<>(); + } + + @After + public void teardown() { + if (testingJobGraphStoreWatcher != null) { + testingJobGraphStoreWatcher.stop(); + } + } + + @Test + public void testRecoverJobGraph() throws Exception { + final RetrievableStateHandle<JobGraph> stateHandle = jobGraphStorageHelper.store(testingJobGraph); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setGetFunction(ignore -> stateHandle) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + + final JobGraph recoveredJobGraph = jobGraphStore.recoverJobGraph(testingJobGraph.getJobID()); + assertThat(recoveredJobGraph, is(notNullValue())); + assertThat(recoveredJobGraph.getJobID(), is(testingJobGraph.getJobID())); + + jobGraphStore.stop(); + } + + @Test + public void testRecoverJobGraphWhenNotExist() throws Exception { + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setGetFunction(ignore -> { + throw new StateHandleStore.NotExistException("Not exist exception."); + }) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + + final JobGraph recoveredJobGraph = jobGraphStore.recoverJobGraph(testingJobGraph.getJobID()); + assertThat(recoveredJobGraph, is(nullValue())); + } + + @Test + public void testRecoverJobGraphExceptionForwarding() throws Exception { + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setGetFunction(ignore -> { + throw new FlinkException("Other exception."); + }) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + + final String expectedErrorMsg = "Could not retrieve the submitted job graph state handle " + + "for " + testingJobGraph.getJobID() + " from the submitted job graph store"; + try { + jobGraphStore.recoverJobGraph(testingJobGraph.getJobID()); + fail("Exception should be thrown"); + } catch (Exception ex) { + assertThat(ex.getMessage(), containsString(expectedErrorMsg)); + } Review comment: I will use the new introduced `FlinkMatcher#containsMessage` or `FlinkMatcher#containsCause` to check the exception. ---------------------------------------------------------------- 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]
