libenchao commented on code in PR #24128:
URL: https://github.com/apache/flink/pull/24128#discussion_r1458590983
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java:
##########
@@ -72,6 +76,8 @@ public abstract class CommonExecTableSourceScan extends
ExecNodeBase<RowData>
@JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE)
private final DynamicTableSourceSpec tableSourceSpec;
+ protected boolean sourceParallelismConfigured;
Review Comment:
No need to be a class field
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformationWrapper.java:
##########
@@ -0,0 +1,70 @@
+/*
+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.dag.Transformation;
+import org.apache.flink.streaming.api.graph.TransformationTranslator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+
+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 downstream.
+ *
+ * It is used only when the parallelism of the source transformation differs
from the default parallelism,
+ * ensuring that the parallelism of downstream operations is not affected.
+ *
+ * Moreover, this transformation does not have a corresponding {@link
TransformationTranslator}, meaning
+ * it will not become a node in the StreamGraph.
+ *
+ * @param <T> The type of the elements in the input {@code Transformation}
+ * */
+@Internal
+public class SourceTransformationWrapper<T> extends PhysicalTransformation<T> {
+
+ private final Transformation<T> input;
+
+ public SourceTransformationWrapper(Transformation<T> input, String name,
int parallelism) {
Review Comment:
`name` and `parallelism` is not necessary to be a variable, it can be set to
a default value.
##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala:
##########
@@ -421,4 +424,81 @@ class TableSourceITCase extends StreamingTestBase {
val expected = Seq("1,Sarah,1", "2,Rob,1", "3,Mike,1")
assertThat(sink.getAppendResults.sorted).isEqualTo(expected.sorted)
}
+
+ private def innerTestSetParallelism(provider: String, parallelism: Int,
index: Int): Unit = {
+ val dataId = TestValuesTableFactory.registerData(data1)
+ val sourceTableName =
s"test_para_source_${provider.toLowerCase.trim}_$index"
+ val sinkTableName = s"test_para_sink_${provider.toLowerCase.trim}_$index"
+ tEnv.executeSql(s"""
+ |CREATE TABLE $sourceTableName (
+ | the_month INT,
+ | area STRING,
+ | product INT
+ |) WITH (
+ | 'connector' = 'values',
+ | 'data-id' = '$dataId',
+ | 'bounded' = 'true',
+ | 'runtime-source' = '$provider',
+ | 'scan.parallelism' = '$parallelism',
+ | 'enable-projection-push-down' = 'false'
+ |)
+ |""".stripMargin)
+ tEnv.executeSql(s"""
+ |CREATE TABLE $sinkTableName (
+ | the_month INT,
+ | area STRING,
+ | product INT
+ |) WITH (
+ | 'connector' = 'values',
+ | 'sink-insert-only' = 'true'
+ |)
+ |""".stripMargin)
+ tEnv.executeSql(s"INSERT INTO $sinkTableName SELECT * FROM
$sourceTableName").await()
+ }
+
+ @Test
+ def testParallelismWithSourceFunction(): Unit = {
Review Comment:
I'm not sure if these tests can really verify that the parallelism is
configured correctly.
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformationWrapper.java:
##########
@@ -0,0 +1,70 @@
+/*
+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.dag.Transformation;
+import org.apache.flink.streaming.api.graph.TransformationTranslator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+
+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 downstream.
+ *
+ * It is used only when the parallelism of the source transformation differs
from the default parallelism,
+ * ensuring that the parallelism of downstream operations is not affected.
+ *
+ * Moreover, this transformation does not have a corresponding {@link
TransformationTranslator}, meaning
+ * it will not become a node in the StreamGraph.
+ *
+ * @param <T> The type of the elements in the input {@code Transformation}
+ * */
+@Internal
+public class SourceTransformationWrapper<T> extends PhysicalTransformation<T> {
Review Comment:
It might no need to extends `PhysicalTransformation`.
`PhysicalTransformation` means it should have a corresponding physical
operator.
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java:
##########
@@ -105,54 +112,92 @@ protected Transformation<RowData> translateToPlanInternal(
planner.getFlinkContext(),
ShortcutUtils.unwrapTypeFactory(planner));
ScanTableSource.ScanRuntimeProvider provider =
tableSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+ final int sourceParallelism = deriveSourceParallelism(provider);
+ 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);
+ 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);
+ sourceTransform.setParallelism(sourceParallelism,
sourceParallelismConfigured);
+ 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);
+ sourceTransform.setParallelism(sourceParallelism,
sourceParallelismConfigured);
+ 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;
+ sourceTransform.setParallelism(sourceParallelism,
sourceParallelismConfigured);
Review Comment:
If there is no parallelism set in source, we do not need to set default for
`sourceTransform`
--
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]