wangyang0918 commented on a change in pull request #13864: URL: https://github.com/apache/flink/pull/13864#discussion_r517116763
########## 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: Actually, Kubernetes could not ensure that the `resourceVersion` is always `integer`. I will add a `ResourceVersion` as interface for the return result. ---------------------------------------------------------------- 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]
