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

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_r207658085
 
 

 ##########
 File path: 
flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java
 ##########
 @@ -91,6 +101,19 @@ private void establishConnection() throws SQLException, 
ClassNotFoundException {
                } else {
                        dbConn = DriverManager.getConnection(dbURL, username, 
password);
                }
+               timer = new Timer();
+               timer.schedule(new TimerTask() {
+                       @Override
+                       public void run() {
+                               try {
 
 Review comment:
   Should we throw a runtime exceptions here? It might be a better idea to wait 
until the actual JDBC write to throw in case it is just transient, and might 
got recovered afterwards during batch write 

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