wangyang0918 commented on a change in pull request #13864: URL: https://github.com/apache/flink/pull/13864#discussion_r515803255
########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java ########## @@ -0,0 +1,356 @@ +/* + * 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.kubernetes.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.runtime.statehandle.StateHandleStore; +import org.apache.flink.runtime.statehandle.TestingLongStateHandleHelper; +import org.apache.flink.runtime.statehandle.TestingLongStateHandleHelper.LongStateStorage; +import org.apache.flink.util.function.FunctionUtils; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.function.Predicate; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link KubernetesStateHandleStore} operations. + */ +public class KubernetesStateHandleStoreTest extends KubernetesHighAvailabilityTestBase { Review comment: `<reuseForks>false</reuseForks>` is already set in the pom. Do you think we need to make this class thread-safe? Then we need to use a concurrent map to store the `numberOfGlobalDiscardCalls `. And key could be `LongRetrievableStateHandle@thread-id`. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java ########## @@ -0,0 +1,305 @@ +/* + * 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 TestingStateHandleStore.Builder<JobGraph> builder = TestingStateHandleStore.builder(); + private final JobGraph testingJobGraph = new JobGraph(); + private final long timeout = 3 * 1000; + + private TestingRetrievableStateStorageHelper<JobGraph> jobGraphStorageHelper; + private TestingJobGraphStoreWatcher testingJobGraphStoreWatcher = new TestingJobGraphStoreWatcher(); + private TestingJobGraphListener testingJobGraphListener = new TestingJobGraphListener(); + + @Before + public void setup() { + 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(); Review comment: Actually, we do not need to do. Because the `TestingJobGraphStoreWatcher` does nothing. ########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java ########## @@ -0,0 +1,356 @@ +/* + * 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.kubernetes.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.runtime.statehandle.StateHandleStore; +import org.apache.flink.runtime.statehandle.TestingLongStateHandleHelper; +import org.apache.flink.runtime.statehandle.TestingLongStateHandleHelper.LongStateStorage; +import org.apache.flink.util.function.FunctionUtils; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.function.Predicate; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link KubernetesStateHandleStore} operations. + */ +public class KubernetesStateHandleStoreTest extends KubernetesHighAvailabilityTestBase { Review comment: I will add `<reuseForks>false</reuseForks>` to the `flink-kubernetes` module pom. Do you prefer to make this class thread-safe? Then we need to use a concurrent map to store the `numberOfGlobalDiscardCalls `. And key could be `LongRetrievableStateHandle@thread-id`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java ########## @@ -156,7 +158,7 @@ public ZooKeeperStateHandleStore( return storeHandle; } catch (KeeperException.NodeExistsException e) { - throw new ConcurrentModificationException("ZooKeeper unexpectedly modified", e); + throw new StateHandleStore.AlreadyExistException(e.getMessage()); Review comment: // We wrap the exception here so that it could be caught in DefaultJobGraphStore ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobGraphStoreWatcher.java ########## @@ -0,0 +1,51 @@ +/* + * 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 static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * {@link JobGraphStoreWatcher} implementation for testing purposes. + */ +public class TestingJobGraphStoreWatcher implements JobGraphStoreWatcher { + + private JobGraphStore.JobGraphListener jobGraphListener; + + @Override + public void start(JobGraphStore.JobGraphListener jobGraphListener) { + this.jobGraphListener = jobGraphListener; + } + + @Override + public void stop() { + // noop + } + + public void addJobGraph(JobID jobID) { Review comment: Frankly speaking, I do not like package private. It will take some additional burn if we want to do some refactor, especially moving from one package to another. ########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java ########## @@ -0,0 +1,400 @@ +/* + * 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.kubernetes.highavailability; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.runtime.state.RetrievableStateHandle; +import org.apache.flink.runtime.statehandle.RetrievableStateStorageHelper; +import org.apache.flink.runtime.statehandle.StateHandleStore; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.InstantiationUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletionException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Class which stores state via the provided {@link RetrievableStateStorageHelper} and writes the + * returned state handle to ConfigMap. + * + * <p>Added state is persisted via {@link RetrievableStateHandle RetrievableStateHandles}, + * which in turn are written to ConfigMap. This level of indirection is necessary to keep the + * amount of data in ConfigMap small. ConfigMap is build for data less than 1MB whereas + * state can grow to multiple MBs and GBs. + * + * <p>This is a very different implementation with {@link org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore}. + * Benefit from the {@link FlinkKubeClient#checkAndUpdateConfigMap} transactional operation, we could guarantee that + * only the leader could update the store. Then we will completely get rid of the lock-and-release in Zookeeper + * implementation. + * + * @param <T> Type of state + */ +public class KubernetesStateHandleStore<T extends Serializable> implements StateHandleStore<T> { + + private static final Logger LOG = LoggerFactory.getLogger(KubernetesStateHandleStore.class); + + private final FlinkKubeClient kubeClient; + + private final String configMapName; + + private final RetrievableStateStorageHelper<T> storage; + + private final Predicate<String> filter; + + private final String lockIdentity; + + private final Supplier<Exception> configMapNotExistSupplier; + + /** + * Creates a {@link KubernetesStateHandleStore}. + * + * @param kubeClient The Kubernetes client. + * @param storage To persist the actual state and whose returned state handle is then written to ConfigMap + * @param configMapName ConfigMap to store the state handle store pointer + * @param filter filter to get the expected keys for state handle + * @param lockIdentity lock identity of current HA service + */ + public KubernetesStateHandleStore( + FlinkKubeClient kubeClient, + String configMapName, + RetrievableStateStorageHelper<T> storage, + Predicate<String> filter, + String lockIdentity) { + + this.kubeClient = checkNotNull(kubeClient, "Kubernetes client"); + this.storage = checkNotNull(storage, "State storage"); + this.configMapName = checkNotNull(configMapName, "ConfigMap name"); + this.filter = checkNotNull(filter); + this.lockIdentity = checkNotNull(lockIdentity, "Lock identity of current HA service"); + + this.configMapNotExistSupplier = () -> new KubernetesException("ConfigMap " + configMapName + " not exists."); + } + + /** + * Creates a state handle, stores it in ConfigMap. We could guarantee that only the leader could update the + * ConfigMap. Since “Get(check the leader)-and-Update(write back to the ConfigMap)” is a + * transactional operation. + * + * @param key Key in ConfigMap + * @param state State to be added + * + * @throws AlreadyExistException if the name already exists + * @throws Exception if persisting state or writing state handle failed + */ + @Override + public RetrievableStateHandle<T> add(String key, T state) throws Exception { + checkNotNull(key, "Key in ConfigMap."); + checkNotNull(state, "State."); + + final RetrievableStateHandle<T> storeHandle = storage.store(state); + + boolean success = false; + + try { + final byte[] serializedStoreHandle = InstantiationUtil.serializeObject(storeHandle); + success = kubeClient.checkAndUpdateConfigMap( + configMapName, + c -> { + if (KubernetesLeaderElector.hasLeadership(c, lockIdentity)) { + if (!c.getData().containsKey(key)) { + c.getData().put(key, Base64.getEncoder().encodeToString(serializedStoreHandle)); + return Optional.of(c); + } else { + throw new CompletionException(new StateHandleStore.AlreadyExistException( + key + " already exists in ConfigMap " + configMapName)); + } + } + return Optional.empty(); + }).get(); + return storeHandle; + } catch (Exception ex) { + ExceptionUtils.rethrowException(ExceptionUtils.stripCompletionException(ex)); + // Never reached + return null; + } finally { + if (!success) { + // Cleanup the state handle if it was not written to ConfigMap. + if (storeHandle != null) { + storeHandle.discardState(); + } + } + } + } + + /** + * Replaces a state handle in ConfigMap and discards the old state handle. Wo do not lock resource + * version and then replace in Kubernetes. Since the ConfigMap is periodically updated by leader, the + * resource version changes very fast. We use a "check-existence and update" transactional operation instead. + * + * @param key Key in ConfigMap + * @param resourceVersion resource version when checking existence via {@link #exists}. + * @param state State to be added + * + * @throws NotExistException if the name does not exist + * @throws Exception if persisting state or writing state handle failed + */ + @Override + public void replace(String key, String resourceVersion, T state) throws Exception { + checkNotNull(key, "Key in ConfigMap."); + checkNotNull(state, "State."); + + final RetrievableStateHandle<T> oldStateHandle = get(key); + + final RetrievableStateHandle<T> newStateHandle = storage.store(state); + + boolean success = false; + + try { + final byte[] serializedStoreHandle = InstantiationUtil.serializeObject(newStateHandle); + success = kubeClient.checkAndUpdateConfigMap( + configMapName, + c -> { + if (KubernetesLeaderElector.hasLeadership(c, lockIdentity)) { + // Check the existence + if (c.getData().containsKey(key)) { + c.getData().put(key, Base64.getEncoder().encodeToString(serializedStoreHandle)); + } else { + throw new CompletionException(new StateHandleStore.NotExistException( + "Could not find " + key + " in ConfigMap " + configMapName)); + } + return Optional.of(c); + } + return Optional.empty(); + }).get(); + } catch (Exception ex) { + ExceptionUtils.rethrowException(ExceptionUtils.stripCompletionException(ex)); + } finally { + if (success) { + oldStateHandle.discardState(); + } else { + newStateHandle.discardState(); + } + } + } + + /** + * Returns the resource version of the ConfigMap or {@link StateHandleStore#NON_EXIST_RESOURCE_VERSION} if the key + * does not exist. + * + * @param key Key in ConfigMap + * + * @return resource version or {@link #NON_EXIST_RESOURCE_VERSION} when not exists. + * + * @throws Exception if the check existence operation failed + */ + @Override + public String exists(String key) throws Exception { + checkNotNull(key, "Key in ConfigMap."); + + return kubeClient.getConfigMap(configMapName) + .map(configMap -> { + if (configMap.getData().containsKey(key)) { + return configMap.getResourceVersion(); + } + return StateHandleStore.NON_EXIST_RESOURCE_VERSION; + }) + .orElseThrow(configMapNotExistSupplier); + } + + /** + * Gets the {@link RetrievableStateHandle} stored in the given ConfigMap. + * + * @param key Key in ConfigMap + * + * @return The retrieved state handle from the specified ConfigMap and key + * + * @throws IOException if the method failed to deserialize the stored state handle + * @throws NotExistException when the name does not exist + * @throws Exception if get state handle from ConfigMap failed + */ + @Override + public RetrievableStateHandle<T> get(String key) throws Exception { + checkNotNull(key, "Key in ConfigMap."); + + final Optional<KubernetesConfigMap> optional = kubeClient.getConfigMap(configMapName); + if (optional.isPresent()) { + final KubernetesConfigMap configMap = optional.get(); + if (configMap.getData().containsKey(key)) { + return deserializeObject(configMap.getData().get(key)); + } else { + throw new StateHandleStore.NotExistException( + "Could not find " + key + " in ConfigMap " + configMapName); + } + } else { + throw configMapNotExistSupplier.get(); + } + } + + /** + * Gets all available state handles sorted by key from Kubernetes. + * + * @return All state handles from ConfigMap. + * + * @throws Exception if get state handle from ConfigMap failed + */ + @SuppressWarnings("unchecked") + @Override + public List<Tuple2<RetrievableStateHandle<T>, String>> getAll() throws Exception { + + return kubeClient.getConfigMap(configMapName) + .map( + configMap -> { + final List<Tuple2<RetrievableStateHandle<T>, String>> stateHandles = new ArrayList<>(); + configMap.getData().entrySet().stream() + .filter(entry -> filter.test(entry.getKey())) + .sorted(Comparator.comparing(Map.Entry::getKey)) + .forEach(entry -> { + try { + stateHandles.add(new Tuple2(deserializeObject(entry.getValue()), entry.getKey())); + } catch (Exception e) { Review comment: We could. ########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesJobGraphStoreWatcherTest.java ########## @@ -0,0 +1,135 @@ +/* + * 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.kubernetes.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.jobmanager.JobGraphStoreWatcher; +import org.apache.flink.runtime.jobmanager.TestingJobGraphListener; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link KubernetesRunningJobsRegistry} operations. + */ +public class KubernetesJobGraphStoreWatcherTest extends KubernetesHighAvailabilityTestBase { + + private TestingJobGraphListener testingJobGraphListener; + + @Before + public void setup() { + super.setup(); + testingJobGraphListener = new TestingJobGraphListener(); + } + + @Test + public void testJobGraphAddedAndRemovedShouldNotifyJobGraphListener() throws Exception { + new Context() {{ + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final JobGraphStoreWatcher jobGraphStoreWatcher = + createAndStartJobGraphStoreWatcher(flinkKubeClient); + + final JobID jobID = JobID.generate(); + final String key = KubernetesUtils.jobIDToJobGraphConfigMapKey(jobID); + + assertThat(configMapCallbackFutures.size(), is(3)); + final FlinkKubeClient.WatchCallbackHandler<KubernetesConfigMap> callbackHandler = + configMapCallbackFutures.get(2).get(TIMEOUT, TimeUnit.MILLISECONDS); + // A new job graph is added + getLeaderConfigMap().getData().put(key, "job graph data"); + callbackHandler.onModified(Collections.singletonList(getLeaderConfigMap())); + + assertThat(testingJobGraphListener.getAddedJobGraphs(), contains(jobID)); + + // Job graph is deleted + getLeaderConfigMap().getData().remove(key); + callbackHandler.onModified(Collections.singletonList(getLeaderConfigMap())); + + assertThat(testingJobGraphListener.getRemovedJobGraphs(), contains(jobID)); + + jobGraphStoreWatcher.stop(); + }); + }}; + } + + /** + * We already have three job graphs in the store, job0 and job1 is removed and job3 is newly added. + */ + @Test + public void testMultipleJobGraphChanges() throws Exception { Review comment: Make sense. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java ########## @@ -0,0 +1,305 @@ +/* + * 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 TestingStateHandleStore.Builder<JobGraph> builder = TestingStateHandleStore.builder(); + private final JobGraph testingJobGraph = new JobGraph(); + private final long timeout = 3 * 1000; + + private TestingRetrievableStateStorageHelper<JobGraph> jobGraphStorageHelper; + private TestingJobGraphStoreWatcher testingJobGraphStoreWatcher = new TestingJobGraphStoreWatcher(); + private TestingJobGraphListener testingJobGraphListener = new TestingJobGraphListener(); + + @Before + public void setup() { + 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)); + } + } + + @Test + public void testPutJobGraphWhenNotExist() throws Exception { + final CompletableFuture<JobGraph> addFuture = new CompletableFuture<>(); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setExistsFunction(ignore -> NON_EXIST_RESOURCE_VERSION) + .setAddFunction((ignore, state) -> { + addFuture.complete(state); + return jobGraphStorageHelper.store(state); + }) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.putJobGraph(testingJobGraph); + + final JobGraph actual = addFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual.getJobID(), is(testingJobGraph.getJobID())); + } + + @Test + public void testPutJobGraphWhenAlreadyExist() throws Exception { + final CompletableFuture<Tuple3<String, String, JobGraph>> replaceFuture = new CompletableFuture<>(); + final String resourceVersion = "100"; + final AtomicBoolean alreadyExist = new AtomicBoolean(false); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setExistsFunction(ignore -> { + if (alreadyExist.get()) { + return resourceVersion; + } else { + alreadyExist.set(true); + return NON_EXIST_RESOURCE_VERSION; + } + }) + .setAddFunction((ignore, state) -> jobGraphStorageHelper.store(state)) + .setReplaceConsumer(replaceFuture::complete) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.putJobGraph(testingJobGraph); + // Replace + jobGraphStore.putJobGraph(testingJobGraph); + + final Tuple3<String, String, JobGraph> actual = replaceFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual.f0, is(testingJobGraph.getJobID().toString())); + assertThat(actual.f1, is(resourceVersion)); + assertThat(actual.f2.getJobID(), is(testingJobGraph.getJobID())); + } + + @Test + public void testRemoveJobGraph() throws Exception { + final CompletableFuture<JobID> removeFuture = new CompletableFuture<>(); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setAddFunction((ignore, state) -> jobGraphStorageHelper.store(state)) + .setRemoveFunction(name -> removeFuture.complete(JobID.fromHexString(name))) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + + jobGraphStore.putJobGraph(testingJobGraph); + jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + final JobID actual = removeFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual, is(testingJobGraph.getJobID())); + } + + @Test + public void testRemoveJobGraphWithNonExistName() throws Exception { + final CompletableFuture<JobID> removeFuture = new CompletableFuture<>(); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setRemoveFunction(name -> removeFuture.complete(JobID.fromHexString(name))) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + + try { + removeFuture.get(timeout, TimeUnit.MILLISECONDS); + fail("We should get an expected timeout because we are removing a non-existed job graph."); + } catch (TimeoutException ex) { + // expected + } + assertThat(removeFuture.isDone(), is(false)); + } + + @Test + public void testGetJobIds() throws Exception { + final List<JobID> existingJobIds = Arrays.asList(new JobID(0, 0), new JobID(0, 1)); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setGetAllNamesSupplier( + () -> existingJobIds.stream().map(AbstractID::toString).collect(Collectors.toList())) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + final Collection<JobID> jobIds = jobGraphStore.getJobIds(); + assertThat(jobIds, contains(existingJobIds.toArray())); + } + + @Test + public void testOnAddedJobGraphShouldNotProcessKnownJobGraphs() throws Exception { + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setAddFunction((ignore, state) -> jobGraphStorageHelper.store(state)) + .build(); + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.putJobGraph(testingJobGraph); + + testingJobGraphStoreWatcher.addJobGraph(testingJobGraph.getJobID()); + assertThat(testingJobGraphListener.getAddedJobGraphs().size(), is(0)); + } + + @Test + public void testOnAddedJobGraphShouldOnlyProcessUnknownJobGraphs() throws Exception { Review comment: You inspire that I should add known job graphs here. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java ########## @@ -0,0 +1,305 @@ +/* + * 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 TestingStateHandleStore.Builder<JobGraph> builder = TestingStateHandleStore.builder(); + private final JobGraph testingJobGraph = new JobGraph(); + private final long timeout = 3 * 1000; + + private TestingRetrievableStateStorageHelper<JobGraph> jobGraphStorageHelper; + private TestingJobGraphStoreWatcher testingJobGraphStoreWatcher = new TestingJobGraphStoreWatcher(); + private TestingJobGraphListener testingJobGraphListener = new TestingJobGraphListener(); + + @Before + public void setup() { + 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)); + } + } + + @Test + public void testPutJobGraphWhenNotExist() throws Exception { + final CompletableFuture<JobGraph> addFuture = new CompletableFuture<>(); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setExistsFunction(ignore -> NON_EXIST_RESOURCE_VERSION) + .setAddFunction((ignore, state) -> { + addFuture.complete(state); + return jobGraphStorageHelper.store(state); + }) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.putJobGraph(testingJobGraph); + + final JobGraph actual = addFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual.getJobID(), is(testingJobGraph.getJobID())); + } + + @Test + public void testPutJobGraphWhenAlreadyExist() throws Exception { + final CompletableFuture<Tuple3<String, String, JobGraph>> replaceFuture = new CompletableFuture<>(); + final String resourceVersion = "100"; + final AtomicBoolean alreadyExist = new AtomicBoolean(false); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setExistsFunction(ignore -> { + if (alreadyExist.get()) { + return resourceVersion; + } else { + alreadyExist.set(true); + return NON_EXIST_RESOURCE_VERSION; + } + }) + .setAddFunction((ignore, state) -> jobGraphStorageHelper.store(state)) + .setReplaceConsumer(replaceFuture::complete) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.putJobGraph(testingJobGraph); + // Replace + jobGraphStore.putJobGraph(testingJobGraph); + + final Tuple3<String, String, JobGraph> actual = replaceFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual.f0, is(testingJobGraph.getJobID().toString())); + assertThat(actual.f1, is(resourceVersion)); + assertThat(actual.f2.getJobID(), is(testingJobGraph.getJobID())); + } + + @Test + public void testRemoveJobGraph() throws Exception { + final CompletableFuture<JobID> removeFuture = new CompletableFuture<>(); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setAddFunction((ignore, state) -> jobGraphStorageHelper.store(state)) + .setRemoveFunction(name -> removeFuture.complete(JobID.fromHexString(name))) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + + jobGraphStore.putJobGraph(testingJobGraph); + jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + final JobID actual = removeFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual, is(testingJobGraph.getJobID())); + } + + @Test + public void testRemoveJobGraphWithNonExistName() throws Exception { + final CompletableFuture<JobID> removeFuture = new CompletableFuture<>(); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setRemoveFunction(name -> removeFuture.complete(JobID.fromHexString(name))) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + + try { + removeFuture.get(timeout, TimeUnit.MILLISECONDS); + fail("We should get an expected timeout because we are removing a non-existed job graph."); + } catch (TimeoutException ex) { + // expected + } + assertThat(removeFuture.isDone(), is(false)); + } + + @Test + public void testGetJobIds() throws Exception { + final List<JobID> existingJobIds = Arrays.asList(new JobID(0, 0), new JobID(0, 1)); + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setGetAllNamesSupplier( + () -> existingJobIds.stream().map(AbstractID::toString).collect(Collectors.toList())) + .build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + final Collection<JobID> jobIds = jobGraphStore.getJobIds(); + assertThat(jobIds, contains(existingJobIds.toArray())); + } + + @Test + public void testOnAddedJobGraphShouldNotProcessKnownJobGraphs() throws Exception { + final TestingStateHandleStore<JobGraph> stateHandleStore = builder + .setAddFunction((ignore, state) -> jobGraphStorageHelper.store(state)) + .build(); + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + jobGraphStore.putJobGraph(testingJobGraph); + + testingJobGraphStoreWatcher.addJobGraph(testingJobGraph.getJobID()); + assertThat(testingJobGraphListener.getAddedJobGraphs().size(), is(0)); + } + + @Test + public void testOnAddedJobGraphShouldOnlyProcessUnknownJobGraphs() throws Exception { Review comment: You inspire that I should add known job graphs here. Then the test makes more sense. ########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java ########## @@ -0,0 +1,356 @@ +/* + * 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.kubernetes.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.runtime.statehandle.StateHandleStore; +import org.apache.flink.runtime.statehandle.TestingLongStateHandleHelper; +import org.apache.flink.runtime.statehandle.TestingLongStateHandleHelper.LongStateStorage; +import org.apache.flink.util.function.FunctionUtils; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.function.Predicate; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link KubernetesStateHandleStore} operations. + */ +public class KubernetesStateHandleStoreTest extends KubernetesHighAvailabilityTestBase { Review comment: After checking the doc of `maven-surefire-plugin`, I realize that maybe we should not set the `reuseForks` but `forkCount `. Right? ---------------------------------------------------------------- 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]
