Airblader commented on a change in pull request #15588:
URL: https://github.com/apache/flink/pull/15588#discussion_r612239343



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/connectors/CollectDynamicSink.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.connectors;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import org.apache.flink.streaming.api.operators.collect.CollectResultIterator;
+import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator;
+import 
org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory;
+import org.apache.flink.streaming.api.operators.collect.CollectStreamSink;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.internal.CollectResultProvider;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.ExternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+/** Table sink for {@link TableResult#collect()}. */
+@Internal
+final class CollectDynamicSink implements DynamicTableSink {
+
+    private final ObjectIdentifier tableIdentifier;
+
+    private final DataType consumedDataType;
+
+    // mutable attributes
+
+    private CollectResultIterator<RowData> iterator;
+
+    CollectDynamicSink(ObjectIdentifier tableIdentifier, DataType 
consumedDataType) {
+        this.tableIdentifier = tableIdentifier;
+        this.consumedDataType = consumedDataType;
+    }
+
+    public CollectResultProvider getSelectResultProvider() {
+        return new CollectResultProvider() {
+            @Override
+            public void setJobClient(JobClient jobClient) {
+                iterator.setJobClient(jobClient);
+            }
+
+            @Override
+            @SuppressWarnings({"unchecked", "rawtypes"})
+            public CloseableIterator<Row> getResultIterator() {
+                // Row after deserialization
+                return (CloseableIterator) iterator;
+            }
+        };
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        return requestedMode;
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+        return (DataStreamSinkProvider)
+                inputStream -> {
+                    final CheckpointConfig checkpointConfig =
+                            
inputStream.getExecutionEnvironment().getCheckpointConfig();
+                    final ExecutionConfig config = 
inputStream.getExecutionConfig();
+
+                    final TypeSerializer<RowData> externalSerializer =
+                            ExternalTypeInfo.<RowData>of(consumedDataType, 
true)
+                                    .createSerializer(config);
+                    final String accumulatorName = 
tableIdentifier.getObjectName();
+
+                    final CollectSinkOperatorFactory<RowData> factory =
+                            new 
CollectSinkOperatorFactory<>(externalSerializer, accumulatorName);
+                    final CollectSinkOperator<RowData> operator =
+                            (CollectSinkOperator<RowData>) 
factory.getOperator();
+
+                    this.iterator =
+                            new CollectResultIterator<>(
+                                    operator.getOperatorIdFuture(),
+                                    externalSerializer,
+                                    accumulatorName,
+                                    checkpointConfig);
+
+                    final CollectStreamSink<RowData> sink =
+                            new CollectStreamSink<>(inputStream, factory);
+                    return sink.name("Collect table sink");
+                };
+    }
+
+    @Override
+    public DynamicTableSink copy() {
+        final CollectDynamicSink copy = new 
CollectDynamicSink(tableIdentifier, consumedDataType);
+        copy.iterator = iterator;

Review comment:
       This violates the copy contract, doesn't it?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java
##########
@@ -75,6 +80,49 @@
 @Internal
 public final class DynamicSinkUtils {
 
+    /** Converts an {@link TableResult#collect()} sink to a {@link RelNode}. */
+    public static RelNode convertCollectToRel(
+            FlinkRelBuilder relBuilder,
+            RelNode input,
+            CollectModifyOperation collectModifyOperation) {
+        final DataTypeFactory dataTypeFactory =
+                
unwrapContext(relBuilder).getCatalogManager().getDataTypeFactory();
+        final ResolvedSchema childSchema = 
collectModifyOperation.getChild().getResolvedSchema();
+        final ResolvedSchema schema =
+                ResolvedSchema.physical(
+                        childSchema.getColumnNames(), 
childSchema.getColumnDataTypes());
+        final CatalogTable unresolvedTable = new InlineCatalogTable(schema);
+        final ResolvedCatalogTable catalogTable = new 
ResolvedCatalogTable(unresolvedTable, schema);
+
+        final DataType consumedDataType = fixCollectDataType(dataTypeFactory, 
schema);
+
+        final CollectDynamicSink tableSink =
+                new CollectDynamicSink(
+                        collectModifyOperation.getTableIdentifier(), 
consumedDataType);
+        
collectModifyOperation.setSelectResultProvider(tableSink.getSelectResultProvider());
+        return convertSinkToRel(
+                relBuilder,
+                input,
+                collectModifyOperation.getTableIdentifier(),
+                Collections.emptyMap(),
+                false,
+                tableSink,
+                catalogTable);
+    }
+
+    /** Temporary solution until we drop legacy types. */
+    private static DataType fixCollectDataType(
+            DataTypeFactory dataTypeFactory, ResolvedSchema schema) {
+        final DataType fixedDataType =
+                DataTypeUtils.transform(
+                        dataTypeFactory,
+                        schema.toSourceRowDataType(),
+                        TypeTransformations.legacyRawToTypeInfoRaw(),
+                        TypeTransformations.legacyToNonLegacy());
+        // TODO erase the conversion class earlier when dropping legacy code

Review comment:
       Is this a TODO for a specific issue?




-- 
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:
[email protected]


Reply via email to