wuchong commented on a change in pull request #8867: [FLINK-12956][jdbc] 
Introduce upsert table sink for JDBC
URL: https://github.com/apache/flink/pull/8867#discussion_r298554434
 
 

 ##########
 File path: 
flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java
 ##########
 @@ -0,0 +1,240 @@
+/*
+ * 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.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.io.jdbc.upsert.QueryExecutor;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.sinks.BatchTableSink;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.sinks.UpsertStreamTableSink;
+import org.apache.flink.table.utils.TableConnectorUtils;
+import org.apache.flink.types.Row;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.flink.api.java.io.jdbc.AbstractJDBCOutputFormat.DEFAULT_BATCH_INTERVAL;
+import static 
org.apache.flink.api.java.io.jdbc.JDBCUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES;
+
+/**
+ * An upsert {@link UpsertStreamTableSink} for JDBC.
+ */
+public class JDBCUpsertTableSink implements UpsertStreamTableSink<Row>, 
BatchTableSink<Tuple2<Boolean, Row>> {
+
+       private final String tableName;
+       private final String username;
+       private final String password;
+       private final String drivername;
+       private final String dbURL;
+       private final QueryExecutor appendExec;
+       private final QueryExecutor replaceExec;
+       private final QueryExecutor deleteExec;
+       private final int batchInterval;
+       private final int maxRetryTime;
+
+       private String[] keyFields;
+       private String[] fieldNames;
+       private TypeInformation[] fieldTypes;
+
+       private JDBCUpsertTableSink(
+                       String tableName, String username, String password, 
String drivername, String dbURL,
+                       QueryExecutor appendExec, QueryExecutor replaceExec, 
QueryExecutor deleteExec,
+                       int batchInterval, int maxRetryTime) {
+               this.tableName = tableName;
+               this.username = username;
+               this.password = password;
+               this.drivername = drivername;
+               this.dbURL = dbURL;
+               this.appendExec = appendExec;
+               this.replaceExec = replaceExec;
+               this.deleteExec = deleteExec;
+               this.batchInterval = batchInterval;
+               this.maxRetryTime = maxRetryTime;
+       }
+
+       private JDBCUpsertOutputFormat newFormat() {
+               // sql types
+               int[] sqlTypes = Arrays.stream(fieldTypes)
+                               
.mapToInt(JDBCTypeUtil::typeInformationToSqlType).toArray();
+
+               // pk fields
+               List<String> nameList = Arrays.asList(fieldNames);
+               int[] pkFields = keyFields == null ? null :
+                               
Arrays.stream(keyFields).mapToInt(nameList::indexOf).toArray();
+
+               return JDBCUpsertOutputFormat.builder()
+                               .setTableName(tableName)
+                               .setFieldNames(fieldNames)
+                               .setUsername(username)
+                               .setPassword(password)
+                               .setDrivername(drivername)
+                               .setDBUrl(dbURL)
+                               .setAppendExec(appendExec)
+                               .setReplaceExec(replaceExec)
+                               .setDeleteExec(deleteExec)
+                               .setBatchInterval(batchInterval)
+                               .setMaxRetryTimes(maxRetryTime)
+                               .setSqlTypes(sqlTypes)
+                               .setPkFields(pkFields)
+                               .build();
+       }
+
+       @Override
+       public void emitDataStream(DataStream<Tuple2<Boolean, Row>> dataStream) 
{
+               dataStream
+                               .addSink(new 
JDBCUpsertSinkFunction(newFormat()))
+                               
.name(TableConnectorUtils.generateRuntimeName(this.getClass(), fieldNames));
+       }
+
+       @Override
+       public void emitDataSet(DataSet<Tuple2<Boolean, Row>> dataSet) {
+               dataSet.output(newFormat());
+       }
+
+       @Override
+       public void setKeyFields(String[] keys) {
+               this.keyFields = keys;
+       }
+
+       @Override
+       public void setIsAppendOnly(Boolean isAppendOnly) {
+       }
+
+       @Override
+       public TypeInformation<Tuple2<Boolean, Row>> getOutputType() {
+               return new TupleTypeInfo<>(Types.BOOLEAN, getRecordType());
+       }
+
+       @Override
+       public TypeInformation<Row> getRecordType() {
+               return new RowTypeInfo(fieldTypes, fieldNames);
+       }
+
+       @Override
+       public String[] getFieldNames() {
+               return fieldNames;
+       }
+
+       @Override
+       public TypeInformation<?>[] getFieldTypes() {
+               return fieldTypes;
+       }
+
+       @Override
+       public TableSink<Tuple2<Boolean, Row>> configure(String[] fieldNames, 
TypeInformation<?>[] fieldTypes) {
+               JDBCUpsertTableSink copy = new JDBCUpsertTableSink(tableName, 
username, password,
+                               drivername, dbURL, appendExec, replaceExec, 
deleteExec, batchInterval, maxRetryTime);
+
+               copy.fieldNames = fieldNames;
+               copy.fieldTypes = fieldTypes;
 
 Review comment:
   We shouldn't simply override field types. We need to check the query field 
types equal to the sink's field types. See `JDBCAppendTableSink#configure`.

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


With regards,
Apache Git Services

Reply via email to