zentol commented on code in PR #22384:
URL: https://github.com/apache/flink/pull/22384#discussion_r1201761208
##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java:
##########
@@ -32,87 +29,49 @@
*/
public class TestingLeaderElectionService implements LeaderElectionService {
- private LeaderContender contender = null;
- private boolean hasLeadership = false;
- private CompletableFuture<LeaderConnectionInfo> confirmationFuture = null;
- private CompletableFuture<Void> startFuture = new CompletableFuture<>();
- private UUID issuedLeaderSessionId = null;
-
- /**
- * Gets a future that completes when leadership is confirmed.
- *
- * <p>Note: the future is created upon calling {@link #isLeader(UUID)}.
- */
- public synchronized CompletableFuture<LeaderConnectionInfo>
getConfirmationFuture() {
- return confirmationFuture;
- }
+ private final TestingLeaderElection startedLeaderElection = new
TestingLeaderElection();
@Override
- public synchronized void start(LeaderContender contender) {
- Preconditions.checkState(!getStartFuture().isDone());
-
- this.contender = contender;
-
- if (hasLeadership) {
- contender.grantLeadership(issuedLeaderSessionId);
- }
-
- startFuture.complete(null);
+ public synchronized LeaderElection createLeaderElection() {
+ return startedLeaderElection;
}
@Override
public synchronized void stop() throws Exception {
- if (hasLeadership && contender != null) {
- contender.revokeLeadership();
- }
-
- contender = null;
- hasLeadership = false;
- issuedLeaderSessionId = null;
- startFuture.cancel(false);
- startFuture = new CompletableFuture<>();
+ startedLeaderElection.triggerContenderCleanup();
}
- @Override
- public synchronized void confirmLeadership(UUID leaderSessionID, String
leaderAddress) {
- if (confirmationFuture != null) {
- confirmationFuture.complete(new
LeaderConnectionInfo(leaderSessionID, leaderAddress));
- }
+ public synchronized CompletableFuture<LeaderInformation> isLeader(UUID
leaderSessionID) {
+ return startedLeaderElection.isLeader(leaderSessionID);
}
- @Override
- public synchronized boolean hasLeadership(@Nonnull UUID leaderSessionId) {
- return hasLeadership && leaderSessionId.equals(issuedLeaderSessionId);
+ public synchronized LeaderInformation isConfirmedLeader(UUID
leaderSessionID) {
Review Comment:
The intention is good but I was a tad confused why isLeader and
isConfirmedLeader return different types.
I feel like calling `isLeader.join()` explicitly in the test, or renaming
this method to `isLeaderAndWaitForConfirm` may alleviate this.
##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElection.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * {@code TestingLeaderElection} implements simple leader election for test
cases where no {@code
+ * LeaderElectionService} is required.
+ */
+public class TestingLeaderElection implements LeaderElection {
+
+ /**
+ * Is {@code null} if the {@code LeaderElection} isn't started.
+ *
+ * @see LeaderElection#startLeaderElection(LeaderContender)
+ */
+ @Nullable private LeaderContender contender = null;
+
+ @Nullable private CompletableFuture<LeaderInformation> confirmationFuture
= null;
Review Comment:
wondering if we should cancel this in 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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]