11package cn .leancloud .kafka .consumer ;
22
3- import org .apache .kafka .clients .consumer .Consumer ;
4- import org .apache .kafka .clients .consumer .ConsumerRecord ;
5- import org .apache .kafka .clients .consumer .KafkaConsumer ;
6- import org .apache .kafka .clients .consumer .MockConsumer ;
3+ import org .apache .kafka .clients .consumer .*;
74import org .apache .kafka .common .serialization .Deserializer ;
85import org .slf4j .Logger ;
96import org .slf4j .LoggerFactory ;
@@ -85,6 +82,8 @@ private static void requireArgument(boolean expression, String template, Object.
8582 private Map <String , Object > configs ;
8683 private ConsumerRecordHandler <K , V > consumerRecordHandler ;
8784 private ConsumerSeekDestination forceSeekTo = ConsumerSeekDestination .NONE ;
85+ private Duration syncCommitRetryInterval = Duration .ofSeconds (1 );
86+ private int maxAttemptsForEachSyncCommit = 3 ;
8887 @ Nullable
8988 private Consumer <K , V > consumer ;
9089 @ Nullable
@@ -235,6 +234,83 @@ public LcKafkaConsumerBuilder<K, V> gracefulShutdownTimeout(Duration gracefulShu
235234 return this ;
236235 }
237236
237+ /**
238+ * Sets the amount of time to wait before retry a failed synchronous commit on calling {@link KafkaConsumer#commitSync()}.
239+ * or {@link KafkaConsumer#commitSync(Map)}. Every synchronous commit may fail but most of times they are caused by
240+ * {@link org.apache.kafka.common.errors.RetriableException} and we can retry commit on this kind of exception safely.
241+ * This configuration set the interval between each retry.
242+ * <p>
243+ * For those failures of asynchronous commit by calling {@link KafkaConsumer#commitAsync()} or
244+ * {@link KafkaConsumer#commitAsync(OffsetCommitCallback)}, we retry them by a synchronous commit automatically
245+ * when we found any of them. So we only need configurations for synchronous commits.
246+ * <p>
247+ * The default {@code syncCommitRetryIntervalMillis} is 1000.
248+ *
249+ * @param syncCommitRetryIntervalMillis the retry interval in milliseconds
250+ * @return this
251+ * @throws IllegalArgumentException if {@code syncCommitRetryIntervalMillis} is a negative value
252+ */
253+ public LcKafkaConsumerBuilder <K , V > syncCommitRetryIntervalMillis (long syncCommitRetryIntervalMillis ) {
254+ requireArgument (syncCommitRetryIntervalMillis >= 0 ,
255+ "syncCommitRetryIntervalMillis: %s (expected >= 0)" , syncCommitRetryIntervalMillis );
256+ this .syncCommitRetryInterval = Duration .ofMillis (syncCommitRetryIntervalMillis );
257+ return this ;
258+ }
259+
260+ /**
261+ * Sets the amount of time to wait before retry a failed synchronous commit on calling {@link KafkaConsumer#commitSync()}.
262+ * or {@link KafkaConsumer#commitSync(Map)}. Every synchronous commit may fail but most of times they are caused by
263+ * {@link org.apache.kafka.common.errors.RetriableException} and we can retry commit on this kind of exception safely.
264+ * This configuration set the interval between each retry.
265+ * <p>
266+ * For those failures of asynchronous commit by calling {@link KafkaConsumer#commitAsync()} or
267+ * {@link KafkaConsumer#commitAsync(OffsetCommitCallback)}, we retry them by a synchronous commit automatically
268+ * when we found any of them. So we only need configurations for synchronous commits.
269+ * <p>
270+ * The default {@code syncCommitRetryInterval} is 1 second.
271+ *
272+ * @param syncCommitRetryInterval the retry interval
273+ * @return this
274+ * @throws NullPointerException if {@code syncCommitRetryInterval} is null
275+ * @throws IllegalArgumentException if {@code syncCommitRetryInterval} is a negative duration
276+ */
277+ public LcKafkaConsumerBuilder <K , V > syncCommitRetryInterval (Duration syncCommitRetryInterval ) {
278+ requireNonNull (syncCommitRetryInterval , "syncCommitRetryInterval" );
279+ requireArgument (!syncCommitRetryInterval .isNegative (),
280+ "syncCommitRetryInterval: %s (expect positive or zero duration)" , syncCommitRetryInterval );
281+ this .syncCommitRetryInterval = syncCommitRetryInterval ;
282+ return this ;
283+ }
284+
285+ /**
286+ * Sets the maximum attempt times for a synchronous commit by calling {@link KafkaConsumer#commitSync()}.
287+ * or {@link KafkaConsumer#commitSync(Map)}. Every synchronous commit may fail but most of times they are caused by
288+ * {@link org.apache.kafka.common.errors.RetriableException} and we can retry commit on this kind of exception safely.
289+ * This configuration cap the maximum retry times. If attempts reach to {@code maxAttemptsForEachSyncCommit}, the cached
290+ * {@link org.apache.kafka.common.errors.RetriableException} will be rethrown by then it will cause the Kafka Consumer
291+ * to stop and quit.
292+ * <p>
293+ * For those failures of asynchronous commit by calling {@link KafkaConsumer#commitAsync()} or
294+ * {@link KafkaConsumer#commitAsync(OffsetCommitCallback)}, we retry them by a synchronous commit automatically
295+ * when we found any of them. So we only need configurations for synchronous commits.
296+ * <p>
297+ * Please note that {@code maxAttemptsForEachSyncCommit} multiplies
298+ * {@code syncCommitRetryInterval} should far lower than {@code max.poll.interval.ms}, otherwise Kafka Consumer
299+ * may encounter session timeout or polling timeout due to not calling {@link KafkaConsumer#poll(long)} for too long.
300+ * <p>
301+ * The default {@code maxAttemptsForEachSyncCommit} is 3.
302+ *
303+ * @param maxAttemptsForEachSyncCommit maximum attempt times for a synchronous commit
304+ * @return this
305+ * @throws IllegalArgumentException if {@code maxAttemptsForEachSyncCommit} is zero or a negative value
306+ */
307+ public LcKafkaConsumerBuilder <K , V > maxAttemptsForEachSyncCommit (int maxAttemptsForEachSyncCommit ) {
308+ requireArgument (maxAttemptsForEachSyncCommit > 0 ,
309+ "maxAttemptsForEachSyncCommit: %s (expect > 0)" , maxAttemptsForEachSyncCommit );
310+ this .maxAttemptsForEachSyncCommit = maxAttemptsForEachSyncCommit ;
311+ return this ;
312+ }
313+
238314 /**
239315 * When using async consumer to commit offset asynchronously, this argument can force consumer to do a synchronous
240316 * commit after there's already this ({@code maxPendingAsyncCommits}) many async commits on the fly without
@@ -407,7 +483,8 @@ public LcKafkaConsumerBuilder<K, V> forceSeekTo(ConsumerSeekDestination destinat
407483 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildAuto () {
408484 checkConfigs (AutoCommitConsumerConfigs .values ());
409485 consumer = buildConsumer (true );
410- policy = workerPool == ImmediateExecutorService .INSTANCE ? NoOpCommitPolicy .getInstance () : new AutoCommitPolicy <>(consumer );
486+ policy = workerPool == ImmediateExecutorService .INSTANCE ?
487+ NoOpCommitPolicy .getInstance () : new AutoCommitPolicy <>(consumer );
411488 return doBuild ();
412489 }
413490
@@ -435,7 +512,11 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildAuto() {
435512 */
436513 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildSync () {
437514 consumer = buildConsumer (false );
438- policy = new SyncCommitPolicy <>(consumer , getRecommitInterval ());
515+ policy = new SyncCommitPolicy <>(
516+ consumer ,
517+ syncCommitRetryInterval ,
518+ maxAttemptsForEachSyncCommit ,
519+ getRecommitInterval ());
439520 return doBuild ();
440521 }
441522
@@ -462,7 +543,11 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildSync() {
462543 */
463544 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildPartialSync () {
464545 consumer = buildConsumer (false );
465- policy = new PartialSyncCommitPolicy <>(consumer , getRecommitInterval ());
546+ policy = new PartialSyncCommitPolicy <>(
547+ consumer ,
548+ syncCommitRetryInterval ,
549+ maxAttemptsForEachSyncCommit ,
550+ getRecommitInterval ());
466551 return doBuild ();
467552 }
468553
@@ -494,7 +579,12 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildPartialSync() {
494579 */
495580 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildAsync () {
496581 consumer = buildConsumer (false );
497- policy = new AsyncCommitPolicy <>(consumer , getRecommitInterval (), maxPendingAsyncCommits );
582+ policy = new AsyncCommitPolicy <>(
583+ consumer ,
584+ syncCommitRetryInterval ,
585+ maxAttemptsForEachSyncCommit ,
586+ getRecommitInterval (),
587+ maxPendingAsyncCommits );
498588 return doBuild ();
499589 }
500590
@@ -525,7 +615,12 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildAsync() {
525615 */
526616 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildPartialAsync () {
527617 consumer = buildConsumer (false );
528- policy = new PartialAsyncCommitPolicy <>(consumer , getRecommitInterval (), maxPendingAsyncCommits );
618+ policy = new PartialAsyncCommitPolicy <>(
619+ consumer ,
620+ syncCommitRetryInterval ,
621+ maxAttemptsForEachSyncCommit ,
622+ getRecommitInterval (),
623+ maxPendingAsyncCommits );
529624 return doBuild ();
530625 }
531626
0 commit comments