18
18
19
19
package com .dtstack .flink .sql .sink .kudu ;
20
20
21
+ import com .dtstack .flink .sql .factory .DTThreadFactory ;
21
22
import com .dtstack .flink .sql .outputformat .AbstractDtRichOutputFormat ;
23
+ import com .dtstack .flink .sql .sink .kudu .table .KuduTableInfo ;
22
24
import com .dtstack .flink .sql .util .KrbUtils ;
23
25
import org .apache .flink .api .common .typeinfo .TypeInformation ;
24
26
import org .apache .flink .api .java .tuple .Tuple2 ;
31
33
import org .apache .kudu .client .KuduTable ;
32
34
import org .apache .kudu .client .Operation ;
33
35
import org .apache .kudu .client .PartialRow ;
36
+ import org .apache .kudu .client .SessionConfiguration ;
34
37
import org .slf4j .Logger ;
35
38
import org .slf4j .LoggerFactory ;
36
39
40
43
import java .sql .Timestamp ;
41
44
import java .util .Date ;
42
45
import java .util .Objects ;
46
+ import java .util .concurrent .ScheduledExecutorService ;
47
+ import java .util .concurrent .ScheduledFuture ;
48
+ import java .util .concurrent .ScheduledThreadPoolExecutor ;
49
+ import java .util .concurrent .TimeUnit ;
50
+ import java .util .concurrent .atomic .AtomicInteger ;
43
51
44
52
/**
45
53
* @author gituser
46
54
* @modify xiuzhu
47
55
*/
48
- public class KuduOutputFormat extends AbstractDtRichOutputFormat <Tuple2 > {
56
+ public class KuduOutputFormat extends AbstractDtRichOutputFormat <Tuple2 < Boolean , Row > > {
49
57
50
58
private static final long serialVersionUID = 1L ;
51
59
@@ -75,6 +83,24 @@ public class KuduOutputFormat extends AbstractDtRichOutputFormat<Tuple2> {
75
83
private String keytab ;
76
84
private String krb5conf ;
77
85
86
+ /**
87
+ * batch size
88
+ */
89
+ private Integer batchSize ;
90
+ private Integer batchWaitInterval ;
91
+ /**
92
+ * kudu session flush mode
93
+ */
94
+ private String flushMode ;
95
+
96
+ private transient AtomicInteger rowCount ;
97
+
98
+ /**
99
+ * 定时任务
100
+ */
101
+ private transient ScheduledExecutorService scheduler ;
102
+ private transient ScheduledFuture <?> scheduledFuture ;
103
+
78
104
private KuduOutputFormat () {
79
105
}
80
106
@@ -91,6 +117,33 @@ public void configure(Configuration parameters) {
91
117
public void open (int taskNumber , int numTasks ) throws IOException {
92
118
establishConnection ();
93
119
initMetric ();
120
+ initSchedulerTask ();
121
+ rowCount = new AtomicInteger (0 );
122
+ }
123
+
124
+ /**
125
+ * init the scheduler task of {@link KuduOutputFormat#flush()}
126
+ */
127
+ private void initSchedulerTask () {
128
+ try {
129
+ if (batchWaitInterval > 0 ) {
130
+ this .scheduler = new ScheduledThreadPoolExecutor (
131
+ 1 ,
132
+ new DTThreadFactory ("kudu-batch-flusher" )
133
+ );
134
+
135
+ this .scheduledFuture = this .scheduler .scheduleWithFixedDelay (
136
+ () -> {
137
+ synchronized (KuduOutputFormat .this ) {
138
+ flush ();
139
+ }
140
+ }, batchWaitInterval , batchWaitInterval , TimeUnit .MILLISECONDS
141
+ );
142
+ }
143
+ } catch (Exception e ) {
144
+ LOG .error ("init schedule task failed !" );
145
+ throw new RuntimeException (e );
146
+ }
94
147
}
95
148
96
149
private void establishConnection () throws IOException {
@@ -127,17 +180,46 @@ private void establishConnection() throws IOException {
127
180
}
128
181
LOG .info ("connect kudu is succeed!" );
129
182
130
- session = client .newSession ();
183
+ session = buildSessionWithFlushMode (flushMode , client );
184
+ }
185
+
186
+ /**
187
+ * According to the different flush mode, construct different session. Detail see {@link SessionConfiguration.FlushMode}
188
+ *
189
+ * @param flushMode flush mode
190
+ * @param kuduClient kudu client
191
+ * @return KuduSession with flush mode
192
+ * @throws KuduException kudu exception when session flush
193
+ */
194
+ private KuduSession buildSessionWithFlushMode (String flushMode , KuduClient kuduClient ) throws KuduException {
195
+ KuduSession kuduSession = kuduClient .newSession ();
196
+ if (flushMode .equalsIgnoreCase (KuduTableInfo .KuduFlushMode .MANUAL_FLUSH .name ())) {
197
+ kuduSession .setFlushMode (SessionConfiguration .FlushMode .MANUAL_FLUSH );
198
+ kuduSession .setMutationBufferSpace (
199
+ Integer .parseInt (String .valueOf (Math .round (batchSize * 1.2 )))
200
+ );
201
+ }
202
+
203
+ if (flushMode .equalsIgnoreCase (KuduTableInfo .KuduFlushMode .AUTO_FLUSH_SYNC .name ())) {
204
+ LOG .warn ("Parameter [batchSize] will not take effect at AUTO_FLUSH_SYNC mode." );
205
+ kuduSession .setFlushMode (SessionConfiguration .FlushMode .AUTO_FLUSH_SYNC );
206
+ }
207
+
208
+ if (flushMode .equalsIgnoreCase (KuduTableInfo .KuduFlushMode .AUTO_FLUSH_BACKGROUND .name ())) {
209
+ LOG .warn ("Unable to determine the order of data at AUTO_FLUSH_BACKGROUND mode." );
210
+ kuduSession .setFlushMode (SessionConfiguration .FlushMode .AUTO_FLUSH_BACKGROUND );
211
+ }
212
+
213
+ return kuduSession ;
131
214
}
132
215
133
216
@ Override
134
- public void writeRecord (Tuple2 record ) throws IOException {
135
- Tuple2 <Boolean , Row > tupleTrans = record ;
136
- Boolean retract = tupleTrans .getField (0 );
217
+ public void writeRecord (Tuple2 <Boolean , Row > record ) throws IOException {
218
+ Boolean retract = record .getField (0 );
137
219
if (!retract ) {
138
220
return ;
139
221
}
140
- Row row = tupleTrans .getField (1 );
222
+ Row row = record .getField (1 );
141
223
if (row .getArity () != fieldNames .length ) {
142
224
if (outDirtyRecords .getCount () % DIRTY_PRINT_FREQUENCY == 0 ) {
143
225
LOG .error ("record insert failed ..{}" , row .toString ());
@@ -151,6 +233,9 @@ public void writeRecord(Tuple2 record) throws IOException {
151
233
if (outRecords .getCount () % ROW_PRINT_FREQUENCY == 0 ) {
152
234
LOG .info ("Receive data : {}" , row );
153
235
}
236
+ if (rowCount .getAndIncrement () >= batchSize ) {
237
+ flush ();
238
+ }
154
239
session .apply (toOperation (writeMode , row ));
155
240
outRecords .inc ();
156
241
} catch (KuduException e ) {
@@ -162,6 +247,22 @@ public void writeRecord(Tuple2 record) throws IOException {
162
247
}
163
248
}
164
249
250
+ private synchronized void flush () {
251
+ try {
252
+ if (session .isClosed ()) {
253
+ throw new IllegalStateException ("session is closed! flush data error!" );
254
+ }
255
+
256
+ session .flush ();
257
+ // clear
258
+ rowCount .set (0 );
259
+ } catch (KuduException kuduException ) {
260
+ LOG .error (
261
+ "flush data error!" , kuduException );
262
+ throw new RuntimeException (kuduException );
263
+ }
264
+ }
265
+
165
266
@ Override
166
267
public void close () {
167
268
if (Objects .nonNull (session ) && !session .isClosed ()) {
@@ -179,6 +280,14 @@ public void close() {
179
280
throw new IllegalArgumentException ("[closeKuduClient]:" + e .getMessage ());
180
281
}
181
282
}
283
+
284
+ if (scheduledFuture != null ) {
285
+ scheduledFuture .cancel (false );
286
+ }
287
+
288
+ if (scheduler != null ) {
289
+ scheduler .shutdownNow ();
290
+ }
182
291
}
183
292
184
293
private Operation toOperation (WriteMode writeMode , Row row ) {
@@ -345,6 +454,21 @@ public KuduOutputFormatBuilder setEnableKrb(boolean enableKrb) {
345
454
return this ;
346
455
}
347
456
457
+ public KuduOutputFormatBuilder setBatchSize (Integer batchSize ) {
458
+ kuduOutputFormat .batchSize = batchSize ;
459
+ return this ;
460
+ }
461
+
462
+ public KuduOutputFormatBuilder setBatchWaitInterval (Integer batchWaitInterval ) {
463
+ kuduOutputFormat .batchWaitInterval = batchWaitInterval ;
464
+ return this ;
465
+ }
466
+
467
+ public KuduOutputFormatBuilder setFlushMode (String flushMode ) {
468
+ kuduOutputFormat .flushMode = flushMode ;
469
+ return this ;
470
+ }
471
+
348
472
public KuduOutputFormat finish () {
349
473
if (kuduOutputFormat .kuduMasters == null ) {
350
474
throw new IllegalArgumentException ("No kuduMasters supplied." );
0 commit comments