wangyang0918 commented on a change in pull request #15501: URL: https://github.com/apache/flink/pull/15501#discussion_r636683446
########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesSharedInformer.java ########## @@ -0,0 +1,285 @@ +/* + * 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.kubeclient.resources; + +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient.WatchCallbackHandler; +import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.Preconditions; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.KubernetesResourceList; +import io.fabric8.kubernetes.client.NamespacedKubernetesClient; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import io.fabric8.kubernetes.client.informers.SharedInformerFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; +import java.util.function.Function; + +/** Base class for shared watcher based on {@link SharedIndexInformer}. */ +public abstract class KubernetesSharedInformer< + T extends HasMetadata, TList extends KubernetesResourceList<T>, R> + implements KubernetesSharedWatcher<R> { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + private final NamespacedKubernetesClient client; + private final SharedIndexInformer<T> sharedIndexInformer; + private final Function<T, R> eventWrapper; + + private final ExecutorService watchingExecutor; + + private final Object processorsLock = new Object(); + + @GuardedBy("processorsLock") + private final Map<String, Processor> watchingProcessors = new HashMap<>(); + + public KubernetesSharedInformer( + NamespacedKubernetesClient client, + Class<T> apiTypeClass, + Class<TList> apiListTypeClass, + Map<String, String> labels, + Function<T, R> eventWrapper) { + Preconditions.checkArgument( + !CollectionUtil.isNullOrEmpty(labels), "Labels must not be null or empty"); + this.client = client; + SharedInformerFactory informerFactory = + client.informers( + Executors.newSingleThreadExecutor( Review comment: Do we need to close the single thread executor? ########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesSharedInformer.java ########## @@ -0,0 +1,285 @@ +/* + * 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.kubeclient.resources; + +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient.WatchCallbackHandler; +import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.Preconditions; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.KubernetesResourceList; +import io.fabric8.kubernetes.client.NamespacedKubernetesClient; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import io.fabric8.kubernetes.client.informers.SharedInformerFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; +import java.util.function.Function; + +/** Base class for shared watcher based on {@link SharedIndexInformer}. */ +public abstract class KubernetesSharedInformer< + T extends HasMetadata, TList extends KubernetesResourceList<T>, R> + implements KubernetesSharedWatcher<R> { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + private final NamespacedKubernetesClient client; + private final SharedIndexInformer<T> sharedIndexInformer; + private final Function<T, R> eventWrapper; + + private final ExecutorService watchingExecutor; + + private final Object processorsLock = new Object(); + + @GuardedBy("processorsLock") + private final Map<String, Processor> watchingProcessors = new HashMap<>(); + + public KubernetesSharedInformer( + NamespacedKubernetesClient client, + Class<T> apiTypeClass, + Class<TList> apiListTypeClass, + Map<String, String> labels, + Function<T, R> eventWrapper) { + Preconditions.checkArgument( + !CollectionUtil.isNullOrEmpty(labels), "Labels must not be null or empty"); + this.client = client; + SharedInformerFactory informerFactory = + client.informers( + Executors.newSingleThreadExecutor( + new ExecutorThreadFactory("KubernetesClient-Informer"))); + informerFactory.withLabels(labels); + this.sharedIndexInformer = + informerFactory.sharedIndexInformerFor(apiTypeClass, apiListTypeClass, 0); + this.sharedIndexInformer.addEventHandler( + new ResourceEventHandler<T>() { + @Override + public void onAdd(T obj) { + notify(obj.getMetadata().getName()); + } + + @Override + public void onUpdate(T oldObj, T newObj) { + notify(newObj.getMetadata().getName()); + } + + @Override + public void onDelete(T obj, boolean deletedFinalStateUnknown) { + notify(obj.getMetadata().getName()); + } + + private void notify(String name) { + final String resourceKey = getResourceKey(name); + synchronized (processorsLock) { + Optional.ofNullable(watchingProcessors.get(resourceKey)) + .ifPresent(processor -> processor.triggerEvent(resourceKey)); + } + } + }); + + this.watchingExecutor = + Executors.newCachedThreadPool( + new ExecutorThreadFactory("KubernetesSharedInformer-Watching-Processor")); + this.eventWrapper = eventWrapper; + } + + @Override + public Watch watch(String name, WatchCallbackHandler<R> handler) { + final String resourceKey = getResourceKey(name); + synchronized (processorsLock) { + Processor existed = watchingProcessors.get(resourceKey); + if (existed != null) { + final Optional<UUID> uuid = existed.registerListener(handler); + if (uuid.isPresent()) { + return () -> existed.unregisterListener(uuid.get()); + } + } + Processor processor = new Processor(resourceKey); Review comment: nit: Use `final` as much as possible. Maybe we could also have a comment here. Something like "Create a new one if the processor is stopped or does not existed". ########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesSharedInformerITCase.java ########## @@ -0,0 +1,165 @@ +/* + * 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.kubeclient.resources; + +import org.apache.flink.kubernetes.KubernetesResource; +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.KubernetesConfigMapSharedWatcher; +import org.apache.flink.kubernetes.kubeclient.KubernetesSharedWatcher.Watch; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMap; + +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; +import org.junit.ClassRule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; + +/** IT Tests for the {@link KubernetesSharedInformer}. */ +public class KubernetesSharedInformerITCase extends TestLogger { + + @ClassRule public static KubernetesResource kubernetesResource = new KubernetesResource(); + + @Test + public void testWatch() throws Exception { + final ImmutableMap<String, String> labels = + ImmutableMap.of("app", "shared-informer-test-cluster"); + final String configMapPrefix = "shared-informer-test-cluster"; + try (FlinkKubeClient client = kubernetesResource.getFlinkKubeClient(); + KubernetesConfigMapSharedWatcher configMapSharedWatcher = + client.createConfigMapSharedWatcher(labels)) { + + configMapSharedWatcher.run(); + createConfigMapsInRange(client, labels, configMapPrefix, 0, 5); + + List<TestingCallbackHandler> callbackHandlers = new ArrayList<>(); + List<Watch> watchers = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + final String name = configMapPrefix + i % 10; + final TestingCallbackHandler handler = new TestingCallbackHandler(name); + callbackHandlers.add(handler); + final Watch watch = configMapSharedWatcher.watch(name, handler); + watchers.add(watch); + } + + createConfigMapsInRange(client, labels, configMapPrefix, 5, 10); + for (int i = 0; i < 10; i++) { + final String configMapName = configMapPrefix + i; + client.checkAndUpdateConfigMap( + configMapName, + configMap -> { + configMap.getData().put("foo", "bar"); + return Optional.of(configMap); + }) + .get(); + } + try { + for (TestingCallbackHandler handler : callbackHandlers) { + handler.addFuture.get(2000, TimeUnit.MILLISECONDS); + handler.addOrUpdateFuture.get(2000, TimeUnit.MILLISECONDS); + } + client.deleteConfigMapsByLabels(labels).get(); + for (TestingCallbackHandler handler : callbackHandlers) { + handler.deleteFuture.get(2000, TimeUnit.MILLISECONDS); + } + } finally { + client.deleteConfigMapsByLabels(labels).get(); + } + watchers.forEach(Watch::close); + } + } + + private void createConfigMapsInRange( + FlinkKubeClient client, + Map<String, String> labels, + String namePrefix, + int start, + int end) + throws Exception { + for (int i = start; i < end; i++) { + client.createConfigMap( + new KubernetesConfigMap( + new ConfigMapBuilder() + .withNewMetadata() + .withName(namePrefix + i) + .withLabels(labels) + .endMetadata() + .build())) + .get(); + } + } + + private static final class TestingCallbackHandler + extends NoOpWatchCallbackHandler<KubernetesConfigMap> { + + private final CompletableFuture<Void> addFuture = new CompletableFuture<>(); + private final CompletableFuture<Void> addOrUpdateFuture = new CompletableFuture<>(); + private final CompletableFuture<Void> deleteFuture = new CompletableFuture<>(); + + private final String name; + + private TestingCallbackHandler(String name) { + this.name = name; + } + + @Override + public void onAdded(List<KubernetesConfigMap> resources) { + final KubernetesConfigMap kubernetesConfigMap = resources.get(0); + assertThat(kubernetesConfigMap.getName(), is(name)); + addFuture.complete(null); + final String foo = kubernetesConfigMap.getData().get("foo"); + if (foo != null) { + assertThat(foo, is("bar")); + addOrUpdateFuture.complete(null); + } + } + + @Override + public void onModified(List<KubernetesConfigMap> resources) { + final KubernetesConfigMap kubernetesConfigMap = resources.get(0); + assertThat(kubernetesConfigMap.getName(), is(name)); + final String foo = kubernetesConfigMap.getData().get("foo"); + assertThat(foo, is("bar")); + if (addOrUpdateFuture.isDone()) { + assertNotNull( + kubernetesConfigMap + .getInternalResource() + .getMetadata() + .getDeletionTimestamp()); + } else { Review comment: Could we guarantee that the `onAdded` always happens before `onModified`? -- 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]
