@@ -89,7 +89,7 @@ private static void requireArgument(boolean expression, String template, Object.
8989 @ Nullable
9090 private CommitPolicy <K , V > policy ;
9191 @ Nullable
92- private Duration forceWholeCommitInterval ;
92+ private Duration recommitInterval ;
9393
9494 private LcKafkaConsumerBuilder (Map <String , Object > kafkaConsumerConfigs ,
9595 ConsumerRecordHandler <K , V > consumerRecordHandler ) {
@@ -193,52 +193,56 @@ public LcKafkaConsumerBuilder<K, V> maxPendingAsyncCommits(int maxPendingAsyncCo
193193 }
194194
195195 /**
196- * The interval to commit all partitions and it's completed offsets to broker on a partial commit consumer.
196+ * The interval to commit all partitions and it's completed offsets to broker on a non-automatic commit consumer.
197197 * <p>
198- * This configuration is only valid and is required on partial commit consumer build with
198+ * This configuration is only valid and is required on a non-automatic commit consumer build with
199+ * {@link LcKafkaConsumerBuilder#buildSync()}, {@link LcKafkaConsumerBuilder#buildAsync()},
199200 * {@link LcKafkaConsumerBuilder#buildPartialSync()} or {@link LcKafkaConsumerBuilder#buildPartialAsync()}.
200201 * For these kind of consumers, usually they only commit offsets of a partition when there was records consumed from
201202 * that partition and all these consumed records was handled successfully. But we must periodically commit those
202- * subscribed partitions who have not have any records too. Otherwise, after commit offset log retention timeout,
203- * Kafka broker may forget where the current commit offset of these partition for the consumer are. Then, when the
204- * consumer crashed and recovered, if the consumer set "auto.offset.reset" configuration to "earliest", it may
205- * consume a already consumed record again. So please make sure that {@code forceWholeCommitIntervalInMillis}
206- * is within log retention time set on Kafka broker.
207- * <p>
208- * The default {@code forceWholeCommitInterval} is 1 hour.
203+ * subscribed partitions who have had records but no new records for a long time too. Otherwise, after commit offset
204+ * retention timeout, Kafka broker may forget where the current commit offset of these partition for the consumer
205+ * are. Then, when the consumer crashed and recovered, if the consumer set <code>auto.offset.reset</code>
206+ * configuration to <code>earliest</code>, it may consume a already consumed record again. So please make sure
207+ * that {@code recommitIntervalInMillis} is within the limit set by <code>offsets.retention.minutes</code>
208+ * on Kafka broker or even within 1/3 of that limit to tolerate some commit failures on async commit consumer.
209+ * <p>
210+ * The default {@code recommitInterval} is 1 hour.
209211 *
210- * @param forceWholeCommitIntervalInMillis the interval in millis seconds to do a whole commit
212+ * @param recommitIntervalInMillis the interval in millis seconds to do a recommit
211213 * @return this
212214 */
213- public LcKafkaConsumerBuilder <K , V > forceWholeCommitIntervalInMillis (long forceWholeCommitIntervalInMillis ) {
214- requireArgument (forceWholeCommitIntervalInMillis > 0 ,
215- "forceWholeCommitIntervalInMillis : %s (expected > 0)" , forceWholeCommitIntervalInMillis );
215+ public LcKafkaConsumerBuilder <K , V > recommitIntervalInMillis (long recommitIntervalInMillis ) {
216+ requireArgument (recommitIntervalInMillis > 0 ,
217+ "recommitIntervalInMillis : %s (expected > 0)" , recommitIntervalInMillis );
216218
217- this .forceWholeCommitInterval = Duration .ofMillis (forceWholeCommitIntervalInMillis );
219+ this .recommitInterval = Duration .ofMillis (recommitIntervalInMillis );
218220 return this ;
219221 }
220222
221223 /**
222- * The interval to commit all partitions and it's completed offsets to broker on a partial commit consumer.
224+ * The interval to commit all partitions and it's completed offsets to broker on a non-automatic commit consumer.
223225 * <p>
224- * This configuration is only valid on partial commit consumer build with
226+ * This configuration is only valid and is required on a non-automatic commit consumer build with
227+ * {@link LcKafkaConsumerBuilder#buildSync()}, {@link LcKafkaConsumerBuilder#buildAsync()},
225228 * {@link LcKafkaConsumerBuilder#buildPartialSync()} or {@link LcKafkaConsumerBuilder#buildPartialAsync()}.
226229 * For these kind of consumers, usually they only commit offsets of a partition when there was records consumed from
227230 * that partition and all these consumed records was handled successfully. But we must periodically commit those
228- * subscribed partitions who have not have any records too. Otherwise, after commit offset log retention timeout,
229- * Kafka broker may forget where the current commit offset of these partition for the consumer are. Then, when the
230- * consumer crashed and recovered, if the consumer set "auto.offset.reset" configuration to "earliest", it may
231- * consume a already consumed record again. So please make sure that {@code forceWholeCommitInterval}
232- * is within log retention time set on Kafka broker.
233- * <p>
234- * The default {@code forceWholeCommitInterval} is 1 hour.
231+ * subscribed partitions who have had records but no new records for a long time too. Otherwise, after commit offset
232+ * retention timeout, Kafka broker may forget where the current commit offset of these partition for the consumer
233+ * are. Then, when the consumer crashed and recovered, if the consumer set <code>auto.offset.reset</code>
234+ * configuration to <code>earliest</code>, it may consume a already consumed record again. So please make sure
235+ * that {@code recommitInterval} is within the limit set by <code>offsets.retention.minutes</code> on
236+ * Kafka broker or even within 1/3 of that limit to tolerate some commit failures on async commit consumer..
237+ * <p>
238+ * The default {@code recommitInterval} is 1 hour.
235239 *
236- * @param forceWholeCommitInterval the interval to do a whole commit
240+ * @param recommitInterval the interval to do a recommit
237241 * @return this
238242 */
239- public LcKafkaConsumerBuilder <K , V > forceWholeCommitInterval (Duration forceWholeCommitInterval ) {
240- requireNonNull (forceWholeCommitInterval , "forceWholeCommitInterval " );
241- this .forceWholeCommitInterval = forceWholeCommitInterval ;
243+ public LcKafkaConsumerBuilder <K , V > recommitInterval (Duration recommitInterval ) {
244+ requireNonNull (recommitInterval , "recommitInterval " );
245+ this .recommitInterval = recommitInterval ;
242246 return this ;
243247 }
244248
@@ -342,7 +346,7 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildAuto() {
342346 */
343347 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildSync () {
344348 consumer = buildConsumer (false );
345- policy = new SyncCommitPolicy <>(consumer );
349+ policy = new SyncCommitPolicy <>(consumer , getRecommitInterval () );
346350 return doBuild ();
347351 }
348352
@@ -367,15 +371,8 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildSync() {
367371 * @return this
368372 */
369373 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildPartialSync () {
370- if (forceWholeCommitInterval == null ) {
371- logger .warn ("Force whole commit interval is not set for a partial commit consumer, the default " +
372- "interval of 1 hour will be used." );
373- forceWholeCommitInterval = Duration .ofHours (1 );
374- }
375- assert forceWholeCommitInterval != null ;
376-
377374 consumer = buildConsumer (false );
378- policy = new PartialSyncCommitPolicy <>(consumer , forceWholeCommitInterval );
375+ policy = new PartialSyncCommitPolicy <>(consumer , getRecommitInterval () );
379376 return doBuild ();
380377 }
381378
@@ -406,7 +403,7 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildPartialSync() {
406403 */
407404 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildAsync () {
408405 consumer = buildConsumer (false );
409- policy = new AsyncCommitPolicy <>(consumer , maxPendingAsyncCommits );
406+ policy = new AsyncCommitPolicy <>(consumer , getRecommitInterval (), maxPendingAsyncCommits );
410407 return doBuild ();
411408 }
412409
@@ -435,15 +432,8 @@ public <K1 extends K, V1 extends V> LcKafkaConsumer<K1, V1> buildAsync() {
435432 * @return this
436433 */
437434 public <K1 extends K , V1 extends V > LcKafkaConsumer <K1 , V1 > buildPartialAsync () {
438- if (forceWholeCommitInterval == null ) {
439- logger .warn ("Force whole commit interval is not set for a partial commit consumer, the default " +
440- "interval of 30 seconds will be used." );
441- forceWholeCommitInterval = Duration .ofSeconds (30 );
442- }
443- assert forceWholeCommitInterval != null ;
444-
445435 consumer = buildConsumer (false );
446- policy = new PartialAsyncCommitPolicy <>(consumer , forceWholeCommitInterval , maxPendingAsyncCommits );
436+ policy = new PartialAsyncCommitPolicy <>(consumer , getRecommitInterval () , maxPendingAsyncCommits );
447437 return doBuild ();
448438 }
449439
@@ -494,6 +484,16 @@ private Consumer<K, V> buildConsumer(boolean autoCommit) {
494484 return new KafkaConsumer <>(configs , keyDeserializer , valueDeserializer );
495485 }
496486
487+ Duration getRecommitInterval () {
488+ if (recommitInterval == null ) {
489+ logger .warn ("Recommit interval is not set for a non-automatic commit consumer, the default " +
490+ "interval of 1 hour will be used." );
491+ recommitInterval = Duration .ofHours (1 );
492+ }
493+
494+ return recommitInterval ;
495+ }
496+
497497 private void checkConfigs (KafkaConfigsChecker [] checkers ) {
498498 for (KafkaConfigsChecker check : checkers ) {
499499 check .check (configs );
0 commit comments