You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/27 11:31:17 UTC

[GitHub] [flink] XComp commented on a diff in pull request #20378: [FLINK-28523][tests] Increase Zookeeper session timeouts

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


##########
flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java:
##########
@@ -162,8 +163,7 @@ private Configuration getConfiguration() throws Exception {
         // Testing HA Scenario / ZKCompletedCheckpointStore with incremental checkpoints
         StateBackendEnum stateBackendEnum = getStateBackend();
         if (ROCKSDB_INCREMENTAL_ZK.equals(stateBackendEnum)) {
-            zkServer = new TestingServer();
-            zkServer.start();
+            zkServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   Should we switch to `zkServer.close()` here as well to be consistent with the other test cases? I guess, `close` is appropriate here because we removing the reference here as well. On close also deletes the data directory...



##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java:
##########
@@ -37,6 +41,33 @@ public class ZooKeeperTestUtils {
 
     private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperTestUtils.class);
 
+    /**
+     * Creates a new {@link TestingServer}, setting additional configuration properties for
+     * stability purposes.
+     */
+    public static TestingServer createZookeeperTestingServer() throws Exception {

Review Comment:
   ```suggestion
       public static TestingServer createAndStartZookeeperTestingServer() throws Exception {
   ```
   Just as a proposal to make it more obvious what the method does...



##########
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java:
##########
@@ -37,6 +41,33 @@ public class ZooKeeperTestUtils {
 
     private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperTestUtils.class);
 
+    /**
+     * Creates a new {@link TestingServer}, setting additional configuration properties for
+     * stability purposes.
+     */
+    public static TestingServer createZookeeperTestingServer() throws Exception {
+        return new TestingServer(getZookeeperInstanceSpecWithIncreasedSessionTimeout(), true);
+    }
+
+    private static InstanceSpec getZookeeperInstanceSpecWithIncreasedSessionTimeout() {
+        final InstanceSpec instanceSpec = InstanceSpec.newInstanceSpec();
+
+        final Map<String, Object> properties = new HashMap<>();
+        properties.put("maxSessionTimeout", 60_000);
+
+        return new InstanceSpec(
+                instanceSpec.getDataDirectory(),
+                instanceSpec.getPort(),
+                instanceSpec.getElectionPort(),
+                instanceSpec.getQuorumPort(),
+                instanceSpec.deleteDataDirectoryOnClose(),
+                instanceSpec.getServerId(),
+                instanceSpec.getTickTime(),
+                instanceSpec.getMaxClientCnxns(),
+                properties,
+                instanceSpec.getHostname());

Review Comment:
   ```suggestion
           final boolean deleteDataDirectoryOnClose = true;
           return new InstanceSpec(
                   instanceSpec.getDataDirectory(),
                   instanceSpec.getPort(),
                   instanceSpec.getElectionPort(),
                   instanceSpec.getQuorumPort(),
                   deleteDataDirectoryOnClose,
                   instanceSpec.getServerId(),
                   instanceSpec.getTickTime(),
                   instanceSpec.getMaxClientCnxns(),
                   properties,
                   instanceSpec.getHostname());
   ```
   I would make this parameter more explicit (it's actually true in the default `InstanceSpec` we use) to improve the readability of the code since this is a more important property (in contrast to the other ones) here.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java:
##########
@@ -185,7 +186,7 @@ private static final class ZooKeeperServiceClass implements ServiceClass {
         @Override
         public void setup(FatalErrorHandler fatalErrorHandler) throws Exception {
             try {
-                testingServer = new TestingServer();
+                testingServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   `close` instead of `stop` in `teardown` method.



##########
flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java:
##########
@@ -110,7 +111,7 @@ class YARNHighAvailabilityITCase extends YarnTestBase {
 
     @BeforeAll
     static void setup(@TempDir File tempDir) throws Exception {
-        zkServer = new TestingServer();
+        zkServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   We should use `zkServer.close()` instead of `stop()` in the `teardown()` method below as well.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalTest.java:
##########
@@ -75,7 +76,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger {
 
     @Before
     public void before() throws Exception {
-        testingServer = new TestingServer();
+        testingServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   We could use `close` in the `after()` method instead of `stop()`.



##########
flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java:
##########
@@ -62,7 +63,7 @@ protected StateBackend createStateBackend() throws Exception {
 
     @BeforeClass
     public static void setup() throws Exception {
-        zkServer = new TestingServer();
+        zkServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   nit: `zkServer.stop()` could be removed in the `tearDown()` method



##########
flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java:
##########
@@ -113,7 +114,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
     @Before
     public void before() {
         try {
-            testingServer = new TestingServer();
+            testingServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   `close` instead of `stop` in the `after` method



##########
flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java:
##########
@@ -62,7 +63,7 @@ protected StateBackend createStateBackend() throws Exception {
 
     @BeforeClass
     public static void setup() throws Exception {
-        zkServer = new TestingServer();
+        zkServer = ZooKeeperTestUtils.createZookeeperTestingServer();

Review Comment:
   `zkServer.stop()` could be removed in the `tearDown()` method.



-- 
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-unsubscribe@flink.apache.org

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