This is an automated email from the ASF dual-hosted git repository.

guozhang pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 5d32f24cc3 MINOR: Improve KafkaProducer Javadocs (#12537)
5d32f24cc3 is described below

commit 5d32f24cc3597760d3b846647e6a19dddc6e3d71
Author: Guozhang Wang <wangg...@gmail.com>
AuthorDate: Fri Aug 19 10:09:48 2022 -0700

    MINOR: Improve KafkaProducer Javadocs (#12537)
    
    While reviewing KIP-588 and KIP-691 I went through the exception throwing 
behavior and wanted to improve the related javadocs a little bit.
    
    Reviewers: John Roesler <vvcep...@apache.org>
---
 .../kafka/clients/producer/KafkaProducer.java      | 22 +++++++++++++++++++---
 1 file changed, 19 insertions(+), 3 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index 2d5c8994b4..ec8b8725c8 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -675,6 +675,11 @@ public class KafkaProducer<K, V> implements Producer<K, V> 
{
      * and should also not commit offsets manually (via {@link 
KafkaConsumer#commitSync(Map) sync} or
      * {@link KafkaConsumer#commitAsync(Map, OffsetCommitCallback) async} 
commits).
      *
+     * <p>
+     * This method is a blocking call that waits until the request has been 
received and acknowledged by the consumer group
+     * coordinator; but the offsets are not considered as committed until the 
transaction itself is successfully committed later (via
+     * the {@link #commitTransaction()} call).
+     *
      * @throws IllegalStateException if no transactional.id has been 
configured, no transaction has been started
      * @throws ProducerFencedException fatal error indicating another producer 
with the same transactional.id is active
      * @throws org.apache.kafka.common.errors.UnsupportedVersionException 
fatal error indicating the broker
@@ -685,6 +690,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
      *         transactional.id is not authorized, or the consumer group id is 
not authorized.
      * @throws org.apache.kafka.common.errors.InvalidProducerEpochException if 
the producer has attempted to produce with an old epoch
      *         to the partition leader. See the exception for more details
+     * @throws TimeoutException if the time taken for sending the offsets has 
surpassed <code>max.block.ms</code>.
      * @throws KafkaException if the producer has encountered a previous fatal 
or abortable error, or for any
      *         other unexpected error
      *
@@ -711,6 +717,11 @@ public class KafkaProducer<K, V> implements Producer<K, V> 
{
      * requires the brokers to be on version 2.5 or newer to understand.
      *
      * <p>
+     * This method is a blocking call that waits until the request has been 
received and acknowledged by the consumer group
+     * coordinator; but the offsets are not considered as committed until the 
transaction itself is successfully committed later (via
+     * the {@link #commitTransaction()} call).
+     *
+     * <p>
      * Note, that the consumer should have {@code enable.auto.commit=false} 
and should
      * also not commit offsets manually (via {@link 
KafkaConsumer#commitSync(Map) sync} or
      * {@link KafkaConsumer#commitAsync(Map, OffsetCommitCallback) async} 
commits).
@@ -735,7 +746,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
      *         to the partition leader. See the exception for more details
      * @throws KafkaException if the producer has encountered a previous fatal 
or abortable error, or for any
      *         other unexpected error
-     * @throws TimeoutException if the time taken for sending offsets has 
surpassed max.block.ms.
+     * @throws TimeoutException if the time taken for sending the offsets has 
surpassed <code>max.block.ms</code>.
      * @throws InterruptException if the thread is interrupted while blocked
      */
     public void sendOffsetsToTransaction(Map<TopicPartition, 
OffsetAndMetadata> offsets,
@@ -765,7 +776,10 @@ public class KafkaProducer<K, V> implements Producer<K, V> 
{
      * Note that exceptions thrown by callbacks are ignored; the producer 
proceeds to commit the transaction in any case.
      * <p>
      * Note that this method will raise {@link TimeoutException} if the 
transaction cannot be committed before expiration
-     * of {@code max.block.ms}. Additionally, it will raise {@link 
InterruptException} if interrupted.
+     * of {@code max.block.ms}, but this does not mean the request did not 
actually reach the broker. In fact, it only indicates
+     * that we cannot get the acknowledgement response in time, so it's up to 
the application's logic
+     * to decide how to handle time outs.
+     * Additionally, it will raise {@link InterruptException} if interrupted.
      * It is safe to retry in either case, but it is not possible to attempt a 
different operation (such as abortTransaction)
      * since the commit may already be in the progress of completing. If not 
retrying, the only option is to close the producer.
      *
@@ -798,7 +812,9 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
      * {@link ProducerFencedException} or an instance of {@link 
org.apache.kafka.common.errors.AuthorizationException}.
      *
      * Note that this method will raise {@link TimeoutException} if the 
transaction cannot be aborted before expiration
-     * of {@code max.block.ms}. Additionally, it will raise {@link 
InterruptException} if interrupted.
+     * of {@code max.block.ms}, but this does not mean the request did not 
actually reach the broker. In fact, it only indicates
+     * that we cannot get the acknowledgement response in time, so it's up to 
the application's logic
+     * to decide how to handle time outs. Additionally, it will raise {@link 
InterruptException} if interrupted.
      * It is safe to retry in either case, but it is not possible to attempt a 
different operation (such as commitTransaction)
      * since the abort may already be in the progress of completing. If not 
retrying, the only option is to close the producer.
      *

Reply via email to