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/04/20 07:35:57 UTC

[GitHub] [flink] zentol opened a new pull request, #19523: [FLINK-27287][tests] Use random ports

zentol opened a new pull request, #19523:
URL: https://github.com/apache/flink/pull/19523

   - migrates some tests to MiniClusterResources (where it was easy to do so)
   - introduces MiniClusterConfiguration#withRandomPorts
     - this gives us a single point where we can configure ports in the future


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


[GitHub] [flink] dmvk commented on a diff in pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
dmvk commented on code in PR #19523:
URL: https://github.com/apache/flink/pull/19523#discussion_r854022591


##########
flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java:
##########
@@ -214,6 +220,15 @@ public MiniClusterConfiguration build() {
                     RestOptions.ADDRESS,
                     modifiedConfiguration.getString(RestOptions.ADDRESS, "localhost"));
 
+            if (useRandomPorts) {
+                if (!configuration.contains(JobManagerOptions.PORT)) {

Review Comment:
   This feels weird. Why not force the options when `useRandomPorts` is enabled?



##########
flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/JobManagerWatermarkTrackerTest.java:
##########
@@ -18,57 +18,30 @@
 package org.apache.flink.streaming.connectors.kinesis.util;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.RestOptions;
-import org.apache.flink.runtime.minicluster.MiniCluster;
-import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 
 /** Test for {@link JobManagerWatermarkTracker}. */
 public class JobManagerWatermarkTrackerTest {
 
-    private static MiniCluster flink;
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        final Configuration config = new Configuration();
-        config.setInteger(RestOptions.PORT, 0);
-
-        final MiniClusterConfiguration miniClusterConfiguration =
-                new MiniClusterConfiguration.Builder()
-                        .setConfiguration(config)
-                        .setNumTaskManagers(1)
-                        .setNumSlotsPerTaskManager(1)
-                        .build();
-
-        flink = new MiniCluster(miniClusterConfiguration);
-
-        flink.start();
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        if (flink != null) {
-            flink.close();
-        }
-    }
+    @ClassRule
+    public static final MiniClusterResource FLINK =
+            new MiniClusterResource(
+                    new MiniClusterResourceConfiguration.Builder()

Review Comment:
   `withRandomPorts`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java:
##########
@@ -206,6 +207,11 @@ public Builder setHaServices(MiniCluster.HaServices haServices) {
             return this;
         }
 
+        public Builder withRandomPorts() {

Review Comment:
   Would it make sense to use random ports by default and have a builder method for the static port allocation? That's IMO what we want in most scenarios.
   
   eg.
   ```
   withStaticPorts(...)
   ```



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


[GitHub] [flink] zentol commented on a diff in pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19523:
URL: https://github.com/apache/flink/pull/19523#discussion_r854026236


##########
flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/JobManagerWatermarkTrackerTest.java:
##########
@@ -18,57 +18,30 @@
 package org.apache.flink.streaming.connectors.kinesis.util;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.RestOptions;
-import org.apache.flink.runtime.minicluster.MiniCluster;
-import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 
 /** Test for {@link JobManagerWatermarkTracker}. */
 public class JobManagerWatermarkTrackerTest {
 
-    private static MiniCluster flink;
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        final Configuration config = new Configuration();
-        config.setInteger(RestOptions.PORT, 0);
-
-        final MiniClusterConfiguration miniClusterConfiguration =
-                new MiniClusterConfiguration.Builder()
-                        .setConfiguration(config)
-                        .setNumTaskManagers(1)
-                        .setNumSlotsPerTaskManager(1)
-                        .build();
-
-        flink = new MiniCluster(miniClusterConfiguration);
-
-        flink.start();
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        if (flink != null) {
-            flink.close();
-        }
-    }
+    @ClassRule
+    public static final MiniClusterResource FLINK =
+            new MiniClusterResource(
+                    new MiniClusterResourceConfiguration.Builder()

Review Comment:
   MiniClusterResource uses random ports by default.



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


[GitHub] [flink] zentol commented on a diff in pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19523:
URL: https://github.com/apache/flink/pull/19523#discussion_r854032049


##########
flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java:
##########
@@ -214,6 +220,15 @@ public MiniClusterConfiguration build() {
                     RestOptions.ADDRESS,
                     modifiedConfiguration.getString(RestOptions.ADDRESS, "localhost"));
 
+            if (useRandomPorts) {
+                if (!configuration.contains(JobManagerOptions.PORT)) {

Review Comment:
   Because we may extend the set of ports in the future but want to allow single ports to be configured.
   
   I think it would be weirder if an explicit port configuration by the user is simply ignored.



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


[GitHub] [flink] flinkbot commented on pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #19523:
URL: https://github.com/apache/flink/pull/19523#issuecomment-1103572810

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c2ee118c9f6a01deb8994ecc82fbb862a679f62d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c2ee118c9f6a01deb8994ecc82fbb862a679f62d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c2ee118c9f6a01deb8994ecc82fbb862a679f62d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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


[GitHub] [flink] zentol merged pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
zentol merged PR #19523:
URL: https://github.com/apache/flink/pull/19523


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


[GitHub] [flink] zentol commented on pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #19523:
URL: https://github.com/apache/flink/pull/19523#issuecomment-1103832309

   > My biggest concern is whether this doesn't add unnecessary complexity for the test authors, that could be mitigated by using random ports by default.
   
   That's a fair point, but the long-term goal should be to move as many tests as possible to the MiniClusterResource. Then the MiniCluster can continue to be a "production-like" single-JVM deployment, while us still having the benefit of random ports being used at all times.


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


[GitHub] [flink] zentol commented on a diff in pull request #19523: [FLINK-27287][tests] Use random ports

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19523:
URL: https://github.com/apache/flink/pull/19523#discussion_r854032674


##########
flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java:
##########
@@ -206,6 +207,11 @@ public Builder setHaServices(MiniCluster.HaServices haServices) {
             return this;
         }
 
+        public Builder withRandomPorts() {

Review Comment:
   The MiniCluster is in a weird spot where it is not a pure testing utility.
   If you consider it to just be a single JVM Flink deployment, then it makes perfect sense for it to use the usual ports.
   
   I'm wary of changing the default MiniCluster behavior because it may break plenty of stuff for users.



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