[ https://issues.apache.org/jira/browse/FLINK-6281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16093737#comment-16093737 ]
ASF GitHub Bot commented on FLINK-6281: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3712#discussion_r128219296 --- Diff: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java --- @@ -0,0 +1,85 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.BatchTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +/** + * An at-least-once Table sink for JDBC. + */ +public class JDBCAppendTableSink implements AppendStreamTableSink<Row>, BatchTableSink<Row> { + private final JDBCSinkFunction sink; + + private String[] fieldNames; + private TypeInformation[] fieldTypes; + + JDBCAppendTableSink(JDBCOutputFormat outputFormat) { + this.sink = new JDBCSinkFunction(outputFormat); + } + + public static JDBCAppendTableSinkBuilder builder() { + return new JDBCAppendTableSinkBuilder(); + } + + @Override + public void emitDataStream(DataStream<Row> dataStream) { + dataStream.addSink(sink); + } + + @Override + public void emitDataSet(DataSet<Row> dataSet) { + dataSet.output(sink.outputFormat); + } + + @Override + public TypeInformation<Row> getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation<?>[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink<Row> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) { + JDBCAppendTableSink copy = new JDBCAppendTableSink(sink.outputFormat); --- End diff -- We could validate that the types of the `JDBCOutputFormat` match the `fieldTypes` which are provided by the optimizer. Or do you have concerns regarding such a check? > Create TableSink for JDBC > ------------------------- > > Key: FLINK-6281 > URL: https://issues.apache.org/jira/browse/FLINK-6281 > Project: Flink > Issue Type: Improvement > Components: Table API & SQL > Reporter: Haohui Mai > Assignee: Haohui Mai > > It would be nice to integrate the table APIs with the JDBC connectors so that > the rows in the tables can be directly pushed into JDBC. -- This message was sent by Atlassian JIRA (v6.4.14#64029)