zentol commented on code in PR #22384:
URL: https://github.com/apache/flink/pull/22384#discussion_r1196648554


##########
flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/AbstractLeaderElectionService.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.runtime.leaderelection;
+
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.UUID;
+
+/**
+ * {@code AbstractLeaderElectionService} provides a generic implementation of 
the {@link
+ * LeaderElection} handling.
+ */
+public abstract class AbstractLeaderElectionService implements 
LeaderElectionService {
+    @Override
+    public LeaderElection createLeaderElection() {
+        return new LeaderElectionImpl(this);
+    }
+
+    /**
+     * Registers the given {@link LeaderContender} with the underlying {@code
+     * LeaderElectionService}. Leadership changes are starting to be reported 
to the {@code
+     * LeaderContender}.
+     */
+    protected abstract void register(LeaderContender contender) throws 
Exception;
+
+    /** Confirms the leadership with the given session ID and address. */
+    protected abstract void confirmLeadership(UUID leaderSessionID, String 
leaderAddress);
+
+    /**
+     * Checks whether the {@code LeaderElectionService} has the leadership 
acquired for the given
+     * session ID.
+     *
+     * @return {@code true} if the service has leadership with the passed 
session ID acquired;
+     *     {@code false} otherwise.
+     */
+    protected abstract boolean hasLeadership(UUID leaderSessionId);
+
+    /** {@code LeaderElectionImpl} is the default implementation of {@link 
LeaderElection}. */
+    private static class LeaderElectionImpl implements LeaderElection {

Review Comment:
   can we move this into a dedicated class and rename it to 
`DefaultLeaderElection`?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java:
##########
@@ -28,7 +28,7 @@
  * Test {@link LeaderElectionService} implementation which directly forwards 
isLeader and notLeader
  * calls to the contender.
  */
-public class TestingLeaderElectionService implements LeaderElectionService {
+public class TestingLeaderElectionService extends 
AbstractLeaderElectionService {

Review Comment:
   I don't know, it seems odd that this relies on a concrete implementation.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingDispatcherRestEndpoint.java:
##########
@@ -86,20 +86,20 @@ public List<Tuple2<RestHandlerSpecification, 
ChannelInboundHandler>> initializeH
         return super.initializeHandlers(localAddressFuture);
     }
 
-    private enum NoOpElectionService implements LeaderElectionService {
-        INSTANCE;
+    private static class NoOpElectionService extends 
AbstractLeaderElectionService {

Review Comment:
   Shouldn't this keep implementing `LeaderElectionService` to avoid 
dependencies on runtime internals?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java:
##########
@@ -41,13 +42,20 @@ public TestingContender(
         this.leaderElectionService = leaderElectionService;
     }
 
+    public LeaderElection startLeaderElection() throws Exception {
+        leaderElection = leaderElectionService.createLeaderElection();

Review Comment:
   should this maybe be set up in the constructor? Maybe even the leader 
election start.
   
   So far, all prod usages set this up when a component is started, and the 
equivalent in this class for that is the constructor.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalTest.java:
##########
@@ -120,6 +121,7 @@ void 
testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exception
         long sleepingTime = 1000;
 
         LeaderElectionService leaderElectionService = null;
+        LeaderElection leaderElection = null;

Review Comment:
   unused?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java:
##########
@@ -147,7 +147,8 @@ private void runTestWithZooKeeperConnectionProblem(
             
client.getConnectionStateListenable().addListener(connectionStateListener);
 
             final TestingContender contender = new TestingContender();
-            leaderElectionService.start(contender);
+            final LeaderElection leaderElection = 
leaderElectionService.createLeaderElection();

Review Comment:
   nit: variable may be unnecessary.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to