You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "XComp (via GitHub)" <gi...@apache.org> on 2023/04/13 13:01:26 UTC

[GitHub] [flink] XComp opened a new pull request, #22390: [WIP][FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

XComp opened a new pull request, #22390:
URL: https://github.com/apache/flink/pull/22390

   * [FLINK-30765](https://issues.apache.org/jira/browse/FLINK-30765) (PR https://github.com/apache/flink/pull/21742)
   * [FLINK-31768](https://issues.apache.org/jira/browse/FLINK-31768) (PR https://github.com/apache/flink/pull/22379)
   * [FLINK-31773](https://issues.apache.org/jira/browse/FLINK-31773) (PR https://github.com/apache/flink/pull/22380)
   * [FLINK-31776](https://issues.apache.org/jira/browse/FLINK-31776) (PR https://github.com/apache/flink/pull/22384)
   * FLINK-31785/FLINK-31786
   * 
   ## What is the purpose of the change
   
   This PR moves the leader election deregistration from the `LeaderElectionService` interface into `LeaderElectionService.LeaderElection`.
   
   ## Brief change log
   
   * Introduced `AbstractLeaderElectionService.remove` method that is then used by `LeaderElection.close`
   * Updated the tests accordingly.
   
   ## Verifying this change
   
   * Existing tests should still succeed
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): (yes / no / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: yes
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] XComp commented on a diff in pull request #22390: [FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on code in PR #22390:
URL: https://github.com/apache/flink/pull/22390#discussion_r1200741844


##########
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImplTest.java:
##########
@@ -105,17 +108,15 @@ public void teardown() throws Exception {
             resourceManagerService.close();
         }
 
-        if (leaderElectionService != null) {
-            leaderElectionService.stop();
-        }
-
         if (fatalErrorHandler.hasExceptionOccurred()) {
             fatalErrorHandler.rethrowError();
         }
     }
 
     @AfterClass
     public static void teardownClass() throws Exception {
+        haService.closeAndCleanupAllData();

Review Comment:
   tbh I don't know anymore why I added this. The haServices are testing implementations anyway. The LeaderElection will be cleaned up through the `ResourceManagerServiceImpl.close()` call in [line 108](https://github.com/XComp/flink/blob/FLINK-31785/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImplTest.java#L108)



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] XComp commented on pull request #22390: [FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on PR #22390:
URL: https://github.com/apache/flink/pull/22390#issuecomment-1568349397

   Thanks for the approval. I squashed the commits and rebased the branch for a final CI run.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zentol commented on a diff in pull request #22390: [FLINK-31785][FLINK-31786][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #22390:
URL: https://github.com/apache/flink/pull/22390#discussion_r1200368122


##########
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImplTest.java:
##########
@@ -79,6 +80,8 @@ public class ResourceManagerServiceImplTest extends TestLogger {
 
     private TestingResourceManagerFactory.Builder rmFactoryBuilder;
     private TestingLeaderElectionService leaderElectionService;
+    private LeaderElection leaderElection;

Review Comment:
   :thinking: 



##########
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServiceImplTest.java:
##########
@@ -105,17 +108,15 @@ public void teardown() throws Exception {
             resourceManagerService.close();
         }
 
-        if (leaderElectionService != null) {
-            leaderElectionService.stop();
-        }
-
         if (fatalErrorHandler.hasExceptionOccurred()) {
             fatalErrorHandler.rethrowError();
         }
     }
 
     @AfterClass
     public static void teardownClass() throws Exception {
+        haService.closeAndCleanupAllData();

Review Comment:
   This is required because we dont have access to the created `LeaderElection`? Shouldn't this use `TestingLeaderElectionService#triggerContenderCleanup`?



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] XComp commented on pull request #22390: [FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on PR #22390:
URL: https://github.com/apache/flink/pull/22390#issuecomment-1560570908

   I addressed the comments, did some minor changes and rebased the PR onto the most-recent version of #22384 


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] XComp commented on pull request #22390: [FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp commented on PR #22390:
URL: https://github.com/apache/flink/pull/22390#issuecomment-1563845998

   Did a rebase to `master` after FLINK-31776 (PR #22384) was merged.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] flinkbot commented on pull request #22390: [WIP][FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "flinkbot (via GitHub)" <gi...@apache.org>.
flinkbot commented on PR #22390:
URL: https://github.com/apache/flink/pull/22390#issuecomment-1506940038

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "152e71f5cffd7626fed20f57387a256c2c078a53",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "152e71f5cffd7626fed20f57387a256c2c078a53",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 152e71f5cffd7626fed20f57387a256c2c078a53 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zentol commented on pull request #22390: [FLINK-31785][FLINK-31786][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on PR #22390:
URL: https://github.com/apache/flink/pull/22390#issuecomment-1557019841

   Note: Maybe move FLINK-31786 into a separate PR because we could already merge that.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] XComp merged pull request #22390: [FLINK-31785][runtime] Moves LeaderElectionService.stop into LeaderElection.close

Posted by "XComp (via GitHub)" <gi...@apache.org>.
XComp merged PR #22390:
URL: https://github.com/apache/flink/pull/22390


-- 
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-unsubscribe@flink.apache.org

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