[ 
https://issues.apache.org/jira/browse/FLINK-9794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16568716#comment-16568716
 ] 

ASF GitHub Bot commented on FLINK-9794:
---------------------------------------

walterddr commented on a change in pull request #6301: [FLINK-9794] [jdbc] 
JDBCOutputFormat does not consider idle connection and multithreads 
synchronization
URL: https://github.com/apache/flink/pull/6301#discussion_r207659233
 
 

 ##########
 File path: 
flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
 ##########
 @@ -107,98 +130,11 @@ private void establishConnection() throws SQLException, 
ClassNotFoundException {
         */
        @Override
        public void writeRecord(Row row) throws IOException {
-
                if (typesArray != null && typesArray.length > 0 && 
typesArray.length != row.getArity()) {
                        LOG.warn("Column SQL types array doesn't match arity of 
passed Row! Check the passed array...");
                }
                try {
-
-                       if (typesArray == null) {
-                               // no types provided
-                               for (int index = 0; index < row.getArity(); 
index++) {
-                                       LOG.warn("Unknown column type for 
column {}. Best effort approach to set its value: {}.", index + 1, 
row.getField(index));
-                                       upload.setObject(index + 1, 
row.getField(index));
-                               }
-                       } else {
-                               // types provided
-                               for (int index = 0; index < row.getArity(); 
index++) {
-
-                                       if (row.getField(index) == null) {
-                                               upload.setNull(index + 1, 
typesArray[index]);
-                                       } else {
-                                               // casting values as suggested 
by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html
-                                               switch (typesArray[index]) {
-                                                       case 
java.sql.Types.NULL:
-                                                               
upload.setNull(index + 1, typesArray[index]);
-                                                               break;
-                                                       case 
java.sql.Types.BOOLEAN:
-                                                       case java.sql.Types.BIT:
-                                                               
upload.setBoolean(index + 1, (boolean) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.CHAR:
-                                                       case 
java.sql.Types.NCHAR:
-                                                       case 
java.sql.Types.VARCHAR:
-                                                       case 
java.sql.Types.LONGVARCHAR:
-                                                       case 
java.sql.Types.LONGNVARCHAR:
-                                                               
upload.setString(index + 1, (String) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.TINYINT:
-                                                               
upload.setByte(index + 1, (byte) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.SMALLINT:
-                                                               
upload.setShort(index + 1, (short) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.INTEGER:
-                                                               
upload.setInt(index + 1, (int) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.BIGINT:
-                                                               
upload.setLong(index + 1, (long) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.REAL:
-                                                               
upload.setFloat(index + 1, (float) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.FLOAT:
-                                                       case 
java.sql.Types.DOUBLE:
-                                                               
upload.setDouble(index + 1, (double) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.DECIMAL:
-                                                       case 
java.sql.Types.NUMERIC:
-                                                               
upload.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.DATE:
-                                                               
upload.setDate(index + 1, (java.sql.Date) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.TIME:
-                                                               
upload.setTime(index + 1, (java.sql.Time) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.TIMESTAMP:
-                                                               
upload.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index));
-                                                               break;
-                                                       case 
java.sql.Types.BINARY:
-                                                       case 
java.sql.Types.VARBINARY:
-                                                       case 
java.sql.Types.LONGVARBINARY:
-                                                               
upload.setBytes(index + 1, (byte[]) row.getField(index));
-                                                               break;
-                                                       default:
-                                                               
upload.setObject(index + 1, row.getField(index));
-                                                               
LOG.warn("Unmanaged sql type ({}) for column {}. Best effort approach to set 
its value: {}.",
-                                                                       
typesArray[index], index + 1, row.getField(index));
-                                                               // case 
java.sql.Types.SQLXML
-                                                               // case 
java.sql.Types.ARRAY:
-                                                               // case 
java.sql.Types.JAVA_OBJECT:
-                                                               // case 
java.sql.Types.BLOB:
-                                                               // case 
java.sql.Types.CLOB:
-                                                               // case 
java.sql.Types.NCLOB:
-                                                               // case 
java.sql.Types.DATALINK:
-                                                               // case 
java.sql.Types.DISTINCT:
-                                                               // case 
java.sql.Types.OTHER:
-                                                               // case 
java.sql.Types.REF:
-                                                               // case 
java.sql.Types.ROWID:
-                                                               // case 
java.sql.Types.STRUC
-                                               }
-                                       }
-                               }
-                       }
+                       fillStmt(row);
 
 Review comment:
   any reason to use another private function? seems identical except 
indentation.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> JDBCOutputFormat does not consider idle connection and multithreads 
> synchronization
> -----------------------------------------------------------------------------------
>
>                 Key: FLINK-9794
>                 URL: https://issues.apache.org/jira/browse/FLINK-9794
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming Connectors
>    Affects Versions: 1.4.0, 1.5.0
>            Reporter: wangsan
>            Assignee: wangsan
>            Priority: Major
>              Labels: pull-request-available
>
> Current implementation of  JDBCOutputFormat has two potential problems: 
> 1. The Connection was established when JDBCOutputFormat is opened, and will 
> be used all the time. But if this connection lies idle for a long time, the 
> database will force close the connection, thus errors may occur.
> 2. The flush() method is called when batchCount exceeds the threshold, but it 
> is also called while snapshotting state. So two threads may modify upload and 
> batchCount, but without synchronization.
> We need fix these two problems to make JDBCOutputFormat more reliable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to