twalthr commented on a change in pull request #18667:
URL: https://github.com/apache/flink/pull/18667#discussion_r803659709



##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/connector/ProviderContext.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.table.connector;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+
+import java.util.Optional;
+
+/**
+ * Context provided by some {@link DynamicTableSink.SinkRuntimeProvider} and 
{@link

Review comment:
       nit: static import for both and mention source before sink.

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationScanProvider.java
##########
@@ -41,11 +42,13 @@
 public interface TransformationScanProvider extends 
ScanTableSource.ScanRuntimeProvider {
 
     /** Helper method for creating a static provider. The boundedness is 
derived automatically. */
-    static TransformationScanProvider of(Transformation<RowData> 
transformation) {
+    static TransformationScanProvider of(

Review comment:
       I would rather suggest to remove the `of` method. Assigning a name only 
to the leaf transformation is never enough and can lead to side effects. This 
class is internal anyway.

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationSinkProvider.java
##########
@@ -37,15 +40,24 @@
 @Internal
 public interface TransformationSinkProvider extends 
DynamicTableSink.SinkRuntimeProvider {
 
-    /** Creates a transformation for transforming the input provided in the 
context. */
+    /**
+     * Creates a transformation for transforming the input provided in the 
context.
+     *
+     * <p>This method MUST set an uid for each node of the transformation 
sink, when the job is
+     * unbounded, which can be generated with {@link 
Context#generateUid(String)}.
+     */
     Transformation<?> createTransformation(Context context);
 
     /** Context for {@link #createTransformation(Context)}. */
-    interface Context {
+    interface Context extends ProviderContext {

Review comment:
       Rename to `TransformationProviderContext`?

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java
##########
@@ -1644,10 +1652,14 @@ public SinkRuntimeProvider 
getSinkRuntimeProvider(Context context) {
                         return new DataStreamSinkProvider() {
                             @Override
                             public DataStreamSink<?> consumeDataStream(
+                                    ProviderContext providerContext,
                                     DataStream<RowData> dataStream) {
-                                return dataStream.addSink(
-                                        new AppendingSinkFunction(
-                                                tableName, converter, 
rowtimeIndex));
+                                DataStreamSink<RowData> sink =
+                                        dataStream.addSink(
+                                                new AppendingSinkFunction(
+                                                        tableName, converter, 
rowtimeIndex));
+                                
providerContext.generateUid("values").ifPresent(sink::uid);

Review comment:
       `sink-function`

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java
##########
@@ -825,8 +827,14 @@ public ScanRuntimeProvider 
getScanRuntimeProvider(ScanContext runtimeProviderCon
                         return new DataStreamScanProvider() {
                             @Override
                             public DataStream<RowData> produceDataStream(
+                                    ProviderContext providerContext,
                                     StreamExecutionEnvironment execEnv) {
-                                return execEnv.addSource(function);
+                                DataStreamSource<RowData> sourceStream =
+                                        execEnv.addSource(function);
+                                providerContext
+                                        .generateUid("from-elements")

Review comment:
       `source-function`

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/CollectDynamicSink.java
##########
@@ -93,39 +98,40 @@ public ChangelogMode getChangelogMode(ChangelogMode 
requestedMode) {
 
     @Override
     public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
-        return (DataStreamSinkProvider)
-                inputStream -> {

Review comment:
       did we break the API for lambdas now?

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/TransformationSinkProvider.java
##########
@@ -37,15 +40,24 @@
 @Internal
 public interface TransformationSinkProvider extends 
DynamicTableSink.SinkRuntimeProvider {
 
-    /** Creates a transformation for transforming the input provided in the 
context. */
+    /**
+     * Creates a transformation for transforming the input provided in the 
context.
+     *
+     * <p>This method MUST set an uid for each node of the transformation 
sink, when the job is
+     * unbounded, which can be generated with {@link 
Context#generateUid(String)}.
+     */
     Transformation<?> createTransformation(Context context);
 
     /** Context for {@link #createTransformation(Context)}. */
-    interface Context {
+    interface Context extends ProviderContext {
 
         /** Helper method for creating the default implementation of {@link 
Context}. */
-        static Context of(Transformation<RowData> inputTransformation, int 
rowtimeIndex) {
+        static Context of(

Review comment:
       remove

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSource.java
##########
@@ -117,7 +119,8 @@ public ScanRuntimeProvider 
getScanRuntimeProvider(ScanContext runtimeProviderCon
                         null, // will be filled by the framework
                         externalTransformation.getParallelism());
 
-        return TransformationScanProvider.of(conversionTransformation);
+        return TransformationScanProvider.of(
+                conversionTransformation, EXTERNAL_DATASTREAM_TRANSFORMATION);

Review comment:
       assign the uid via `createOneInputTransformation` instead of using the 
`of`




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