wuchong commented on a change in pull request #14822: URL: https://github.com/apache/flink/pull/14822#discussion_r569922375
########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java ########## @@ -192,6 +196,9 @@ public CommonExecSink( finalInputTransform = new PartitionTransformation<>(inputTransform, partitioner); finalInputTransform.setParallelism(parallelism); } + + final SinkOperator operator = + new SinkOperator(env.clean(sinkFunction), rowtimeFieldIndex, enforcer); Review comment: I think we need to do some refactoring for the `createSinkTransformation` method, currently the logic is quite mess. One idea is we can make enforcer always as an single operator if there are not null fields, so the enforcer will not be wrap in the `SinkOperator`. ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java ########## @@ -170,6 +146,27 @@ public CommonExecSink( parallelism = inputParallelism; } + final SinkFunction<RowData> sinkFunction; + if (runtimeProvider instanceof SinkFunctionProvider) { + sinkFunction = ((SinkFunctionProvider) runtimeProvider).createSinkFunction(); + } else if (runtimeProvider instanceof OutputFormatProvider) { + sinkFunction = + new OutputFormatSinkFunction<>( + ((OutputFormatProvider) runtimeProvider).createOutputFormat()); + } else if (runtimeProvider instanceof SinkProvider) { + return new SinkTransformation<>( Review comment: We should also add key partitioner if needed. ########## File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/factories/TestFileSinkFactory.java ########## @@ -0,0 +1,114 @@ +/* + * 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.factories; + +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.filesystem.FileSystemOptions; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.HashSet; +import java.util.Set; + +/** Test file source {@link DynamicTableSinkFactory}. */ +public class TestFileSinkFactory implements DynamicTableSinkFactory { Review comment: I would suggest to merge `TestFileSinkFactory` and `TestFileSourceFactory` into a single `TestFileFactory`, in this way, a registered table can both be used as sink and source. ########## File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala ########## @@ -922,6 +925,42 @@ class TableSinkITCase extends StreamingTestBase { } + @Test + def testSinkProvider(): Unit = { + val file = tempFolder.newFolder() + tEnv.executeSql( + s""" + |CREATE TABLE MyFileSinkTable ( + | `a` STRING, + | `b` STRING, + | `c` STRING + |) WITH ( + | 'connector' = 'filesink', + | 'path' = '${file.getAbsolutePath}' + |) + |""".stripMargin) + + val stringTupleData3: Seq[(String, String, String)] = { + val data = new mutable.MutableList[(String, String, String)] + data.+=(("Test", "Sink", "Hi")) + data.+=(("Sink", "Provider", "Hello")) + data.+=(("Test", "Provider", "Hello world")) + data + } + val table = env.fromCollection(stringTupleData3).toTable(tEnv, 'a, 'b, 'c) + table.executeInsert("MyFileSinkTable").await() + + val source = Source.fromFile(new File(file.getAbsolutePath, file.list()(0)).listFiles()(0).getAbsolutePath) Review comment: If we merge the `filesink` and `filesource` into one connector, we can simply read `MyFileSinkTable` using SELECT query. ---------------------------------------------------------------- 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