This is an automated email from the ASF dual-hosted git repository. dmvk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 058b1f5045d607a1985ee063618e3819b14fb75d Author: David Moravek <d...@apache.org> AuthorDate: Fri May 5 10:09:49 2023 +0200 [FLINK-32010][kubernetes] Remove an unused parameter from KubernetesLeaderRetrievalDriver. --- .../highavailability/KubernetesLeaderRetrievalDriver.java | 4 ---- .../highavailability/KubernetesLeaderRetrievalDriverFactory.java | 6 ------ .../KubernetesMultipleComponentLeaderRetrievalDriverFactory.java | 1 - .../highavailability/KubernetesHighAvailabilityTestBase.java | 1 - .../KubernetesLeaderElectionAndRetrievalITCase.java | 1 - 5 files changed, 13 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriver.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriver.java index fb3516a30ac..33e88ba903f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriver.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriver.java @@ -49,8 +49,6 @@ public class KubernetesLeaderRetrievalDriver implements LeaderRetrievalDriver { private static final Logger LOG = LoggerFactory.getLogger(KubernetesLeaderRetrievalDriver.class); - private final FlinkKubeClient kubeClient; - private final String configMapName; private final LeaderRetrievalEventHandler leaderRetrievalEventHandler; @@ -64,14 +62,12 @@ public class KubernetesLeaderRetrievalDriver implements LeaderRetrievalDriver { private final Function<KubernetesConfigMap, LeaderInformation> leaderInformationExtractor; public KubernetesLeaderRetrievalDriver( - FlinkKubeClient kubeClient, KubernetesConfigMapSharedWatcher configMapSharedWatcher, Executor watchExecutor, String configMapName, LeaderRetrievalEventHandler leaderRetrievalEventHandler, Function<KubernetesConfigMap, LeaderInformation> leaderInformationExtractor, FatalErrorHandler fatalErrorHandler) { - this.kubeClient = checkNotNull(kubeClient, "Kubernetes client"); this.configMapName = checkNotNull(configMapName, "ConfigMap name"); this.leaderRetrievalEventHandler = checkNotNull(leaderRetrievalEventHandler, "LeaderRetrievalEventHandler"); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverFactory.java index cba8d65e9f0..408f0f6f15f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverFactory.java @@ -18,7 +18,6 @@ package org.apache.flink.kubernetes.highavailability; -import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; import org.apache.flink.kubernetes.kubeclient.KubernetesConfigMapSharedWatcher; import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriverFactory; @@ -35,19 +34,15 @@ import java.util.concurrent.Executor; @Deprecated public class KubernetesLeaderRetrievalDriverFactory implements LeaderRetrievalDriverFactory { - private final FlinkKubeClient kubeClient; - private final KubernetesConfigMapSharedWatcher configMapSharedWatcher; private final Executor watchExecutor; private final String configMapName; public KubernetesLeaderRetrievalDriverFactory( - FlinkKubeClient kubeClient, KubernetesConfigMapSharedWatcher configMapSharedWatcher, Executor watchExecutor, String configMapName) { - this.kubeClient = kubeClient; this.configMapSharedWatcher = configMapSharedWatcher; this.watchExecutor = watchExecutor; this.configMapName = configMapName; @@ -57,7 +52,6 @@ public class KubernetesLeaderRetrievalDriverFactory implements LeaderRetrievalDr public KubernetesLeaderRetrievalDriver createLeaderRetrievalDriver( LeaderRetrievalEventHandler leaderEventHandler, FatalErrorHandler fatalErrorHandler) { return new KubernetesLeaderRetrievalDriver( - kubeClient, configMapSharedWatcher, watchExecutor, configMapName, diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderRetrievalDriverFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderRetrievalDriverFactory.java index 134f197d954..44aaf7210e8 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderRetrievalDriverFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderRetrievalDriverFactory.java @@ -66,7 +66,6 @@ public class KubernetesMultipleComponentLeaderRetrievalDriverFactory public LeaderRetrievalDriver createLeaderRetrievalDriver( LeaderRetrievalEventHandler leaderEventHandler, FatalErrorHandler fatalErrorHandler) { return new KubernetesLeaderRetrievalDriver( - kubeClient, configMapSharedWatcher, watchExecutor, configMapName, diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java index 8367949bd09..b898338f5db 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java @@ -162,7 +162,6 @@ class KubernetesHighAvailabilityTestBase { private LeaderRetrievalDriver createLeaderRetrievalDriver() { final KubernetesLeaderRetrievalDriverFactory factory = new KubernetesLeaderRetrievalDriverFactory( - flinkKubeClient, kubernetesTestFixture.getConfigMapSharedWatcher(), watchCallbackExecutorService, LEADER_CONFIGMAP_NAME); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java index c52c89994f9..4d922cbda06 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java @@ -91,7 +91,6 @@ class KubernetesLeaderElectionAndRetrievalITCase { new TestingLeaderRetrievalEventHandler(); leaderRetrievalDriver = new KubernetesLeaderRetrievalDriver( - flinkKubeClient, configMapSharedWatcher, watchExecutorService, configMapName,