Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-9800

[KIP-580] Client Exponential Backoff Implementation

    XMLWordPrintableJSON

Details

    • New Feature
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.7.0
    • None

    Description

      Design:

      The main idea is to bookkeep the failed attempt. Currently, the retry backoff has two main usage patterns:

      1. Synchronous retires and blocking loop. The thread will sleep in each iteration for retry backoff ms.
      2. Async retries. In each polling, the retries do not meet the backoff will be filtered. The data class often maintains a 1:1 mapping to a set of requests which are logically associated. (i.e. a set contains only one initial request and only its retries.)

      For type 1, we can utilize a local failure counter of a Java generic data type.

      For case 2, I already wrapped the exponential backoff/timeout util class in my KIP-601 implementation which takes the number of attempts and returns the backoff/timeout value at the corresponding level. Thus, we can add a new class property to those classes containing retriable data in order to record the number of failed attempts.

       

      Changes:

      KafkaProducer:

      1. Produce request (ApiKeys.PRODUCE). Currently, the backoff applies to each ProducerBatch in Accumulator, which already has an attribute attempts recording the number of failed attempts. So we can let the Accumulator calculate the new retry backoff for each bach when it enqueues them, to avoid instantiate the util class multiple times.
      2. Transaction request (ApiKeys..*TXN). TxnRequestHandler will have a new class property of type `Long` to record the number of attempts.

      KafkaConsumer:

      1. Some synchronous retry use cases. Record the failed attempts in the blocking loop.
      2. Partition request (ApiKeys.OFFSET_FOR_LEADER_EPOCH, ApiKeys.LIST_OFFSETS). Though the actual requests are packed for each node, the current implementation is applying backoff to each topic partition, where the backoff value is kept by TopicPartitionState. Thus, TopicPartitionState will have the new property recording the number of attempts.

      Metadata:

      1.  Metadata lives as a singleton in many clients. Add a new property recording the number of attempts

       AdminClient:

      1. AdminClient has its own request abstraction Call. The failed attempts are already kept by the abstraction. So probably clean the Call class logic a bit.

      Existing tests:

      1. If the tests are testing the retry backoff, add a delta to the assertion, considering the existence of the jitter.
      2. If the tests are testing other functionality, we can specify the same value for both `retry.backoff.ms` and `retry.backoff.max.ms` in order to make the retry backoff static. We can use this trick to make the existing tests compatible with the changes.

      There're other common usages look like client.poll(timeout), where the timeout passed in is the retry backoff value. We won't change these usages since its underlying logic is nioSelector.select(timeout) and nioSelector.selectNow(), which means if no interested op exists, the client will block retry backoff milliseconds. This is an optimization when there's no request that needs to be sent but the client is waiting for responses. Specifically, if the client fails the inflight requests before the retry backoff milliseconds passed, it still needs to wait until that amount of time passed, unless there's a new request need to be sent.

       

      Attachments

        Issue Links

          Activity

            People

              schofielaj Andrew Schofield
              d8tltanc Cheng Tan
              Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: