[ https://issues.apache.org/jira/browse/FLINK-6281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16082468#comment-16082468 ]
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_r126718250 --- Diff: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSink.java --- @@ -0,0 +1,102 @@ +/* + * 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.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +/** + * An at-least-once Table sink for JDBC. + */ +public class JDBCTableSink extends RichSinkFunction<Row> --- End diff -- I would not extend `RichSinkFunction`. Although this might work in practice, I think this mixes the logical representation of a table (in the catalog and during optimization) with the actual runtime code. I'd rather implement a separate JdbcSinkFunction (within this file) and instantiate it in `emitDataStream()`. I also think that we should implement the `BatchTableSink` interface which would directly use the `JdbcOutputFormat`. > 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)