tillrohrmann commented on a change in pull request #13055:
URL: https://github.com/apache/flink/pull/13055#discussion_r464889065



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @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);
+    }
+
+    @After
+    public void after() throws Exception {
+        stopTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(1, Duration.ofSeconds(1));
+
+        ZooKeeperLeaderRetrievalService testInstance = 
ZooKeeperUtils.createLeaderRetrievalService(zooKeeperClient, config);
+        testInstance.start(queueLeaderElectionListener);
+
+        // do the testing
+        CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
+        assertThat("No results are expected, yet, since no leader was 
elected.", firstAddress, is(nullValue()));
+
+        stopTestServer();
+
+        CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+        assertThat("No result is expected since there was no leader elected 
before stopping the server, yet.", secondAddress, is(nullValue()));
+    }
+
+    @Test
+    public void testConnectionSuspendedHandling() throws Exception {
+        // initialize LeaderElection-related instances
+        String leaderAddress = "localhost";
+        LeaderElectionService leaderElectionService = 
ZooKeeperUtils.createLeaderElectionService(zooKeeperClient, config);
+        TestingContender contender = new TestingContender(leaderAddress, 
leaderElectionService);
+        leaderElectionService.start(contender);
+
+        // initialize LeaderRetrievalService

Review comment:
       This is somewhat obvious.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
##########
@@ -209,4 +211,15 @@ protected void handleStateChange(ConnectionState newState) 
{
        public void unhandledError(String s, Throwable throwable) {
                leaderListener.handleError(new FlinkException("Unhandled error 
in ZooKeeperLeaderRetrievalService:" + s, throwable));
        }
+
+       private void notifyLeaderLoss() {
+               if (lastLeaderAddress != null || lastLeaderSessionID != null) {
+                       LOG.debug(
+                                       "No leader information could be 
retrieved. Any listeners will be notified.");
+
+                       lastLeaderAddress = null;
+                       lastLeaderSessionID = null;
+                       leaderListener.notifyLeaderAddress(null, null);
+               }
+       }

Review comment:
       This method is duplicating logic which is already contained in 
`ZooKeeperLeaderRetrievalService.nodeChanged`. I would suggest to factor out a 
method `notifyIfNewLeaderAddress(String newLeaderAddress, UUID 
newLeaderSessionID)`. `notifyLeaderLoss` could call this method with 
`notifyIfNewLeaderAddress(null, null)`.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {

Review comment:
       The class uses whitespaces for indentation. The Flink community uses 
tabs. Here is a pointer on how to set up your IDE to import Flink's checkstyle 
definition: 
https://ci.apache.org/projects/flink/flink-docs-stable/flinkDev/ide_setup.html.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @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);
+    }
+
+    @After
+    public void after() throws Exception {
+        stopTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(1, Duration.ofSeconds(1));
+
+        ZooKeeperLeaderRetrievalService testInstance = 
ZooKeeperUtils.createLeaderRetrievalService(zooKeeperClient, config);
+        testInstance.start(queueLeaderElectionListener);
+
+        // do the testing
+        CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
+        assertThat("No results are expected, yet, since no leader was 
elected.", firstAddress, is(nullValue()));
+
+        stopTestServer();
+
+        CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+        assertThat("No result is expected since there was no leader elected 
before stopping the server, yet.", secondAddress, is(nullValue()));
+    }
+
+    @Test
+    public void testConnectionSuspendedHandling() throws Exception {
+        // initialize LeaderElection-related instances
+        String leaderAddress = "localhost";
+        LeaderElectionService leaderElectionService = 
ZooKeeperUtils.createLeaderElectionService(zooKeeperClient, config);
+        TestingContender contender = new TestingContender(leaderAddress, 
leaderElectionService);
+        leaderElectionService.start(contender);
+
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(2, Duration.ofSeconds(1));
+
+        ZooKeeperLeaderRetrievalService testInstance = 
ZooKeeperUtils.createLeaderRetrievalService(zooKeeperClient, config);
+        testInstance.start(queueLeaderElectionListener);
+
+        // 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));
+
+        stopTestServer();
+
+        CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+        assertThat("The next result must not be missing.", secondAddress, 
not(is(nullValue())));
+        assertThat("The next result is expected to be null.", 
secondAddress.get(), is(nullValue()));
+    }
+
+    private void stopTestServer() throws IOException {
+        if (testingServer != null) {
+            testingServer.stop();
+            testingServer = null;
+        }
+    }
+
+    private static class QueueLeaderElectionListener implements 
LeaderRetrievalListener {
+
+        private final BlockingQueue<CompletableFuture<String>> queue;
+        private final Duration timeout;
+
+        public QueueLeaderElectionListener(int expectedCalls, Duration 
timeout) {
+            this.queue = new ArrayBlockingQueue<>(expectedCalls);
+            this.timeout = timeout;
+        }
+
+        @Override
+        public void notifyLeaderAddress(String leaderAddress, UUID 
leaderSessionID) {
+            try {
+                
this.queue.offer(CompletableFuture.completedFuture(leaderAddress), 
timeout.toMillis(), TimeUnit.MILLISECONDS);
+            } catch (InterruptedException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        public CompletableFuture<String> next() {
+            try {
+                return this.queue.poll(timeout.toMillis(), 
TimeUnit.MILLISECONDS);
+            } catch (InterruptedException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        @Override
+        public void handleError(Exception exception) {
+            // nothing to do

Review comment:
       Let's throw the exception at least. Otherwise we might swallow it which 
could hide a problem and which makes debugging harder.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
##########
@@ -209,4 +211,15 @@ protected void handleStateChange(ConnectionState newState) 
{
        public void unhandledError(String s, Throwable throwable) {
                leaderListener.handleError(new FlinkException("Unhandled error 
in ZooKeeperLeaderRetrievalService:" + s, throwable));
        }
+
+       private void notifyLeaderLoss() {
+               if (lastLeaderAddress != null || lastLeaderSessionID != null) {
+                       LOG.debug(
+                                       "No leader information could be 
retrieved. Any listeners will be notified.");
+
+                       lastLeaderAddress = null;
+                       lastLeaderSessionID = null;

Review comment:
       The state changes are not happening under the `lock`. If you introduce 
`notifyIfNewLeaderAddress`, then this call should happen under the `lock` and 
we could add `@GuardedBy("lock")` to `notifyIfNewLeaderAddress` to state this 
contract.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @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);
+    }
+
+    @After
+    public void after() throws Exception {
+        stopTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(1, Duration.ofSeconds(1));
+
+        ZooKeeperLeaderRetrievalService testInstance = 
ZooKeeperUtils.createLeaderRetrievalService(zooKeeperClient, config);
+        testInstance.start(queueLeaderElectionListener);
+
+        // do the testing
+        CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
+        assertThat("No results are expected, yet, since no leader was 
elected.", firstAddress, is(nullValue()));
+
+        stopTestServer();
+
+        CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+        assertThat("No result is expected since there was no leader elected 
before stopping the server, yet.", secondAddress, is(nullValue()));
+    }
+
+    @Test
+    public void testConnectionSuspendedHandling() throws Exception {
+        // initialize LeaderElection-related instances

Review comment:
       Somewhat obvious.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @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);
+    }
+
+    @After
+    public void after() throws Exception {
+        stopTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(1, Duration.ofSeconds(1));
+
+        ZooKeeperLeaderRetrievalService testInstance = 
ZooKeeperUtils.createLeaderRetrievalService(zooKeeperClient, config);
+        testInstance.start(queueLeaderElectionListener);
+
+        // do the testing
+        CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();
+        assertThat("No results are expected, yet, since no leader was 
elected.", firstAddress, is(nullValue()));
+
+        stopTestServer();
+
+        CompletableFuture<String> secondAddress = 
queueLeaderElectionListener.next();
+        assertThat("No result is expected since there was no leader elected 
before stopping the server, yet.", secondAddress, is(nullValue()));
+    }
+
+    @Test
+    public void testConnectionSuspendedHandling() throws Exception {
+        // initialize LeaderElection-related instances
+        String leaderAddress = "localhost";
+        LeaderElectionService leaderElectionService = 
ZooKeeperUtils.createLeaderElectionService(zooKeeperClient, config);
+        TestingContender contender = new TestingContender(leaderAddress, 
leaderElectionService);
+        leaderElectionService.start(contender);
+
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(2, Duration.ofSeconds(1));

Review comment:
       A timeout of 1 second can be too low for slow testing machines. I would 
suggest to not use a timeout here since we are expecting that the future will 
be delivered.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @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);
+    }
+
+    @After
+    public void after() throws Exception {
+        stopTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        // initialize LeaderRetrievalService
+        QueueLeaderElectionListener queueLeaderElectionListener = new 
QueueLeaderElectionListener(1, Duration.ofSeconds(1));
+
+        ZooKeeperLeaderRetrievalService testInstance = 
ZooKeeperUtils.createLeaderRetrievalService(zooKeeperClient, config);
+        testInstance.start(queueLeaderElectionListener);
+
+        // do the testing
+        CompletableFuture<String> firstAddress = 
queueLeaderElectionListener.next();

Review comment:
       This call adds a delay of 1s. Since we are calling it twice it adds 2s 
to the test duration. Since it is generally hard to test the absence of 
something I would suggest to decrease the timeout to 50 ms.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionConnectionHandlingTest.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import 
org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+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.Test;
+
+import java.io.IOException;
+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.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the error handling in case of a suspended connection to the 
ZooKeeper instance.
+ */
+public class ZooKeeperLeaderElectionConnectionHandlingTest extends TestLogger {
+
+    private TestingServer testingServer;
+
+    private Configuration config;
+
+    private CuratorFramework zooKeeperClient;
+
+    @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);
+    }
+
+    @After
+    public void after() throws Exception {
+        stopTestServer();
+
+        if (zooKeeperClient != null) {
+            zooKeeperClient.close();
+            zooKeeperClient = null;
+        }
+    }
+
+    @Test
+    public void testConnectionSuspendedHandlingDuringInitialization() throws 
Exception {
+        // initialize LeaderRetrievalService

Review comment:
       Somewhat obvious.




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

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


Reply via email to