xintongsong commented on a change in pull request #13864: URL: https://github.com/apache/flink/pull/13864#discussion_r516476923
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/statehandle/StateHandleStore.java ########## @@ -0,0 +1,170 @@ +/* + * 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.statehandle; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RetrievableStateHandle; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +/** + * Class which stores state via the provided {@link RetrievableStateStorageHelper} and writes the + * returned state handle to distributed coordination system(e.g. Zookeeper, Kubernetes, etc.). + * + * <p>To avoid concurrent modification issues, we need to ensure that only the leader could update the state store. + * For Zookeeper, we need to lock a node(aka create an ephemeral node under the specified node to be locked) and release + * (aka delete the ephemeral node). + * For Kubernetes, we could perform a {@link FlinkKubeClient#checkAndUpdateConfigMap} transactional operation for this. + * Then we will completely get rid of the lock-and-release operations. + * + * <p>We do not define the interfaces with lock/release. Because they are not common requirements for different + * implementations. Then the {@link org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore} + * needs some additional interfaces to release the locks. Review comment: I think these descriptions do not belong to the JavaDoc of this class. How to ensure that only leader updates the state store should be described in the ZK/K8s implementation of this interface. Getting rid of `lock-and-release` explains about the change rather than the interface, thus probably belongs to the commit message. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/statehandle/StateHandleStore.java ########## @@ -0,0 +1,170 @@ +/* + * 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.statehandle; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RetrievableStateHandle; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +/** + * Class which stores state via the provided {@link RetrievableStateStorageHelper} and writes the + * returned state handle to distributed coordination system(e.g. Zookeeper, Kubernetes, etc.). + * + * <p>To avoid concurrent modification issues, we need to ensure that only the leader could update the state store. + * For Zookeeper, we need to lock a node(aka create an ephemeral node under the specified node to be locked) and release + * (aka delete the ephemeral node). + * For Kubernetes, we could perform a {@link FlinkKubeClient#checkAndUpdateConfigMap} transactional operation for this. + * Then we will completely get rid of the lock-and-release operations. + * + * <p>We do not define the interfaces with lock/release. Because they are not common requirements for different + * implementations. Then the {@link org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore} + * needs some additional interfaces to release the locks. + * + * @param <T> Type of state + */ +public interface StateHandleStore<T extends Serializable> { + + String NON_EXIST_RESOURCE_VERSION = "-1"; + + /** + * Persist the state to distributed storage(e.g. S3, HDFS, etc.). And then creates a state handle, stores it in + * the distributed coordination system(e.g. ZooKeeper, Kubernetes, etc.). + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @param state State to be added + * + * @throws AlreadyExistException if the name already exists + * @throws Exception if persisting state or writing state handle failed + */ + RetrievableStateHandle<T> add(String name, T state) throws Exception; + + /** + * Replaces a state handle in the distributed coordination system and discards the old state handle. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @param resourceVersion resource version of previous storage object. If the resource version does not match, the + * replace operation will fail. Since there is an unexpected update operation snuck in. + * @param state State to be replace with + * + * @throws NotExistException if the name does not exist + * @throws Exception if persisting state or writing state handle failed + */ + void replace(String name, String resourceVersion, T state) throws Exception; + + /** + * Returns resource version or {@link #NON_EXIST_RESOURCE_VERSION} if the name does not exist. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return current resource version in {@link String} if exist. Or {@link #NON_EXIST_RESOURCE_VERSION} if the name + * does not exist. + * + * @throws Exception if the check existence operation failed + */ + String exists(String name) throws Exception; + + /** + * Gets the {@link RetrievableStateHandle} stored with the given name. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return The retrieved state handle + * + * @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 failed + */ + RetrievableStateHandle<T> get(String name) throws Exception; + + /** + * Gets all available state handles from the storage. + * + * @return All retrieved state handles. + * + * @throws Exception if get state handle operation failed + */ + List<Tuple2<RetrievableStateHandle<T>, String>> getAll() throws Exception; + + /** + * Return a list of all valid name for state handles. + * + * @return List of valid state handle name. The name is key name in ConfigMap or child path name in ZooKeeper. + * + * @throws Exception if get name operation failed + */ + Collection<String> getAllNames() throws Exception; + + /** + * Remove the state handle and discard the state with given name. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return True if the state handle could be removed. + * + * @throws Exception if removing the names or discarding the state failed + */ + boolean remove(String name) throws Exception; + + /** + * Remove all the states. Not only the state handles in the distributed coordination system + * will be removed, but also the real state data on the distributed storage will be discarded. + * + * @throws Exception if removing the names or discarding the state failed + */ + void removeAll() throws Exception; + + /** + * Only remove all the state handle pointers on Kubernetes or ZooKeeper. + * + * @throws Exception if removing the names failed + */ + void removeAllNames() throws Exception; + + /** + * The key does not exist in ConfigMap or the Zookeeper node does not exists. + */ + class NotExistException extends Exception { + /** + * Creates a new Exception with the given message and null as the cause. + * + * @param message The exception message + */ + public NotExistException(String message) { + super(message); + } + } + + /** + * The key already exists in ConfigMap or the Zookeeper node already exists. + */ + class AlreadyExistException extends Exception { Review comment: IDE complains about missing `serialVersionUID`. ########## 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)); + } + } + + @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)); Review comment: ``` @Test(expected = TimeoutException.class) ... try { removeFuture.get(timeout, TimeUnit.MILLISECONDS); } finally { assertThat(removeFuture.isDone(), is(false)); } ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/statehandle/StateHandleStore.java ########## @@ -0,0 +1,170 @@ +/* + * 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.statehandle; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RetrievableStateHandle; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +/** + * Class which stores state via the provided {@link RetrievableStateStorageHelper} and writes the + * returned state handle to distributed coordination system(e.g. Zookeeper, Kubernetes, etc.). + * + * <p>To avoid concurrent modification issues, we need to ensure that only the leader could update the state store. + * For Zookeeper, we need to lock a node(aka create an ephemeral node under the specified node to be locked) and release + * (aka delete the ephemeral node). + * For Kubernetes, we could perform a {@link FlinkKubeClient#checkAndUpdateConfigMap} transactional operation for this. + * Then we will completely get rid of the lock-and-release operations. + * + * <p>We do not define the interfaces with lock/release. Because they are not common requirements for different + * implementations. Then the {@link org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore} + * needs some additional interfaces to release the locks. + * + * @param <T> Type of state + */ +public interface StateHandleStore<T extends Serializable> { + + String NON_EXIST_RESOURCE_VERSION = "-1"; + + /** + * Persist the state to distributed storage(e.g. S3, HDFS, etc.). And then creates a state handle, stores it in + * the distributed coordination system(e.g. ZooKeeper, Kubernetes, etc.). + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @param state State to be added + * + * @throws AlreadyExistException if the name already exists + * @throws Exception if persisting state or writing state handle failed + */ + RetrievableStateHandle<T> add(String name, T state) throws Exception; + + /** + * Replaces a state handle in the distributed coordination system and discards the old state handle. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @param resourceVersion resource version of previous storage object. If the resource version does not match, the + * replace operation will fail. Since there is an unexpected update operation snuck in. + * @param state State to be replace with + * + * @throws NotExistException if the name does not exist + * @throws Exception if persisting state or writing state handle failed + */ + void replace(String name, String resourceVersion, T state) throws Exception; + + /** + * Returns resource version or {@link #NON_EXIST_RESOURCE_VERSION} if the name does not exist. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return current resource version in {@link String} if exist. Or {@link #NON_EXIST_RESOURCE_VERSION} if the name + * does not exist. + * + * @throws Exception if the check existence operation failed + */ + String exists(String name) throws Exception; Review comment: Not sure about changing the type of resource version to `String`. I'm aware that the fabric8 API provides resource version as a string. The problem is that we lose the comparability, i.e., which one among the versions is the latest. If the Kubernetes resource version is guaranteed to be an integer value, I would suggest to convert it to `int` and keep using `int` for this interface. If it's not guaranteed to be an integer value, we could make `resourceVersion` and interface with comparing methods, and provide different implementations for it. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/statehandle/StateHandleStore.java ########## @@ -0,0 +1,170 @@ +/* + * 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.statehandle; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RetrievableStateHandle; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +/** + * Class which stores state via the provided {@link RetrievableStateStorageHelper} and writes the + * returned state handle to distributed coordination system(e.g. Zookeeper, Kubernetes, etc.). + * + * <p>To avoid concurrent modification issues, we need to ensure that only the leader could update the state store. + * For Zookeeper, we need to lock a node(aka create an ephemeral node under the specified node to be locked) and release + * (aka delete the ephemeral node). + * For Kubernetes, we could perform a {@link FlinkKubeClient#checkAndUpdateConfigMap} transactional operation for this. + * Then we will completely get rid of the lock-and-release operations. + * + * <p>We do not define the interfaces with lock/release. Because they are not common requirements for different + * implementations. Then the {@link org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore} + * needs some additional interfaces to release the locks. + * + * @param <T> Type of state + */ +public interface StateHandleStore<T extends Serializable> { + + String NON_EXIST_RESOURCE_VERSION = "-1"; + + /** + * Persist the state to distributed storage(e.g. S3, HDFS, etc.). And then creates a state handle, stores it in + * the distributed coordination system(e.g. ZooKeeper, Kubernetes, etc.). + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @param state State to be added + * + * @throws AlreadyExistException if the name already exists + * @throws Exception if persisting state or writing state handle failed + */ + RetrievableStateHandle<T> add(String name, T state) throws Exception; + + /** + * Replaces a state handle in the distributed coordination system and discards the old state handle. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @param resourceVersion resource version of previous storage object. If the resource version does not match, the + * replace operation will fail. Since there is an unexpected update operation snuck in. + * @param state State to be replace with + * + * @throws NotExistException if the name does not exist + * @throws Exception if persisting state or writing state handle failed + */ + void replace(String name, String resourceVersion, T state) throws Exception; + + /** + * Returns resource version or {@link #NON_EXIST_RESOURCE_VERSION} if the name does not exist. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return current resource version in {@link String} if exist. Or {@link #NON_EXIST_RESOURCE_VERSION} if the name + * does not exist. + * + * @throws Exception if the check existence operation failed + */ + String exists(String name) throws Exception; + + /** + * Gets the {@link RetrievableStateHandle} stored with the given name. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return The retrieved state handle + * + * @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 failed + */ + RetrievableStateHandle<T> get(String name) throws Exception; + + /** + * Gets all available state handles from the storage. + * + * @return All retrieved state handles. + * + * @throws Exception if get state handle operation failed + */ + List<Tuple2<RetrievableStateHandle<T>, String>> getAll() throws Exception; + + /** + * Return a list of all valid name for state handles. + * + * @return List of valid state handle name. The name is key name in ConfigMap or child path name in ZooKeeper. + * + * @throws Exception if get name operation failed + */ + Collection<String> getAllNames() throws Exception; + + /** + * Remove the state handle and discard the state with given name. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * + * @return True if the state handle could be removed. + * + * @throws Exception if removing the names or discarding the state failed + */ + boolean remove(String name) throws Exception; + + /** + * Remove all the states. Not only the state handles in the distributed coordination system + * will be removed, but also the real state data on the distributed storage will be discarded. + * + * @throws Exception if removing the names or discarding the state failed + */ + void removeAll() throws Exception; + + /** + * Only remove all the state handle pointers on Kubernetes or ZooKeeper. + * + * @throws Exception if removing the names failed + */ + void removeAllNames() throws Exception; + + /** + * The key does not exist in ConfigMap or the Zookeeper node does not exists. + */ + class NotExistException extends Exception { Review comment: IDE complains about missing `serialVersionUID`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java ########## @@ -0,0 +1,329 @@ +/* + * 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.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.state.RetrievableStateHandle; +import org.apache.flink.runtime.statehandle.StateHandleStore; +import org.apache.flink.util.FlinkException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Default implementation for {@link JobGraphStore}. Combined with different {@link StateHandleStore}, we could persist + * the job graphs to various distributed storage. Also combined with different {@link JobGraphStoreWatcher}, we could + * get all the changes on the job graph store and do the response. + * + * <p>We have to make some variables {@link #lock}, {@link #addedJobGraphs}, and method {@link #verifyIsRunning()} + * as protected so that they could be used in the {@link ZooKeeperJobGraphStore}. + */ +public class DefaultJobGraphStore implements JobGraphStore, JobGraphStore.JobGraphListener { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultJobGraphStore.class); + + /** Lock to synchronize with the {@link JobGraphListener}. */ + protected final Object lock = new Object(); + + /** The set of IDs of all added job graphs. */ + @GuardedBy("lock") + protected final Set<JobID> addedJobGraphs = new HashSet<>(); + + /** Submitted job graphs handle store. */ + private final StateHandleStore<JobGraph> jobGraphStateHandleStore; + + private final JobGraphStoreWatcher jobGraphStoreWatcher; + + /** The job graph store description. Usually it is the storage name + * (e.g. ZooKeeper path or Kubernetes ConfigMap name). */ + private final String jobGraphStoreDescription; + + private final Function<String, JobID> nameToJobIDConverter; + + private final Function<JobID, String> jobIDToNameConverter; + + /** The external listener to be notified on races. */ + @GuardedBy("lock") + private JobGraphListener jobGraphListener; + + /** Flag indicating whether this instance is running. */ + @GuardedBy("lock") + private volatile boolean running; + + public DefaultJobGraphStore( + StateHandleStore<JobGraph> stateHandleStore, + JobGraphStoreWatcher jobGraphStoreWatcher, + String jobGraphStoreDescription, + Function<String, JobID> nameToJobIDConverter, + Function<JobID, String> jobIDToNameConverter) { Review comment: Might be better to pass in an interface `JobGraphStoreUtil/Helper` that wraps these two functions. ########## 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(); Review comment: These two could be `final`. ########## 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 think we can define a custom exception type, and use `@Test(expect = xxx)` for the validation ########## 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)); + } + } + + @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()); Review comment: I think there is an extra removal. ########## 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())); Review comment: ``` assertThat(recoveredJobGraph, notNullValue()); ``` ---------------------------------------------------------------- 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