nicoweidner commented on a change in pull request #16801:
URL: https://github.com/apache/flink/pull/16801#discussion_r688377346



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalDriverFactory.java
##########
@@ -29,16 +29,28 @@
 
     private final String retrievalPath;
 
-    public ZooKeeperLeaderRetrievalDriverFactory(CuratorFramework client, 
String retrievalPath) {
+    private final 
ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+            leaderInformationClearancePolicy;
+
+    public ZooKeeperLeaderRetrievalDriverFactory(
+            CuratorFramework client,
+            String retrievalPath,
+            ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy

Review comment:
       Is it intentional that there are no `@NonNull` annotations here and in 
`ZooKeeperLeaderRetrievalDriver`?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalConnectionHandlingTest.java
##########
@@ -0,0 +1,390 @@
+/*
+ * 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.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.apache.curator.test.TestingServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance when
+ * retrieving the leader information.
+ */
+public class ZooKeeperLeaderRetrievalConnectionHandlingTest extends TestLogger 
{
+
+    private TestingServer testingServer;
+
+    private Configuration config;

Review comment:
       nit: can be local variable in `before()`

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -18,317 +18,228 @@
 
 package org.apache.flink.runtime.leaderelection;
 
-import org.apache.flink.api.common.time.Deadline;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
-import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
-import org.apache.flink.runtime.rpc.DirectlyFailingFatalErrorHandler;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
-import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
 import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.BiConsumerWithException;
 
 import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
 
-import org.apache.curator.test.TestingServer;
-import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
-import javax.annotation.Nullable;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectOutputStream;
 import java.time.Duration;
 import java.util.UUID;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.CoreMatchers.notNullValue;
-import static org.hamcrest.CoreMatchers.nullValue;
-import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertFalse;
 
-/** Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance. */
+/**
+ * Test behaviors of {@link ZooKeeperLeaderElectionDriver} when losing the 
connection to ZooKeeper.
+ */
 public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
 
-    private TestingServer testingServer;
+    private static final String PATH = "/path";
 
-    private Configuration config;
+    @Rule public final ZooKeeperResource zooKeeperResource = new 
ZooKeeperResource();
 
-    private CuratorFramework zooKeeperClient;
+    @Rule
+    public final TestingFatalErrorHandlerResource fatalErrorHandlerResource =
+            new TestingFatalErrorHandlerResource();
 
-    private final FatalErrorHandler fatalErrorHandler = 
DirectlyFailingFatalErrorHandler.INSTANCE;
+    private final Configuration configuration = new Configuration();
 
     @Before
-    public void before() throws Exception {
-        testingServer = new TestingServer();
-
-        config = new Configuration();
-        config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-        config.setString(
-                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
testingServer.getConnectString());
-
-        zooKeeperClient = ZooKeeperUtils.startCuratorFramework(config);
-        zooKeeperClient.blockUntilConnected();
-    }
-
-    @After
-    public void after() throws Exception {
-        closeTestServer();
-
-        if (zooKeeperClient != null) {
-            zooKeeperClient.close();
-            zooKeeperClient = null;
-        }
+    public void setup() {
+        configuration.set(
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
zooKeeperResource.getConnectString());
     }
 
     @Test
-    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
-        final QueueLeaderElectionListener queueLeaderElectionListener =
-                new QueueLeaderElectionListener(1);
-
-        LeaderRetrievalDriver leaderRetrievalDriver = null;
-        try {
-            leaderRetrievalDriver =
-                    
ZooKeeperUtils.createLeaderRetrievalDriverFactory(zooKeeperClient)
-                            .createLeaderRetrievalDriver(
-                                    queueLeaderElectionListener, 
fatalErrorHandler);
-
-            // do the testing
-            final CompletableFuture<String> firstAddress =
-                    queueLeaderElectionListener.next(Duration.ofMillis(50));
-            assertThat(
-                    "No results are expected, yet, since no leader was 
elected.",
-                    firstAddress,
-                    is(nullValue()));
-
-            closeTestServer();
-
-            // QueueLeaderElectionListener will be notified with an empty 
leader when ZK connection
-            // is suspended
-            final CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
-            assertThat("The next result must not be missing.", secondAddress, 
is(notNullValue()));
-            assertThat(
-                    "The next result is expected to be null.",
-                    secondAddress.get(),
-                    is(nullValue()));
-        } finally {
-            if (leaderRetrievalDriver != null) {
-                leaderRetrievalDriver.close();
-            }
-        }
+    public void testLoseLeadershipOnConnectionSuspended() throws Exception {
+        runTestWithSuspendedZooKeeperConnection(
+                configuration,
+                (connectionStateListener, contender) -> {
+                    connectionStateListener.awaitSuspendedConnection();
+                    contender.awaitRevokeLeadership(Duration.ofSeconds(1L));
+                });
     }
 
     @Test
-    public void testConnectionSuspendedHandling() throws Exception {
-        final String retrievalPath = "/testConnectionSuspendedHandling";
-        final String leaderAddress = "localhost";
-
-        final QueueLeaderElectionListener queueLeaderElectionListener =
-                new QueueLeaderElectionListener(1);
-        ZooKeeperLeaderRetrievalDriver leaderRetrievalDriver = null;
-        try {
-            leaderRetrievalDriver =
-                    new ZooKeeperLeaderRetrievalDriver(
-                            zooKeeperClient,
-                            retrievalPath,
-                            queueLeaderElectionListener,
-                            fatalErrorHandler);
-
-            writeLeaderInformationToZooKeeper(
-                    leaderRetrievalDriver.getConnectionInformationPath(),
-                    leaderAddress,
-                    UUID.randomUUID());
-
-            // do the testing
-            CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
-            assertThat(
-                    "The first result is expected to be the initially set 
leader address.",
-                    firstAddress.get(),
-                    is(leaderAddress));
-
-            closeTestServer();
-
-            CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
-            assertThat("The next result must not be missing.", secondAddress, 
is(notNullValue()));
-            assertThat(
-                    "The next result is expected to be null.",
-                    secondAddress.get(),
-                    is(nullValue()));
-        } finally {
-            if (leaderRetrievalDriver != null) {
-                leaderRetrievalDriver.close();
-            }
-        }
+    public void 
testKeepLeadershipOnSuspendedConnectionIfTolerateSuspendedConnectionsIsEnabled()
+            throws Exception {
+        
configuration.set(HighAvailabilityOptions.ZOOKEEPER_TOLERATE_SUSPENDED_CONNECTIONS,
 true);
+        runTestWithSuspendedZooKeeperConnection(
+                configuration,
+                (connectionStateListener, contender) -> {
+                    connectionStateListener.awaitSuspendedConnection();
+                    connectionStateListener.awaitReconnectedConnection();
+                    assertFalse(contender.hasRevokeLeadershipBeenTriggered());
+                });
     }
 
     @Test
-    public void testSameLeaderAfterReconnectTriggersListenerNotification() 
throws Exception {
-        final String retrievalPath = 
"/testSameLeaderAfterReconnectTriggersListenerNotification";
-        final QueueLeaderElectionListener queueLeaderElectionListener =
-                new QueueLeaderElectionListener(1);
-        ZooKeeperLeaderRetrievalDriver leaderRetrievalDriver = null;
-        try {
-            leaderRetrievalDriver =
-                    new ZooKeeperLeaderRetrievalDriver(
-                            zooKeeperClient,
-                            retrievalPath,
-                            queueLeaderElectionListener,
-                            fatalErrorHandler);
-
-            final String leaderAddress = "foobar";
-            final UUID sessionId = UUID.randomUUID();
-            writeLeaderInformationToZooKeeper(
-                    leaderRetrievalDriver.getConnectionInformationPath(), 
leaderAddress, sessionId);
+    public void 
testLoseLeadershipOnLostConnectionIfTolerateSuspendedConnectionsIsEnabled()

Review comment:
       The other two tests take ~3s on my machine, this one takes ~7s. I would 
have expected the difference to be roughly the size of the timeouts, do you 
know why it is that long?

##########
File path: 
docs/layouts/shortcodes/generated/high_availability_configuration.html
##########
@@ -62,6 +62,12 @@
             <td>Integer</td>
             <td>Defines the session timeout for the ZooKeeper session in 
ms.</td>
         </tr>
+        <tr>
+            
<td><h5>high-availability.zookeeper.client.tolerate-suspended-connections</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>Defines whether a suspended ZooKeeper connection will be 
treated as an error that causes the leader information to be invalidated or 
not. In case you set this option to <code 
class="highlighter-rouge">true</code>, Flink will wait until a ZooKeeper 
connection is marked as lost before it revokes the leadership of components. 
This has the effect that Flink is more resilient against temporary connection 
instabilities at the cost of running more likely into timing issues with 
ZooKeeper.</td>

Review comment:
       I am not sure which timing issues are meant here. Could you elaborate a 
bit?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalConnectionHandlingTest.java
##########
@@ -0,0 +1,390 @@
+/*
+ * 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.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.apache.curator.test.TestingServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance when
+ * retrieving the leader information.
+ */
+public class ZooKeeperLeaderRetrievalConnectionHandlingTest extends TestLogger 
{
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @Rule
+    public final TestingFatalErrorHandlerResource fatalErrorHandlerResource =
+            new TestingFatalErrorHandlerResource();
+
+    @Before
+    public void before() throws Exception {
+        testingServer = new TestingServer();
+
+        config = new Configuration();
+        config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");

Review comment:
       nit: It actually seems irrelevant to set this option. I don't really 
mind either way, but I would suggest using the same approach in 
`ZooKeeperLeaderElectionConnectionHandlingTest` (I got confused why it was not 
required there...)

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -18,317 +18,228 @@
 
 package org.apache.flink.runtime.leaderelection;
 
-import org.apache.flink.api.common.time.Deadline;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
-import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
-import org.apache.flink.runtime.rpc.DirectlyFailingFatalErrorHandler;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
-import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
 import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.BiConsumerWithException;
 
 import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
 
-import org.apache.curator.test.TestingServer;
-import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
-import javax.annotation.Nullable;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectOutputStream;
 import java.time.Duration;
 import java.util.UUID;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.CoreMatchers.notNullValue;
-import static org.hamcrest.CoreMatchers.nullValue;
-import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertFalse;
 
-/** Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance. */
+/**
+ * Test behaviors of {@link ZooKeeperLeaderElectionDriver} when losing the 
connection to ZooKeeper.
+ */
 public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
 
-    private TestingServer testingServer;
+    private static final String PATH = "/path";
 
-    private Configuration config;
+    @Rule public final ZooKeeperResource zooKeeperResource = new 
ZooKeeperResource();
 
-    private CuratorFramework zooKeeperClient;
+    @Rule
+    public final TestingFatalErrorHandlerResource fatalErrorHandlerResource =
+            new TestingFatalErrorHandlerResource();
 
-    private final FatalErrorHandler fatalErrorHandler = 
DirectlyFailingFatalErrorHandler.INSTANCE;
+    private final Configuration configuration = new Configuration();
 
     @Before
-    public void before() throws Exception {
-        testingServer = new TestingServer();
-
-        config = new Configuration();
-        config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-        config.setString(
-                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
testingServer.getConnectString());
-
-        zooKeeperClient = ZooKeeperUtils.startCuratorFramework(config);
-        zooKeeperClient.blockUntilConnected();
-    }
-
-    @After
-    public void after() throws Exception {
-        closeTestServer();
-
-        if (zooKeeperClient != null) {
-            zooKeeperClient.close();
-            zooKeeperClient = null;
-        }
+    public void setup() {
+        configuration.set(
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
zooKeeperResource.getConnectString());
     }
 
     @Test
-    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
-        final QueueLeaderElectionListener queueLeaderElectionListener =
-                new QueueLeaderElectionListener(1);
-
-        LeaderRetrievalDriver leaderRetrievalDriver = null;
-        try {
-            leaderRetrievalDriver =
-                    
ZooKeeperUtils.createLeaderRetrievalDriverFactory(zooKeeperClient)
-                            .createLeaderRetrievalDriver(
-                                    queueLeaderElectionListener, 
fatalErrorHandler);
-
-            // do the testing
-            final CompletableFuture<String> firstAddress =
-                    queueLeaderElectionListener.next(Duration.ofMillis(50));
-            assertThat(
-                    "No results are expected, yet, since no leader was 
elected.",
-                    firstAddress,
-                    is(nullValue()));
-
-            closeTestServer();
-
-            // QueueLeaderElectionListener will be notified with an empty 
leader when ZK connection
-            // is suspended
-            final CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
-            assertThat("The next result must not be missing.", secondAddress, 
is(notNullValue()));
-            assertThat(
-                    "The next result is expected to be null.",
-                    secondAddress.get(),
-                    is(nullValue()));
-        } finally {
-            if (leaderRetrievalDriver != null) {
-                leaderRetrievalDriver.close();
-            }
-        }
+    public void testLoseLeadershipOnConnectionSuspended() throws Exception {
+        runTestWithSuspendedZooKeeperConnection(
+                configuration,
+                (connectionStateListener, contender) -> {
+                    connectionStateListener.awaitSuspendedConnection();
+                    contender.awaitRevokeLeadership(Duration.ofSeconds(1L));
+                });
     }
 
     @Test
-    public void testConnectionSuspendedHandling() throws Exception {
-        final String retrievalPath = "/testConnectionSuspendedHandling";
-        final String leaderAddress = "localhost";
-
-        final QueueLeaderElectionListener queueLeaderElectionListener =
-                new QueueLeaderElectionListener(1);
-        ZooKeeperLeaderRetrievalDriver leaderRetrievalDriver = null;
-        try {
-            leaderRetrievalDriver =
-                    new ZooKeeperLeaderRetrievalDriver(
-                            zooKeeperClient,
-                            retrievalPath,
-                            queueLeaderElectionListener,
-                            fatalErrorHandler);
-
-            writeLeaderInformationToZooKeeper(
-                    leaderRetrievalDriver.getConnectionInformationPath(),
-                    leaderAddress,
-                    UUID.randomUUID());
-
-            // do the testing
-            CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
-            assertThat(
-                    "The first result is expected to be the initially set 
leader address.",
-                    firstAddress.get(),
-                    is(leaderAddress));
-
-            closeTestServer();
-
-            CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
-            assertThat("The next result must not be missing.", secondAddress, 
is(notNullValue()));
-            assertThat(
-                    "The next result is expected to be null.",
-                    secondAddress.get(),
-                    is(nullValue()));
-        } finally {
-            if (leaderRetrievalDriver != null) {
-                leaderRetrievalDriver.close();
-            }
-        }
+    public void 
testKeepLeadershipOnSuspendedConnectionIfTolerateSuspendedConnectionsIsEnabled()
+            throws Exception {
+        
configuration.set(HighAvailabilityOptions.ZOOKEEPER_TOLERATE_SUSPENDED_CONNECTIONS,
 true);
+        runTestWithSuspendedZooKeeperConnection(

Review comment:
       ```suggestion
           runTestWithBrieflySuspendedZooKeeperConnection(
   ```
   More of a nit, I just had to think a bit (or rather check the 
implementation) to understand that it reconnects after a short delay

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalConnectionHandlingTest.java
##########
@@ -0,0 +1,390 @@
+/*
+ * 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.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalDriver;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalEventHandler;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalDriver;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
+
+import org.apache.curator.test.TestingServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.time.Duration;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance when
+ * retrieving the leader information.
+ */
+public class ZooKeeperLeaderRetrievalConnectionHandlingTest extends TestLogger 
{
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @Rule
+    public final TestingFatalErrorHandlerResource fatalErrorHandlerResource =
+            new TestingFatalErrorHandlerResource();
+
+    @Before
+    public void before() throws Exception {
+        testingServer = new TestingServer();
+
+        config = new Configuration();
+        config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+        config.setString(
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
testingServer.getConnectString());
+
+        zooKeeperClient = ZooKeeperUtils.startCuratorFramework(config);
+        zooKeeperClient.blockUntilConnected();
+    }
+
+    @After
+    public void after() throws Exception {
+        closeTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        final QueueLeaderElectionListener queueLeaderElectionListener =
+                new QueueLeaderElectionListener(1);
+
+        LeaderRetrievalDriver leaderRetrievalDriver = null;
+        try {
+            leaderRetrievalDriver =
+                    
ZooKeeperUtils.createLeaderRetrievalDriverFactory(zooKeeperClient)
+                            .createLeaderRetrievalDriver(
+                                    queueLeaderElectionListener,
+                                    
fatalErrorHandlerResource.getFatalErrorHandler());
+
+            // do the testing
+            final CompletableFuture<String> firstAddress =
+                    queueLeaderElectionListener.next(Duration.ofMillis(50));
+            assertThat(
+                    "No results are expected, yet, since no leader was 
elected.",
+                    firstAddress,
+                    is(nullValue()));
+
+            closeTestServer();
+
+            // QueueLeaderElectionListener will be notified with an empty 
leader when ZK connection
+            // is suspended
+            final CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+            assertThat("The next result must not be missing.", secondAddress, 
is(notNullValue()));
+            assertThat(
+                    "The next result is expected to be null.",
+                    secondAddress.get(),
+                    is(nullValue()));
+        } finally {
+            if (leaderRetrievalDriver != null) {
+                leaderRetrievalDriver.close();
+            }
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandling() throws Exception {
+        final String retrievalPath = "/testConnectionSuspendedHandling";
+        final String leaderAddress = "localhost";
+
+        final QueueLeaderElectionListener queueLeaderElectionListener =
+                new QueueLeaderElectionListener(1);
+        ZooKeeperLeaderRetrievalDriver leaderRetrievalDriver = null;
+        try {
+            leaderRetrievalDriver =
+                    new ZooKeeperLeaderRetrievalDriver(
+                            zooKeeperClient,
+                            retrievalPath,
+                            queueLeaderElectionListener,
+                            
ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                    .ON_SUSPENDED_CONNECTION,
+                            fatalErrorHandlerResource.getFatalErrorHandler());
+
+            writeLeaderInformationToZooKeeper(
+                    leaderRetrievalDriver.getConnectionInformationPath(),
+                    leaderAddress,
+                    UUID.randomUUID());
+
+            // do the testing
+            CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
+            assertThat(
+                    "The first result is expected to be the initially set 
leader address.",
+                    firstAddress.get(),
+                    is(leaderAddress));
+
+            closeTestServer();
+
+            CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+            assertThat("The next result must not be missing.", secondAddress, 
is(notNullValue()));
+            assertThat(
+                    "The next result is expected to be null.",
+                    secondAddress.get(),
+                    is(nullValue()));
+        } finally {
+            if (leaderRetrievalDriver != null) {
+                leaderRetrievalDriver.close();
+            }
+        }
+    }
+
+    @Test
+    public void 
testSuspendedConnectionDoesNotClearLeaderInformationIfClearanceOnLostConnection()
+            throws Exception {
+        final String retrievalPath = "/testConnectionSuspendedHandling";
+        final String leaderAddress = "localhost";
+
+        final QueueLeaderElectionListener queueLeaderElectionListener =
+                new QueueLeaderElectionListener(1);
+        ZooKeeperLeaderRetrievalDriver leaderRetrievalDriver = null;
+        try {
+            leaderRetrievalDriver =
+                    new ZooKeeperLeaderRetrievalDriver(
+                            zooKeeperClient,
+                            retrievalPath,
+                            queueLeaderElectionListener,
+                            
ZooKeeperLeaderRetrievalDriver.LeaderInformationClearancePolicy
+                                    .ON_LOST_CONNECTION,
+                            fatalErrorHandlerResource.getFatalErrorHandler());
+
+            writeLeaderInformationToZooKeeper(
+                    leaderRetrievalDriver.getConnectionInformationPath(),
+                    leaderAddress,
+                    UUID.randomUUID());
+
+            // do the testing
+            CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
+            assertThat(
+                    "The first result is expected to be the initially set 
leader address.",
+                    firstAddress.get(),
+                    is(leaderAddress));
+
+            closeTestServer();

Review comment:
       nit: I would prefer to call `testingServer.restart()` here (assuming it 
would reconnect after it is back up) to avoid the potential of actually being 
counted as a lost connection.
   In practice, this is irrelevant of course since we only wait for 100ms...




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