wenlong88 commented on a change in pull request #14637:
URL: https://github.com/apache/flink/pull/14637#discussion_r559890485



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.planner.plan.nodes.exec.common;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.SinkCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.sinks.DataStreamTableSink;
+import org.apache.flink.table.planner.sinks.TableSinkUtils;
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
+import org.apache.flink.table.sinks.AppendStreamTableSink;
+import org.apache.flink.table.sinks.RetractStreamTableSink;
+import org.apache.flink.table.sinks.StreamTableSink;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sinks.UpsertStreamTableSink;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+
+/**
+ * Base {@link ExecNode} to to write data into an external sink defined by a 
{@link TableSink}.
+ *
+ * @param <T> The return type of the {@link TableSink}.
+ */
+public abstract class CommonExecLegacySink<T> extends ExecNodeBase<Object> {
+    protected final TableSink<T> tableSink;
+    protected final @Nullable String[] upsertKeys;
+    protected final boolean needRetraction;
+
+    public CommonExecLegacySink(
+            TableSink<T> tableSink,
+            @Nullable String[] upsertKeys,
+            boolean needRetraction,
+            ExecEdge inputEdge,
+            LogicalType outputType,
+            String description) {
+        super(Collections.singletonList(inputEdge), outputType, description);
+        this.tableSink = tableSink;
+        this.upsertKeys = upsertKeys;
+        this.needRetraction = needRetraction;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<Object> translateToPlanInternal(PlannerBase 
planner) {
+        if (tableSink instanceof StreamTableSink) {
+            final Transformation<T> transform;
+            if (tableSink instanceof RetractStreamTableSink) {
+                transform = translateToTransformation(planner, true);
+            } else if (tableSink instanceof UpsertStreamTableSink) {
+                UpsertStreamTableSink<T> upsertSink = 
(UpsertStreamTableSink<T>) tableSink;
+                final boolean isAppendOnlyTable = !needRetraction;
+                upsertSink.setIsAppendOnly(isAppendOnlyTable);
+                if (upsertKeys != null) {
+                    upsertSink.setKeyFields(upsertKeys);
+                } else {
+                    if (isAppendOnlyTable) {
+                        upsertSink.setKeyFields(null);
+                    } else {
+                        throw new TableException(
+                                "UpsertStreamTableSink requires that Table has 
a full primary keys if it is updated.");
+                    }
+                }
+
+                transform = translateToTransformation(planner, true);
+            } else if (tableSink instanceof AppendStreamTableSink) {
+                // verify table is an insert-only (append-only) table
+                if (needRetraction) {
+                    throw new TableException(
+                            "AppendStreamTableSink requires that Table has 
only insert changes.");
+                }
+                transform = translateToTransformation(planner, false);
+            } else {
+                throw new TableException(
+                        "Stream Tables can only be emitted by 
AppendStreamTableSink, "
+                                + "RetractStreamTableSink, or 
UpsertStreamTableSink.");
+            }
+
+            final DataStream<T> dataStream = new 
DataStream<T>(planner.getExecEnv(), transform);
+            final DataStreamSink<?> dsSink =
+                    ((StreamTableSink<T>) 
tableSink).consumeDataStream(dataStream);
+            if (dsSink == null) {
+                throw new TableException(
+                        String.format(
+                                "The 
StreamTableSink#consumeDataStream(DataStream) must be implemented "
+                                        + "and return the sink transformation 
DataStreamSink. "
+                                        + "However, %s doesn't implement this 
method.",
+                                tableSink.getClass().getCanonicalName()));
+            }
+            return dsSink.getTransformation();
+        } else if (tableSink instanceof DataStreamTableSink) {
+            // In case of table to DataStream through
+            // StreamTableEnvironment#toAppendStream/toRetractStream,
+            // we insert a DataStreamTableSink that wraps the given DataStream 
as a LogicalSink. It
+            // is no real table sink, so we just need translate its input to 
Transformation.
+            return (Transformation<Object>)
+                    translateToTransformation(
+                            planner, ((DataStreamTableSink<T>) 
tableSink).withChangeFlag());
+        } else {
+            throw new TableException(
+                    String.format(
+                            "Only Support StreamTableSink! However %s is not a 
StreamTableSink.",
+                            tableSink.getClass().getCanonicalName()));
+        }
+    }
+
+    /**
+     * Translates {@link TableSink} into a {@link Transformation}.
+     *
+     * @param withChangeFlag Set to true to emit records with change flags.
+     * @return The {@link Transformation} that corresponds to the translated 
{@link TableSink}.
+     */
+    protected abstract Transformation<T> translateToTransformation(

Review comment:
       how about we just make the abstract interface which just returns the 
converted inputRowType, like: createActualSinkInputType. the name and logic is 
a bit of confusing here.

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.planner.plan.nodes.exec.common;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.CodeGenUtils;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.SinkCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.sinks.DataStreamTableSink;
+import org.apache.flink.table.planner.sinks.TableSinkUtils;
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
+import org.apache.flink.table.sinks.AppendStreamTableSink;
+import org.apache.flink.table.sinks.RetractStreamTableSink;
+import org.apache.flink.table.sinks.StreamTableSink;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sinks.UpsertStreamTableSink;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+
+/**
+ * Base {@link ExecNode} to to write data into an external sink defined by a 
{@link TableSink}.
+ *
+ * @param <T> The return type of the {@link TableSink}.
+ */
+public abstract class CommonExecLegacySink<T> extends ExecNodeBase<Object> {
+    protected final TableSink<T> tableSink;
+    protected final @Nullable String[] upsertKeys;
+    protected final boolean needRetraction;
+
+    public CommonExecLegacySink(
+            TableSink<T> tableSink,
+            @Nullable String[] upsertKeys,
+            boolean needRetraction,
+            ExecEdge inputEdge,
+            LogicalType outputType,
+            String description) {
+        super(Collections.singletonList(inputEdge), outputType, description);
+        this.tableSink = tableSink;
+        this.upsertKeys = upsertKeys;
+        this.needRetraction = needRetraction;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<Object> translateToPlanInternal(PlannerBase 
planner) {
+        if (tableSink instanceof StreamTableSink) {
+            final Transformation<T> transform;
+            if (tableSink instanceof RetractStreamTableSink) {
+                transform = translateToTransformation(planner, true);
+            } else if (tableSink instanceof UpsertStreamTableSink) {
+                UpsertStreamTableSink<T> upsertSink = 
(UpsertStreamTableSink<T>) tableSink;
+                final boolean isAppendOnlyTable = !needRetraction;
+                upsertSink.setIsAppendOnly(isAppendOnlyTable);
+                if (upsertKeys != null) {
+                    upsertSink.setKeyFields(upsertKeys);
+                } else {
+                    if (isAppendOnlyTable) {
+                        upsertSink.setKeyFields(null);
+                    } else {
+                        throw new TableException(
+                                "UpsertStreamTableSink requires that Table has 
a full primary keys if it is updated.");
+                    }
+                }
+
+                transform = translateToTransformation(planner, true);
+            } else if (tableSink instanceof AppendStreamTableSink) {
+                // verify table is an insert-only (append-only) table
+                if (needRetraction) {
+                    throw new TableException(
+                            "AppendStreamTableSink requires that Table has 
only insert changes.");
+                }
+                transform = translateToTransformation(planner, false);
+            } else {
+                throw new TableException(
+                        "Stream Tables can only be emitted by 
AppendStreamTableSink, "
+                                + "RetractStreamTableSink, or 
UpsertStreamTableSink.");
+            }
+
+            final DataStream<T> dataStream = new 
DataStream<T>(planner.getExecEnv(), transform);
+            final DataStreamSink<?> dsSink =
+                    ((StreamTableSink<T>) 
tableSink).consumeDataStream(dataStream);
+            if (dsSink == null) {
+                throw new TableException(
+                        String.format(
+                                "The 
StreamTableSink#consumeDataStream(DataStream) must be implemented "
+                                        + "and return the sink transformation 
DataStreamSink. "
+                                        + "However, %s doesn't implement this 
method.",
+                                tableSink.getClass().getCanonicalName()));
+            }
+            return dsSink.getTransformation();
+        } else if (tableSink instanceof DataStreamTableSink) {
+            // In case of table to DataStream through
+            // StreamTableEnvironment#toAppendStream/toRetractStream,
+            // we insert a DataStreamTableSink that wraps the given DataStream 
as a LogicalSink. It
+            // is no real table sink, so we just need translate its input to 
Transformation.
+            return (Transformation<Object>)
+                    translateToTransformation(
+                            planner, ((DataStreamTableSink<T>) 
tableSink).withChangeFlag());
+        } else {
+            throw new TableException(
+                    String.format(
+                            "Only Support StreamTableSink! However %s is not a 
StreamTableSink.",
+                            tableSink.getClass().getCanonicalName()));
+        }
+    }
+
+    /**
+     * Translates {@link TableSink} into a {@link Transformation}.
+     *
+     * @param withChangeFlag Set to true to emit records with change flags.
+     * @return The {@link Transformation} that corresponds to the translated 
{@link TableSink}.
+     */
+    protected abstract Transformation<T> translateToTransformation(
+            PlannerBase planner, boolean withChangeFlag);
+
+    @SuppressWarnings("unchecked")
+    protected Transformation<T> createSinkTransformation(

Review comment:
       rename to: createSinkInputTransformation?




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

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


Reply via email to