2020
2121import com .dtstack .flink .sql .sink .rdb .RdbSink ;
2222import org .apache .commons .lang3 .StringUtils ;
23- import org .apache .flink .api .java .tuple .Tuple ;
2423import org .apache .flink .api .java .tuple .Tuple2 ;
2524import org .apache .flink .configuration .Configuration ;
2625import org .apache .flink .shaded .guava18 .com .google .common .collect .Lists ;
2726import org .apache .flink .shaded .guava18 .com .google .common .collect .Maps ;
2827import org .apache .flink .types .Row ;
2928import org .slf4j .Logger ;
3029import org .slf4j .LoggerFactory ;
31- import java .sql .*;
32- import java .util .*;
3330import java .io .IOException ;
31+ import java .sql .Connection ;
32+ import java .sql .DriverManager ;
33+ import java .sql .PreparedStatement ;
34+ import java .sql .SQLException ;
35+ import java .util .Arrays ;
36+ import java .util .List ;
37+ import java .util .Map ;
38+ import java .util .concurrent .ScheduledFuture ;
39+ import java .util .concurrent .ScheduledThreadPoolExecutor ;
40+ import java .util .concurrent .TimeUnit ;
41+ import java .util .concurrent .atomic .AtomicInteger ;
3442
3543import com .dtstack .flink .sql .sink .MetricOutputFormat ;
3644
3745/**
3846 * OutputFormat to write tuples into a database.
3947 * The OutputFormat has to be configured using the supplied OutputFormatBuilder.
4048 *
41- * @see Tuple
42- * @see DriverManager
4349 */
4450public class RetractJDBCOutputFormat extends MetricOutputFormat {
4551 private static final long serialVersionUID = 1L ;
@@ -53,22 +59,25 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat {
5359 private String tableName ;
5460 private String dbType ;
5561 private RdbSink dbSink ;
56- private int batchInterval = 5000 ;
62+ private long batchWaitInterval ;
63+ private int batchNum ;
5764 private String insertQuery ;
5865 public int [] typesArray ;
5966
6067 private Connection dbConn ;
6168 private PreparedStatement upload ;
6269
63- private int batchCount = 0 ;
70+ private AtomicInteger batchCount = new AtomicInteger ( 0 ) ;
6471
6572 //index field
6673 private Map <String , List <String >> realIndexes = Maps .newHashMap ();
6774 //full field
6875 private List <String > fullField = Lists .newArrayList ();
6976
70- public RetractJDBCOutputFormat () {
77+ private transient ScheduledThreadPoolExecutor timerService ;
7178
79+ public RetractJDBCOutputFormat () {
80+ this .timerService = new ScheduledThreadPoolExecutor (1 );
7281 }
7382
7483 @ Override
@@ -151,13 +160,23 @@ public void writeRecord(Tuple2 tuple2) throws IOException {
151160
152161
153162 private void insertWrite (Row row ) throws SQLException {
154-
155163 updatePreparedStmt (row , upload );
156164 upload .addBatch ();
157- batchCount ++;
158- if (batchCount >= batchInterval ) {
165+ ScheduledFuture <?> scheduledFuture = null ;
166+
167+ if (batchWaitInterval > 0 && batchCount .get () == 0 ) {
168+ scheduledFuture = registerTimer (batchWaitInterval , this );
169+ }
170+
171+ batchCount .incrementAndGet ();
172+
173+ if (batchCount .get () >= batchNum ) {
159174 upload .executeBatch ();
160- batchCount = 0 ;
175+ batchCount .set (0 );
176+
177+ if (scheduledFuture != null ) {
178+ scheduledFuture .cancel (true );
179+ }
161180 }
162181 }
163182
@@ -250,6 +269,28 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce
250269 }
251270 }
252271
272+ public ScheduledFuture <?> registerTimer (long delay , RetractJDBCOutputFormat outputFormat ) {
273+ return timerService .schedule (new DelayExecuteBatch (outputFormat ), delay , TimeUnit .MILLISECONDS );
274+ }
275+
276+ private final static class DelayExecuteBatch implements Runnable {
277+ RetractJDBCOutputFormat outputFormat ;
278+
279+ private DelayExecuteBatch (RetractJDBCOutputFormat outputFormat ) {
280+ this .outputFormat = outputFormat ;
281+ }
282+
283+ @ Override
284+ public void run () {
285+ try {
286+ outputFormat .upload .executeBatch ();
287+ outputFormat .batchCount .set (0 );
288+ } catch (SQLException e ) {
289+ LOG .error ("delay batch insert error..." , e );
290+ }
291+ }
292+ }
293+
253294 /**
254295 * Executes prepared statement and closes all resources of this instance.
255296 *
@@ -266,7 +307,7 @@ public void close() throws IOException {
266307 LOG .info ("Inputformat couldn't be closed - " + se .getMessage ());
267308 } finally {
268309 upload = null ;
269- batchCount = 0 ;
310+ batchCount . set ( 0 ) ;
270311 }
271312
272313 try {
@@ -332,8 +373,8 @@ public void setDbSink(RdbSink dbSink) {
332373 this .dbSink = dbSink ;
333374 }
334375
335- public void setBatchInterval (int batchInterval ) {
336- this .batchInterval = batchInterval ;
376+ public void setBatchNum (int batchNum ) {
377+ this .batchNum = batchNum ;
337378 }
338379
339380 public void setInsertQuery (String insertQuery ) {
@@ -364,6 +405,11 @@ public Map<String, List<String>> getRealIndexes() {
364405 return realIndexes ;
365406 }
366407
408+
409+ public void setBatchWaitInterval (long batchWaitInterval ) {
410+ this .batchWaitInterval = batchWaitInterval ;
411+ }
412+
367413 public List <String > getFullField () {
368414 return fullField ;
369415 }
0 commit comments