alpreu commented on a change in pull request #18309:
URL: https://github.com/apache/flink/pull/18309#discussion_r783060124



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java
##########
@@ -195,6 +201,8 @@ public KafkaDynamicSource(
         this.startupTimestampMillis = startupTimestampMillis;
         this.upsertMode = upsertMode;
         this.tableIdentifier = tableIdentifier;
+        Preconditions.checkArgument(parallelism >= 1, "source.parallelism must 
be at least 1.");

Review comment:
       I think this should be `sourceParallelism`

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java
##########
@@ -195,6 +201,8 @@ public KafkaDynamicSource(
         this.startupTimestampMillis = startupTimestampMillis;
         this.upsertMode = upsertMode;
         this.tableIdentifier = tableIdentifier;
+        Preconditions.checkArgument(parallelism >= 1, "source.parallelism must 
be at least 1.");

Review comment:
       Are you making sure it cannot throw a NullPointerException here? The 
passed in Integer could be null, right?

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
##########
@@ -912,7 +948,39 @@ private static KafkaDynamicSource createExpectedScanSource(
                 FactoryMocks.IDENTIFIER.asSummaryString());
     }
 
-    private static KafkaDynamicSink createExpectedSink(
+    private static KafkaDynamicSource createExpectedScanSource(

Review comment:
       Same as above, rather reuse the current constructor

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
##########
@@ -99,12 +99,20 @@ protected CommonExecCalc(
                         
JavaScalaConversionUtil.toScala(Optional.ofNullable(this.condition)),
                         retainHeader,
                         getClass().getSimpleName());
+
+        // update calc operator parallelism to default parallelism
+        int parallelism = inputTransform.getParallelism();
+        int defaultParallelism = ExecutionConfig.PARALLELISM_DEFAULT;
+        if (parallelism != defaultParallelism) {
+            parallelism = defaultParallelism;
+        }
+

Review comment:
       Can you explain what is the reason we are adding this?

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java
##########
@@ -374,7 +380,41 @@ protected KafkaDynamicSource createKafkaTableSource(
                 specificStartupOffsets,
                 startupTimestampMillis,
                 false,
-                tableIdentifier);
+                tableIdentifier,
+                null);
+    }
+
+    protected KafkaDynamicSource createKafkaTableSource(

Review comment:
       I believe it would be better to not clone but modify the current 
constructor and make `sourceParallelism`nullable

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
##########
@@ -222,6 +223,41 @@ public void testTableSource() {
         assertKafkaSource(provider);
     }
 
+    @Test
+    public void testTableSourceWithParallelism() {

Review comment:
       Are you really asserting that the source parallelism matches the 
expected value?




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