wangyang0918 commented on a change in pull request #13864: URL: https://github.com/apache/flink/pull/13864#discussion_r517871011
########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java ########## @@ -0,0 +1,422 @@ +/* + * 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.Collections; +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)); Review comment: Since the exception could be wrapped(e.g. `RetryException`), I will use the following code to throw the correct exception. ``` throw ExceptionUtils.findThrowable(ex, NotExistException.class).orElseThrow(() -> ex); ``` ---------------------------------------------------------------- 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]
