xintongsong commented on a change in pull request #14485:
URL: https://github.com/apache/flink/pull/14485#discussion_r548399979



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java
##########
@@ -44,8 +50,12 @@ public void start(LeaderContender newContender) throws 
Exception {
 
                contender = Preconditions.checkNotNull(newContender);
 
-               // directly grant leadership to the given contender
-               
contender.grantLeadership(HighAvailabilityServices.DEFAULT_LEADER_ID);
+               // Grant leadership in a separate thread
+               leaderElectorExecutor = Executors.newSingleThreadExecutor(
+                       new ExecutorThreadFactory("LeaderElectorExecutor"));
+               CompletableFuture.runAsync(
+                       () -> 
contender.grantLeadership(HighAvailabilityServices.DEFAULT_LEADER_ID),
+                       leaderElectorExecutor);

Review comment:
       I think having to invoke `grantLeadership` in a separate thread is 
probably an indicator that, the internal correctness of `JobManagerRunnerImpl` 
is depending on how its interfaces are called.
   
   IIUC, `JobManagerRunnerImpl` is trying to keep the following steps in order.
   1. `leaderElectionService.start()`
   2. create `jobMasterService`
   3. `grantLeadership` 
   
   Instead of introducing additional requirements on how and when leader 
election service should invoke `grantLeadership`, an alternative is to allow 
calling `grantLeadership` from any thread at any time, and delay the actual 
processing inside `grantLeadership` if needed.
   
   E.g., something like the following.
   ```
   public class JobManagerRunnerImpl implements XXX {
       private final CompletableFuture<JobMasterService> jmServiceFuture = new 
CompletableFuture<>();
       public JobManagerRunnerImpl(XXX) {
           synchronized (lock) {
               leaderElectionService.start();
               
jmServiceFuture.complete(jobMasterFactory.createJobMasterService(XXX));
           }
       }
       public void grantLeadership(XXX) {
           jmServiceFuture.whenComplete((xxx) -> {
               synchronized (lock) {
                   XXX
               }
           })
       }
   }
   ```
   This should help encapsulate all the logics needed for keeping the above 
mentioned order inside `JobManagerRunnerImpl`.

##########
File path: 
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.kubernetes.KubernetesResource;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.flink.kubernetes.utils.Constants.LABEL_CONFIGMAP_TYPE_HIGH_AVAILABILITY;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for recovering from savepoint when Kubernetes HA is enabled. The 
savepoint will be persisted as a checkpoint
+ * and stored in the ConfigMap when recovered successfully.
+ */
+public class KubernetesHighAvailabilityRecoverFromSavepointITCase extends 
TestLogger {
+
+       private static final long TIMEOUT = 60 * 1000;
+
+       private static final String CLUSTER_ID = "flink-on-k8s-cluster-" + 
System.currentTimeMillis();
+
+       private static final String FLAT_MAP_UID = "my-flat-map";
+
+       @ClassRule
+       public static KubernetesResource kubernetesResource = new 
KubernetesResource();
+
+       @ClassRule
+       public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+       @Rule
+       public MiniClusterWithClientResource miniClusterResource = new 
MiniClusterWithClientResource(
+               new MiniClusterResourceConfiguration.Builder()
+                       .setConfiguration(getConfiguration())
+                       .setNumberTaskManagers(1)
+                       .setNumberSlotsPerTaskManager(1)
+                       .build());
+
+       private FlinkKubeClient flinkKubeClient;
+
+       private ClusterClient clusterClient;

Review comment:
       IDE is complaining about "raw use of parameterized class"
   ```suggestion
        private ClusterClient<?> clusterClient;
   ```

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
##########
@@ -53,7 +59,7 @@ public synchronized void start(LeaderContender contender) {
                this.contender = contender;
 
                if (hasLeadership) {
-                       contender.grantLeadership(issuedLeaderSessionId);
+                       CompletableFuture.runAsync(() -> 
contender.grantLeadership(issuedLeaderSessionId), leaderElectorExecutor);

Review comment:
       This reveals the same problem as `StandaloneLeaderElectionService` does.




----------------------------------------------------------------
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]


Reply via email to