55import com .clickhouse .client .api .insert .InsertResponse ;
66import com .clickhouse .client .api .insert .InsertSettings ;
77import com .clickhouse .data .ClickHouseFormat ;
8+ import com .clickhouse .utils .Utils ;
89import org .apache .flink .api .connector .sink2 .WriterInitContext ;
910import org .apache .flink .connector .base .sink .writer .AsyncSinkWriter ;
1011import org .apache .flink .connector .base .sink .writer .BufferedRequestState ;
1112import org .apache .flink .connector .base .sink .writer .ElementConverter ;
1213import org .apache .flink .connector .base .sink .writer .ResultHandler ;
1314import org .apache .flink .connector .base .sink .writer .config .AsyncSinkWriterConfiguration ;
1415import org .apache .flink .connector .clickhouse .data .ClickHousePayload ;
16+ import org .apache .flink .connector .clickhouse .exception .RetriableException ;
17+ import org .apache .flink .metrics .Counter ;
18+ import org .apache .flink .metrics .groups .SinkWriterMetricGroup ;
1519import org .slf4j .Logger ;
1620import org .slf4j .LoggerFactory ;
1721
@@ -26,6 +30,10 @@ public class ClickHouseAsyncWriter<InputT> extends AsyncSinkWriter<InputT, Click
2630 private final ClickHouseClientConfig clickHouseClientConfig ;
2731 private ClickHouseFormat clickHouseFormat = null ;
2832
33+ private final Counter numBytesSendCounter ;
34+ private final Counter numRecordsSendCounter ;
35+ private final Counter numRequestSubmittedCounter ;
36+
2937 public ClickHouseAsyncWriter (ElementConverter <InputT , ClickHousePayload > elementConverter ,
3038 WriterInitContext context ,
3139 int maxBatchSize ,
@@ -50,6 +58,10 @@ public ClickHouseAsyncWriter(ElementConverter<InputT, ClickHousePayload> element
5058 state );
5159 this .clickHouseClientConfig = clickHouseClientConfig ;
5260 this .clickHouseFormat = clickHouseFormat ;
61+ final SinkWriterMetricGroup metricGroup = context .metricGroup ();
62+ this .numBytesSendCounter = metricGroup .getNumBytesSendCounter ();
63+ this .numRecordsSendCounter = metricGroup .getNumRecordsSendCounter ();
64+ this .numRequestSubmittedCounter = metricGroup .counter ("numRequestSubmitted" );
5365 }
5466
5567 @ Override
@@ -59,25 +71,36 @@ protected long getSizeInBytes(ClickHousePayload clickHousePayload) {
5971
6072 @ Override
6173 protected void submitRequestEntries (List <ClickHousePayload > requestEntries , ResultHandler <ClickHousePayload > resultHandler ) {
74+ this .numRequestSubmittedCounter .inc ();
6275 LOG .info ("Submitting request entries..." );
63- AtomicInteger totalSizeSend = new AtomicInteger ();
6476 Client chClient = this .clickHouseClientConfig .createClient ();
6577 String tableName = clickHouseClientConfig .getTableName ();
6678 // TODO: get from constructor or ClickHousePayload need to think what is the best way
6779 ClickHouseFormat format = null ;
68- if (clickHouseFormat == null ) {
69- // this not define lets try to get it from ClickHousePayload in case of POJO can be RowBinary or RowBinaryWithDefaults
70- } else {
80+ if (clickHouseFormat != null ) {
7181 format = clickHouseFormat ;
82+ } else {
83+ // TODO: check if we configured payload to POJO serialization.
84+ // this not define lets try to get it from ClickHousePayload in case of POJO can be RowBinary or RowBinaryWithDefaults
85+ Boolean supportDefault = clickHouseClientConfig .getSupportDefault ();
86+ if (supportDefault != null ) {
87+ if (supportDefault ) format = ClickHouseFormat .RowBinaryWithDefaults ;
88+ else format = ClickHouseFormat .RowBinary ;
89+ } else {
90+ throw new RuntimeException ("ClickHouseFormat was not set " );
91+ }
7292 }
7393 try {
7494 CompletableFuture <InsertResponse > response = chClient .insert (tableName , out -> {
7595 for (ClickHousePayload requestEntry : requestEntries ) {
7696 byte [] payload = requestEntry .getPayload ();
77- totalSizeSend .addAndGet (payload .length );
97+ // sum the data that is sent to ClickHouse
98+ this .numBytesSendCounter .inc (payload .length );
7899 out .write (payload );
79100 }
80- LOG .info ("Data that will be send to ClickHouse in bytes {} and the amount of records {}." , totalSizeSend .get (), requestEntries .size ());
101+ // send the number that is sent to ClickHouse
102+ this .numRecordsSendCounter .inc (requestEntries .size ());
103+ LOG .info ("Data that will be send to ClickHouse in bytes {} and the amount of records {}." , numBytesSendCounter .getCount (), requestEntries .size ());
81104 out .close ();
82105 }, format , new InsertSettings ().setOption (ClientConfigProperties .ASYNC_OPERATIONS .getKey (), "true" ));
83106 response .whenComplete ((insertResponse , throwable ) -> {
@@ -108,9 +131,16 @@ private void handleFailedRequest(
108131 List <ClickHousePayload > requestEntries ,
109132 ResultHandler <ClickHousePayload > resultHandler ,
110133 Throwable error ) {
111- // TODO extract from error if we can retry
112- error .printStackTrace ();
113-
134+ // TODO: extract from error if we can retry
135+ try {
136+ Utils .handleException (error );
137+ } catch (RetriableException e ) {
138+ LOG .info ("Retriable exception occurred while processing request. " , e );
139+ // TODO: send data again
140+ resultHandler .retryForEntries (requestEntries );
141+ }
142+ LOG .info ("completeExceptionally" );
143+ resultHandler .completeExceptionally ((Exception )error );
114144 }
115145
116146}
0 commit comments