libenchao commented on code in PR #24128:
URL: https://github.com/apache/flink/pull/24128#discussion_r1464205544


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformationWrapper.java:
##########
@@ -0,0 +1,72 @@
+/*
+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.streaming.api.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.graph.TransformationTranslator;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This Transformation is a phantom transformation which used to expose a 
default parallelism to

Review Comment:
   ```suggestion
    * This Transformation is a phantom transformation which is used to expose a 
default parallelism to
   ```



##########
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml:
##########
@@ -727,4 +727,60 @@ Calc(select=[ts, a, b], where=[>(a, 1)], 
changelogMode=[I,UB,UA,D])
 ]]>
     </Resource>
   </TestCase>
+
+  <TestCase name="testSetParallelismForSource">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM src WHERE the_month > 1]]>
+    </Resource>
+       <Resource name="ast">
+      <![CDATA[
+LogicalProject(the_month=[$0], area=[$1], product_id=[$2])
++- LogicalFilter(condition=[>($0, 1)])
++- LogicalTableScan(table=[[default_catalog, default_database, src]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[the_month, area, product_id], where=[(the_month > 1)])
++- TableSourceScan(table=[[default_catalog, default_database, src, 
filter=[]]], fields=[the_month, area, product_id])
+]]>
+       </Resource>
+       <Resource name="transformation">
+      <![CDATA[
+OneInputTransformation{id=3, name='Calc[4]', outputType=ROW<`the_month` INT, 
`area` STRING, `product_id` INT>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=-1}
++- SourceTransformationWrapper{id=2, name='ChangeToDefaultParallel', 
outputType=ROW<`the_month` INT, `area` STRING, `product_id` 
INT>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=-1}
+       +- LegacySourceTransformation{id=1, name='src[3]', 
outputType=ROW<`the_month` INT, `area` STRING, `product_id` 
INT>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=3}
+]]>
+       </Resource>
+  </TestCase>
+  <TestCase name="testSetParallelismForChangelogSource">
+       <Resource name="sql">
+      <![CDATA[SELECT * FROM src WHERE the_month > 1]]>
+       </Resource>
+       <Resource name="ast">
+      <![CDATA[
+LogicalProject(the_month=[$0], area=[$1], product_id=[$2])
++- LogicalFilter(condition=[>($0, 1)])
+   +- LogicalTableScan(table=[[default_catalog, default_database, src]])
+]]>
+       </Resource>
+       <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[the_month, area, product_id], where=[(the_month > 1)])
++- ChangelogNormalize(key=[product_id])
+   +- Exchange(distribution=[hash[product_id]])
+      +- TableSourceScan(table=[[default_catalog, default_database, src, 
filter=[]]], fields=[the_month, area, product_id])
+]]>
+       </Resource>
+       <Resource name="transformation">
+      <![CDATA[
+OneInputTransformation{id=6, name='Calc[8]', outputType=ROW<`the_month` INT, 
`area` STRING, `product_id` INT NOT NULL>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=-1}
+       +- OneInputTransformation{id=5, name='ChangelogNormalize[7]', 
outputType=ROW<`the_month` INT, `area` STRING, `product_id` INT NOT 
NULL>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=-1}
+               +- PartitionTransformation{id=4, name='Exchange[6]', 
outputType=ROW<`the_month` INT, `area` STRING, `product_id` INT NOT 
NULL>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=-1}
+                       +- PartitionTransformation{id=3, name='Partitioner[5]', 
outputType=ROW<`the_month` INT, `area` STRING, `product_id` INT NOT 
NULL>(org.apache.flink.table.data.RowData, 
org.apache.flink.table.runtime.typeutils.RowDataSerializer), parallelism=-1}

Review Comment:
   There are two `PartitionTransformation` in the transformations, did you run 
a real job, and verified if there are two hash shuffle, If yes, I think we 
might need to do some optimization for this case.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java:
##########
@@ -105,54 +117,131 @@ protected Transformation<RowData> 
translateToPlanInternal(
                         planner.getFlinkContext(), 
ShortcutUtils.unwrapTypeFactory(planner));
         ScanTableSource.ScanRuntimeProvider provider =
                 
tableSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+        final int sourceParallelism = deriveSourceParallelism(provider);
+        final boolean sourceParallelismConfigured = 
isParallelismConfigured(provider);
         if (provider instanceof SourceFunctionProvider) {
             final SourceFunctionProvider sourceFunctionProvider = 
(SourceFunctionProvider) provider;
             final SourceFunction<RowData> function = 
sourceFunctionProvider.createSourceFunction();
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     createSourceFunctionTransformation(
                             env,
                             function,
                             sourceFunctionProvider.isBounded(),
                             meta.getName(),
-                            outputTypeInfo);
-            return meta.fill(transformation);
+                            outputTypeInfo,
+                            sourceParallelism,
+                            sourceParallelismConfigured);
+            return meta.fill(sourceTransform);
         } else if (provider instanceof InputFormatProvider) {
             final InputFormat<RowData, ?> inputFormat =
                     ((InputFormatProvider) provider).createInputFormat();
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     createInputFormatTransformation(
                             env, inputFormat, outputTypeInfo, meta.getName());
-            return meta.fill(transformation);
+            meta.fill(sourceTransform);
         } else if (provider instanceof SourceProvider) {
             final Source<RowData, ?, ?> source = ((SourceProvider) 
provider).createSource();
             // TODO: Push down watermark strategy to source scan
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     env.fromSource(
                                     source,
                                     WatermarkStrategy.noWatermarks(),
                                     meta.getName(),
                                     outputTypeInfo)
                             .getTransformation();
-            return meta.fill(transformation);
+            meta.fill(sourceTransform);
         } else if (provider instanceof DataStreamScanProvider) {
-            Transformation<RowData> transformation =
+            sourceTransform =
                     ((DataStreamScanProvider) provider)
                             .produceDataStream(createProviderContext(config), 
env)
                             .getTransformation();
-            meta.fill(transformation);
-            transformation.setOutputType(outputTypeInfo);
-            return transformation;
+            meta.fill(sourceTransform);
+            sourceTransform.setOutputType(outputTypeInfo);
         } else if (provider instanceof TransformationScanProvider) {
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     ((TransformationScanProvider) provider)
                             
.createTransformation(createProviderContext(config));
-            meta.fill(transformation);
-            transformation.setOutputType(outputTypeInfo);
-            return transformation;
+            meta.fill(sourceTransform);
+            sourceTransform.setOutputType(outputTypeInfo);
         } else {
             throw new UnsupportedOperationException(
                     provider.getClass().getSimpleName() + " is unsupported 
now.");
         }
+
+        if (sourceParallelismConfigured) {
+            return applySourceTransformationWrapper(
+                    sourceTransform,
+                    planner.getFlinkContext().getClassLoader(),
+                    outputTypeInfo,
+                    config,
+                    sourceParallelism);
+        } else {
+            return sourceTransform;
+        }
+    }
+
+    private boolean 
isParallelismConfigured(ScanTableSource.ScanRuntimeProvider runtimeProvider) {
+        return runtimeProvider instanceof ParallelismProvider
+                && ((ParallelismProvider) 
runtimeProvider).getParallelism().isPresent();
+    }
+
+    private int deriveSourceParallelism(ScanTableSource.ScanRuntimeProvider 
runtimeProvider) {
+        if (isParallelismConfigured(runtimeProvider)) {
+            int sourceParallelism = ((ParallelismProvider) 
runtimeProvider).getParallelism().get();
+            if (sourceParallelism <= 0) {
+                throw new TableException(
+                        String.format(
+                                "Invalid configured parallelism %s for table 
'%s'.",
+                                sourceParallelism,
+                                tableSourceSpec
+                                        .getContextResolvedTable()
+                                        .getIdentifier()
+                                        .asSummaryString()));
+            }
+            return sourceParallelism;
+        } else {
+            return ExecutionConfig.PARALLELISM_DEFAULT;
+        }
+    }
+
+    protected RowType getPhysicalRowType(ResolvedSchema schema) {
+        return (RowType) schema.toPhysicalRowDataType().getLogicalType();
+    }
+
+    protected int[] getPrimaryKeyIndices(RowType sourceRowType, ResolvedSchema 
schema) {
+        return schema.getPrimaryKey()
+                .map(k -> 
k.getColumns().stream().mapToInt(sourceRowType::getFieldIndex).toArray())
+                .orElse(new int[0]);
+    }
+
+    private Transformation<RowData> applySourceTransformationWrapper(
+            Transformation<RowData> sourceTransform,
+            ClassLoader classLoader,
+            InternalTypeInfo<RowData> outputTypeInfo,
+            ExecNodeConfig config,
+            int sourceParallelism) {
+        sourceTransform.setParallelism(sourceParallelism, true);
+        Transformation<RowData> sourceTransformationWrapper =
+                new SourceTransformationWrapper<>(sourceTransform);
+
+        final ResolvedSchema schema = 
tableSourceSpec.getContextResolvedTable().getResolvedSchema();
+        final RowType physicalRowType = getPhysicalRowType(schema);
+        final int[] primaryKeys = getPrimaryKeyIndices(physicalRowType, 
schema);
+        final boolean hasPk = primaryKeys.length > 0;
+
+        if (hasPk) {

Review Comment:
   Besides, in the FLIP discussion, we agreed that we don't need to apply hash 
shuffle when source only produces insert-only data.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java:
##########
@@ -105,54 +117,131 @@ protected Transformation<RowData> 
translateToPlanInternal(
                         planner.getFlinkContext(), 
ShortcutUtils.unwrapTypeFactory(planner));
         ScanTableSource.ScanRuntimeProvider provider =
                 
tableSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+        final int sourceParallelism = deriveSourceParallelism(provider);
+        final boolean sourceParallelismConfigured = 
isParallelismConfigured(provider);
         if (provider instanceof SourceFunctionProvider) {
             final SourceFunctionProvider sourceFunctionProvider = 
(SourceFunctionProvider) provider;
             final SourceFunction<RowData> function = 
sourceFunctionProvider.createSourceFunction();
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     createSourceFunctionTransformation(
                             env,
                             function,
                             sourceFunctionProvider.isBounded(),
                             meta.getName(),
-                            outputTypeInfo);
-            return meta.fill(transformation);
+                            outputTypeInfo,
+                            sourceParallelism,
+                            sourceParallelismConfigured);
+            return meta.fill(sourceTransform);
         } else if (provider instanceof InputFormatProvider) {
             final InputFormat<RowData, ?> inputFormat =
                     ((InputFormatProvider) provider).createInputFormat();
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     createInputFormatTransformation(
                             env, inputFormat, outputTypeInfo, meta.getName());
-            return meta.fill(transformation);
+            meta.fill(sourceTransform);
         } else if (provider instanceof SourceProvider) {
             final Source<RowData, ?, ?> source = ((SourceProvider) 
provider).createSource();
             // TODO: Push down watermark strategy to source scan
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     env.fromSource(
                                     source,
                                     WatermarkStrategy.noWatermarks(),
                                     meta.getName(),
                                     outputTypeInfo)
                             .getTransformation();
-            return meta.fill(transformation);
+            meta.fill(sourceTransform);
         } else if (provider instanceof DataStreamScanProvider) {
-            Transformation<RowData> transformation =
+            sourceTransform =
                     ((DataStreamScanProvider) provider)
                             .produceDataStream(createProviderContext(config), 
env)
                             .getTransformation();
-            meta.fill(transformation);
-            transformation.setOutputType(outputTypeInfo);
-            return transformation;
+            meta.fill(sourceTransform);
+            sourceTransform.setOutputType(outputTypeInfo);
         } else if (provider instanceof TransformationScanProvider) {
-            final Transformation<RowData> transformation =
+            sourceTransform =
                     ((TransformationScanProvider) provider)
                             
.createTransformation(createProviderContext(config));
-            meta.fill(transformation);
-            transformation.setOutputType(outputTypeInfo);
-            return transformation;
+            meta.fill(sourceTransform);
+            sourceTransform.setOutputType(outputTypeInfo);
         } else {
             throw new UnsupportedOperationException(
                     provider.getClass().getSimpleName() + " is unsupported 
now.");
         }
+
+        if (sourceParallelismConfigured) {
+            return applySourceTransformationWrapper(
+                    sourceTransform,
+                    planner.getFlinkContext().getClassLoader(),
+                    outputTypeInfo,
+                    config,
+                    sourceParallelism);
+        } else {
+            return sourceTransform;
+        }
+    }
+
+    private boolean 
isParallelismConfigured(ScanTableSource.ScanRuntimeProvider runtimeProvider) {
+        return runtimeProvider instanceof ParallelismProvider
+                && ((ParallelismProvider) 
runtimeProvider).getParallelism().isPresent();
+    }
+
+    private int deriveSourceParallelism(ScanTableSource.ScanRuntimeProvider 
runtimeProvider) {
+        if (isParallelismConfigured(runtimeProvider)) {
+            int sourceParallelism = ((ParallelismProvider) 
runtimeProvider).getParallelism().get();
+            if (sourceParallelism <= 0) {
+                throw new TableException(
+                        String.format(
+                                "Invalid configured parallelism %s for table 
'%s'.",
+                                sourceParallelism,
+                                tableSourceSpec
+                                        .getContextResolvedTable()
+                                        .getIdentifier()
+                                        .asSummaryString()));
+            }
+            return sourceParallelism;
+        } else {
+            return ExecutionConfig.PARALLELISM_DEFAULT;
+        }
+    }
+
+    protected RowType getPhysicalRowType(ResolvedSchema schema) {
+        return (RowType) schema.toPhysicalRowDataType().getLogicalType();
+    }
+
+    protected int[] getPrimaryKeyIndices(RowType sourceRowType, ResolvedSchema 
schema) {
+        return schema.getPrimaryKey()
+                .map(k -> 
k.getColumns().stream().mapToInt(sourceRowType::getFieldIndex).toArray())
+                .orElse(new int[0]);
+    }
+
+    private Transformation<RowData> applySourceTransformationWrapper(
+            Transformation<RowData> sourceTransform,
+            ClassLoader classLoader,
+            InternalTypeInfo<RowData> outputTypeInfo,
+            ExecNodeConfig config,
+            int sourceParallelism) {
+        sourceTransform.setParallelism(sourceParallelism, true);
+        Transformation<RowData> sourceTransformationWrapper =
+                new SourceTransformationWrapper<>(sourceTransform);
+
+        final ResolvedSchema schema = 
tableSourceSpec.getContextResolvedTable().getResolvedSchema();
+        final RowType physicalRowType = getPhysicalRowType(schema);
+        final int[] primaryKeys = getPrimaryKeyIndices(physicalRowType, 
schema);
+        final boolean hasPk = primaryKeys.length > 0;
+
+        if (hasPk) {

Review Comment:
   If the parallelism is equal to the job default parallelism, I guess there is 
no need to do a hash shuffle, so I'm wondering if it's possible to obtain the 
job default parallelism here to do this optimization?



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

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

Reply via email to