Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Add max.block.ms to KIP

Table of Contents

Status

Current stateAccepted

Discussion thread: here

...

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

In KAFKA-4879, we first noticed that KafkaConsumer will hang forever during a position() call. However, after looking closer, we have found that several KafkaConsumer methods will continue to block indefinitely unless the offsets are retrieved for the provided TopicPartition.  To avoid this scenario from occurring:

  1. A complementary method will be added for each method that blocks indefinitely, but with an extra Duration parameter timeout, which bounds the amount of time spent in the method.
  2. A TimeoutException will be thrown once the amount of time spent exceeds timeout.

Public Interfaces

 

Unless otherwise specified, the pre-existing variants for these new interfaces will be marked @Deprecated.

Consumer#Position

A TimeoutException will be thrown when the time spent exceeds timeout:

KafkaConsumer has a long history of indefinite blocking behavior which has been a continual cause of user frustration. This KIP aims to fix this problem in two ways:

  1. We introduce a new configuration max.block.ms to control the maximum time that current blocking APIs will await before raising a timeout error.
  2. We add overloaded APIs to allow for custom timeout control. This is intended for advanced usage such as in Kafka Streams

Some care must be taken in the case of the poll() since many applications depend on the current behavior of blocking until an assignment is found. This KIP retains this behavior for poll(long), but introduces a new API poll(Duration), which always returns when the timeout expires. We will deprecate poll(long) and remove it in a later major release.


Public Interfaces

This KIP adds max.block.ms as a new configuration for the consumer. The default value of max.block.ms will be one minute, which matches the default for the same config in the Producer. 

Below we document the APIs that this timeout will impact and how the behavior changes.

The following APIs currently block indefinitely until either the operation completes successfully or an unrecoverable error is encountered. Following this KIP, these methods will now raise org.apache.kafka.common.errors.TimeoutException if neither of these conditions have been reached prior to expiration of the time specified by max.block.ms.


Code Block
void commitSync();


void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets);


long position(TopicPartition partition);


OffsetAndMetadata committed(TopicPartition partition);


The following APIs currently block for at most the time configured by request.timeout.ms until the operation completed successfully or an unrecoverable error is encountered. Following this KIP, they will instead use the timeout indicated by max.block.ms. As before, when the timeout is reached, org.apache.kafka.common.errors.TimeoutException will be raised to the user.

Code Block
List<PartitionInfo> partitionsFor(String topic);

Map<String, List<PartitionInfo>> listTopics();

Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartition, Long> timestampsToSearch);

Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions);

Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions);


As mentioned above, this KIP also adds overloaded APIs to allow for custom timeout control. The new APIs are documented below: 

Consumer#position

A TimeoutException will be thrown when the time spent exceeds timeout:

Code Block
languagejava
titleKafkaConsumer#position(TopicPartition topicPartition))
    /**
     * Get the offset of the <i>next record</i> that will be fetched (if a record with that offset exists).
     * This method may issue a remote call to the server if there is no current position for the given partition.
     * <p>
     * This call will block until either the position could be determined or an unrecoverable error is
     * encountered (in which case it is thrown to the caller).
     *
     * @param partition The partition to get the position for
+    * @param timeout   The maximum duration of the method
     *
     * @return The current position of the consumer (that is, the offset of the next record to be fetched)
     * @throws IllegalArgumentException if the provided TopicPartition is not assigned to this consumer
     * @throws org.apache.kafka.clients.consumer.InvalidOffsetException if no offset is currently defined for
     *             the partition
     * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
     *             function is called
     * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
     *             this function is called
+    * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
     * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
     * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the
     *             configured groupId. See the exception for more details
     * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
     */
+   long position(TopicPartition partition, Duration timeout);

Consumer#committed and Consumer#commitSync

Similarily, this will also be applied to other methods in KafkaConsumer that blocks indefinitely.

Code Block
languagejava
themeEclipse
titleKafkaConsumer#blocking methods
    /**
     * Get the last committed offset for the given partition (whether the commit happened by this process or
     * another). This offset will be used as the position for the consumer in the event of a failure.
     * <p>
     * This call will block to do a remote call to get the latest committed offsets from the server.
     *
     * @param partition The partition to check
+    * @param timeout   The maximum duration of the method
     *
     * @return The last committed offset and metadata or null if there was no prior commit
     * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
Code Block
languagejava
titleKafkaConsumer#position(TopicPartition topicPartition))
    /**
     * Get the offset of the <i>next record</i> that will be fetched (if a record with that offset exists).
     * This method may issue a remote call to the server if there is no current position for the given partition.
     * <p>
     * This call will block until either the position could be determined or anfunction unrecoverableis error iscalled
     * encountered (in which case it@throws org.apache.kafka.common.errors.InterruptException if the calling thread is throwninterrupted tobefore theor caller).while
     *
       * @param partition The partition to getthis thefunction positionis forcalled
+     * @param timeout   The maximum duration of the method @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
     *
 @throws org.apache.kafka.common.errors.AuthorizationException if  *not @returnauthorized The current position of to the consumertopic (thator is,to the
 offset of the next record* to be fetched)
     * @throws IllegalArgumentException if the providedconfigured TopicPartitiongroupId. isSee notthe assignedexception tofor thismore consumerdetails
     * @throws org.apache.kafka.clientscommon.consumer.InvalidOffsetExceptionKafkaException iffor noany offsetother is currently defined for
 unrecoverable errors
+    * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the     the partition{@code timeout}
     * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this/
    OffsetAndMetadata committed(TopicPartition partition, final Duration timeout)'

	/**
     * Commit the specified offsets for the specified list of topics and partitions.
  function   is* called<p>
     * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
     *     This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
     * rebalance and thisalso functionon is called
+    * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}startup. As such, if you need to store offsets in anything other than Kafka, this API
     * should @throwsnot be org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more detailsused. The committed offset should be the next message your application will consume,
     * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to thei.e. lastProcessedMessageOffset + 1.
     * <p>
     * This is a synchronous commits and will block until either the commit  configured groupId. See the exception for more details
     * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errorssucceeds or an unrecoverable error is
     * encountered (in which case it is thrown to the caller).
     * <p>
     */
+ Note that longasynchronous position(TopicPartition partition, Duration timeout);

Consumer#committed and Consumer#commitSync

Similarily, this will also be applied to other methods in KafkaConsumer that blocks indefinitely.

Code Block
languagejava
themeEclipse
titleKafkaConsumer#blocking methods
    /**offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
     * Get the last committed offset for the given partition (whether the commit happened by this process or (or similar) are guaranteed to have their callbacks invoked prior to completion of this method.
     *
     * another). This offset will be used as the position for the consumer in the event of a failure. @param offsets  A map of offsets by partition with associated metadata
+    * @param timeout  Maximum duration to block
     * <p>
     * This call will block to do a remote call to get the latest committed offsets from the server.
 @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried.
     *          *
   This can *only @paramoccur partitionif Theyou partitionare tousing check
+automatic group management with * @param timeout{@link #subscribe(Collection)},
    The maximum* duration of the method
     *
    or *if @returnthere Theis lastan committedactive offsetgroup andwith metadatathe orsame nullgroupId ifwhich thereis wasusing no prior commitgroup management.
     * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
     *             function is called
     * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
     *             this function is called
+.   * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the this function is called{@code timeout}
      * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
     * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the
     *             configured groupId. See the exception for more details details
     * @throws java.lang.IllegalArgumentException if the committed offset is negative
     * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors (e.g. if offset metadata
     *  @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
+    * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}is too large or if the topic does not exist).
     */
    OffsetAndMetadatavoid committed(TopicPartition partitioncommitSync(final Map<TopicPartition, OffsetAndMetadata> offsets, final Duration timeout)';


	    /**
     * Commit the specified offsets foroffsets returned on the last {@link #poll(Duration) poll()} for all the specifiedsubscribed list of topics and
     * partitions.
     * <p>
     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after every
     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
     * should not be used. The committed offset should be the next message your application will consume,
     * i.e. lastProcessedMessageOffset + 1.
     * <p>
     * This is a synchronous commits and will block until either the commit succeeds, or an unrecoverable error is
     * encountered (in which case it is thrown to the caller)), or the passed timeout expires.
     * <p>
     * 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.
     *
     * @param offsets  A map of offsets by partition with associated metadata
+    * @param timeout  Maximum duration to block
     *
     * @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)},
     *             or if there is an active group with the same groupId which is using group management.
     * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
     *             function is called
     * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
     *             this function is called
+.   * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
     * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
     * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the
     *             configured groupId. See the exception for more details
     * @throwsconfigured java.langgroupId.IllegalArgumentException ifSee the committedexception offsetfor ismore negativedetails
     * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors (e.g. if offset metadata
     *             is too large or if the topic does not exist). not exist).
+    * @throws org.apache.kafka.common.errors.TimeoutException if the timeout expires before successful completion
     *            of the offset commit
     */
    void@Override
 commitSync(final Map<TopicPartition, OffsetAndMetadata> offsets,public finalvoid commitSync(Duration timeout);

Currently, commitSync does not accept a user-provided timeout, but by default, will block indefinitely by setting wait time to Long.MAX_VALUE. To accomadate for a potential hanging block,

the new KafkaConsumer#commitSync will accept user-specified timeout.  

Consumer#poll

The pre-existing variant poll(long timeout) would block indefinitely for metadata updates if they were needed, then it would issue a fetch and poll for timeout ms for new records. The initial indefinite metadata block caused applications to become stuck when the brokers became unavailable. The existence of the timeout parameter made the indefinite block especially unintuitive.

...

We will mark the existing poll() method as deprecated.

Consumer#partitionsFor

Code Block
/**
 * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it
 * does not already have any metadata about the given topic.
 *
 * @param topic The topic to get partition metadata for
 * @param timeout The maximum time this operation will block
 *
 * @return The list of partitions
 * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
 * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
 *             function is called
 * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
 *             this function is called
 * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
 * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the specified topic. See the exception for more details
 * @throws org.apache.kafka.common.errors.TimeoutException if the topic metadata could not be fetched before
 *             expiration of the configured request timeout
 * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
 */
List<PartitionInfo> partitionsFor(String topic, Duration timeout);

Consumer#listTopics

Code Block
/**
 * Get metadata about partitions for all topics that the user is authorized to view. This method will issue a
 * remote call to the server.
 *
 * @param timeout The maximum time this operation will block
 *
 * @return The map of topics and its partitions
 * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
 * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
 *             function is called
 * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while
 *             this function is called
 * @throws org.apache.kafka.common.errors.TimeoutException if the topic metadata could not be fetched before
 *             expiration of the configured request timeout
 * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
 */
Map<String, List<PartitionInfo>> listTopics(Duration timeout)

 

Consumer#offsetsForTimes

Code Block
/**
 * Look up the offsets for the given partitions by timestamp. The returned offset for each partition is the
 * earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding partition.
 *
 * This is a blocking call. The consumer does not have to be assigned the partitions.
 * If the message format version in a partition is before 0.10.0, i.e. the messages do not have timestamps, null
 * will be returned for that partition.
 *
 * @param timestampsToSearch the mapping from partition to the timestamp to look up.
 * @param timeout The maximum time this operation will block
 *
 * @return a mapping from partition to the timestamp and offset of the first message with timestamp greater
 *         than or equal to the target timestamp. {@code null} will be returned for the partition if there is no
 *         such message.
 * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
 * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
 * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details
 * @throws IllegalArgumentException if the target timestamp is negative
 * @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before
 *         expiration of the configured {@code request.timeout.ms}
 * @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not support looking up
 *         the offsets by timestamp
 */
Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartition, Long> timestampsToSearch, Duration timeout)

 

Consumer#beginningOffsets

Code Block
/**
 * Get the first offset for the given partitions.
 * <p>
 * This method does not change the current consumer position of the partitions.
 *
 * @see #seekToBeginning(Collection)
 *
 * @param partitions the partitions to get the earliest offsets.
 * @param timeout The maximum time this operation will block
 *
 * @return The earliest available offsets for the given partitions
 * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
 * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
 * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details
 * @throws org.apache.kafka.common.errors.TimeoutException if the offsets could not be fetched before
 *         expiration of the configured {@code request.timeout.ms}
 */
Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions, Duration timeout)

 

Consumer#endOffsets

Code Block
/**
 * Get the end offsets for the given partitions. In the default {@code read_uncommitted} isolation level, the end
 * offset is the high watermark (that is, the offset of the last successfully replicated message plus one). For
 * {@code read_committed} consumers, the end offset is the last stable offset (LSO), which is the minimum of
 * the high watermark and the smallest offset of any open transaction. Finally, if the partition has never been
 * written to, the end offset is 0.
 *
 * <p>
 * This method does not change the current consumer position of the partitions.
 *
 * @see #seekToEnd(Collection)
 *
 * @param partitions the partitions to get the end offsets.
 * @param timeout The maximum time this operation will block
 *
 * @return The end offsets for the given partitions.
 * @throws org.apache.kafka.common.errors.TimeoutException if time spent blocking exceeds the {@code timeout}
 * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details
 * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details
 * @throws org.apache.kafka.common.errors.TimeoutException if the offsets could not be fetched before
 *         expiration of the configured {@code request.timeout.ms}
 */
Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions, Duration timeout)

 

Consumer#close

Notes:

  • close() already is a variant with no parameters and applies a default from config. However, this variant will not be deprecated because it called by the Closeable interface.
  • close(long, TimeUnit) also exists as a variant. This one will be deprecated in favor of the new close(Duration) variant for consistency
  • The existing semantics of close is not to throw a TimeoutException. Instead, after waiting for the timeout, it forces itself closed.

...

Code Block
/**
 * Tries to close the consumer cleanly within the specified timeout. This method waits up to
 * {@code timeout} for the consumer to complete pending commits and leave the group.
 * If auto-commit is enabled, this will commit the current offsets if possible within the
 * timeout. If the consumer is unable to complete offset commits and gracefully leave the group
 * before the timeout expires, the consumer is force closed. Note that {@link #wakeup()} cannot be
 * used to interrupt close.
 *
 * @param timeout The maximum time to wait for consumer to close gracefully. The value must be
 *                non-negative. Specifying a timeout of zero means do not wait for pending requests to complete.
 *
 * @throws IllegalArgumentException If the {@code timeout} is negative.
 * @throws InterruptException If the thread is interrupted before or while this function is called
 * @throws org.apache.kafka.common.KafkaException for any other error during close
 */
public void close(Duration timeout)

 

Compatibility, Deprecation, and Migration Plan

Since old methods will not be modified, preexisting data frameworks will not be affected. However, some of these methods will be deprecated in favor of methods which are bound by a specific time limit.

The introduction of max.block.ms causes a slight change of behavior since some of the blocking APIs will now raise TimeoutException rather than their current blocking behavior. The change is compatible with the current API since TimeoutException is a KafkaException. Additionally, since TimeoutException is retriable, any existing retry logic will work as expected. 

Feasible and Rejected Alternatives

Please see KIP-288 for other rejected alternatives.

...