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


##########
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:
   hm, that made me start thinking. I guess, you're right: The 
`confirmationFuture` is bound to the `LeaderContender`. Resetting the contender 
should, indeed, also cause the `confirmationFuture` to be cancelled.
   
   But I did another round of digging: The `triggerContenderCleanup` was 
actually only exposed because of 
[DispatcherCleanupITCase:309](https://github.com/apache/flink/blob/master/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java#L309)
 where it should have been used instead of calling stop (which felt unnatural 
in this case because we didn't actually wanted to stop the leader election but 
just wanted to reset the instance).
   
   The test itself seems to be odd: We actually don't need to reset the leader 
election because the following code would just start a cleanup process which 
doesn't rely on leader election anymore. This change happened in FLINK-25432 
(in https://github.com/apache/flink/commit/cc5d321d). The test wasn't properly 
cleaned up/refactored to reflect the new behavior. Therefore, we could just 
remove the leader election reset. As a consequence, there wouldn't be a need to 
expose the `triggerContenderCleanup`. 
   
   I'm going to provide a hotfix commit to clean the test up and revert the 
`triggerContenderCleanup` method exposure. The test code shouldn't be able to 
reset the leader election because it might make it possible to workaround badly 
structured code.



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