XComp commented on code in PR #21137:
URL: https://github.com/apache/flink/pull/21137#discussion_r1006684103


##########
flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java:
##########
@@ -683,21 +665,66 @@ public void testJobAlreadyDone() throws Exception {
                     jobManagerRunner.getResultFuture();
 
             JobManagerRunnerResult result = resultFuture.get();
-            assertEquals(
-                    JobStatus.FAILED,
-                    
result.getExecutionGraphInfo().getArchivedExecutionGraph().getState());
+            
assertThat(result.getExecutionGraphInfo().getArchivedExecutionGraph().getState())
+                    .isEqualTo(JobStatus.FAILED);
         }
     }
 
+    @Test
+    void 
testJobMasterServiceLeadershipRunnerCloseWhenElectionServiceGrantLeaderShip()
+            throws Exception {
+        final TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory
+                testingLeaderElectionDriverFactory =
+                        new 
TestingLeaderElectionDriver.TestingLeaderElectionDriverFactory();
+        final LeaderElectionService defaultLeaderElectionService =
+                new 
DefaultLeaderElectionService(testingLeaderElectionDriverFactory);
+
+        final JobMasterServiceLeadershipRunner jobManagerRunner =
+                newJobMasterServiceLeadershipRunnerBuilder()
+                        .setJobMasterServiceProcessFactory(
+                                
TestingJobMasterServiceProcessFactory.newBuilder().build())
+                        .setLeaderElectionService(defaultLeaderElectionService)
+                        .build();
+
+        jobManagerRunner.start();
+        final TestingLeaderElectionDriver currentLeaderDriver =
+                Preconditions.checkNotNull(
+                        
testingLeaderElectionDriverFactory.getCurrentLeaderDriver());
+
+        final CheckedThread contenderCloseThread =
+                new CheckedThread() {
+                    @Override
+                    public void go() {
+                        try {
+                            jobManagerRunner.close();
+                        } catch (Exception e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                };
+        contenderCloseThread.start();
+
+        // grant leadership.
+        currentLeaderDriver.isLeader();

Review Comment:
   This test is not necessarily failing when running it without the fix you're 
proposing in this PR. The `close()` call of `jobManagerRunner` needs to be 
halted to influence the concurrent processing of the 
`currentLeaderDriver.isLeader()` and the `jobManagerRunner.close()` method. 
Ideally, this method should fail/block consistently without the fix but should 
succeed with the fix.
   I guess, you could achieve that by inducing your own 
`TestingJobMasterServiceProcess` through the 
`TestingJobMasterServiceProcessFactory` in [line 
685](https://github.com/apache/flink/pull/21137/files#diff-c010bcd788571261ea6c43a17ad7aaea719a64a7b8a080d76939e7dbbc5a4b32R685).
 This `TestingJobMasterServiceProcess` gives you control over the 
`CompletableFuture` that's returned by [JobMasterServiceProcess.closeAsync() in 
JobMasterServiceLeadershipRunner.closeAsync()](https://github.com/apache/flink/blob/bfe4f9cc3d67d37a2258ab4226d70b6a7d24f22c/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java#L145).
 The `JobMasterServiceLeadershipRunner` will only proceeed with stopping the 
`DefaultLeaderElectionService` if this `CompletableFuture` completes. WDYT?



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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to