KAFKA-19311 Document commitAsync behavioral differences between Classic and Async Consumer (#19864)

#15613 ensures that all `commitAsync` callbacks are triggered before
`commitSync` completes for `AsyncKafkaConsumer`. However, the related
changes to `ClassicKafkaConsumer`, #15693, were not merged. I assume
this might be because we intend to gradually move toward using AsyncConsumer
instead.

In short, this behavioral difference should be documented.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
This commit is contained in:
Ming-Yen Chung 2025-06-01 01:58:26 +08:00 committed by GitHub
parent cc0f06554b
commit f9df9b5652
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
1 changed files with 8 additions and 4 deletions

View File

@ -908,7 +908,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller).
* <p> * <p>
* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)} * Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
* (or similar) are guaranteed to have their callbacks invoked prior to completion of this method. * (or similar) are guaranteed to have their callbacks invoked prior to completion of this method,
* but only when the consumer is using the consumer group protocol.
* *
* @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried. * @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried.
* This fatal error can only occur if you are using automatic group management with {@link #subscribe(Collection)}, * This fatal error can only occur if you are using automatic group management with {@link #subscribe(Collection)},
@ -952,7 +953,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* encountered (in which case it is thrown to the caller), or the passed timeout expires. * encountered (in which case it is thrown to the caller), or the passed timeout expires.
* <p> * <p>
* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)} * Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
* (or similar) are guaranteed to have their callbacks invoked prior to completion of this method. * (or similar) are guaranteed to have their callbacks invoked prior to completion of this method,
* but only when the consumer is using the consumer group protocol.
* *
* @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried. * @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried.
* This can only occur if you are using automatic group management with {@link #subscribe(Collection)}, * This can only occur if you are using automatic group management with {@link #subscribe(Collection)},
@ -1001,7 +1003,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller).
* <p> * <p>
* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)} * Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
* (or similar) are guaranteed to have their callbacks invoked prior to completion of this method. * (or similar) are guaranteed to have their callbacks invoked prior to completion of this method,
* but only when the consumer is using the consumer group protocol.
* *
* @param offsets A map of offsets by partition with associated metadata. This map will be copied internally, so it * @param offsets A map of offsets by partition with associated metadata. This map will be copied internally, so it
* is safe to mutate the map after returning. * is safe to mutate the map after returning.
@ -1053,7 +1056,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* encountered (in which case it is thrown to the caller), or the timeout expires. * encountered (in which case it is thrown to the caller), or the timeout expires.
* <p> * <p>
* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)} * Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
* (or similar) are guaranteed to have their callbacks invoked prior to completion of this method. * (or similar) are guaranteed to have their callbacks invoked prior to completion of this method,
* but only when the consumer is using the consumer group protocol.
* *
* @param offsets A map of offsets by partition with associated metadata. This map will be copied internally, so it * @param offsets A map of offsets by partition with associated metadata. This map will be copied internally, so it
* is safe to mutate the map after returning. * is safe to mutate the map after returning.