Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

The "timer" for each batch starts "ticking" at the creation of the batch. Batches expire in order when max.in.flight.request.per.connection==1. An in-flight batch expire when delivery.timeout.ms has passed since the batch creation irrespective of whether the batch is in flight or not. However, the producer waits the full request.timeout.ms for the in-flight request. This implies that user might be notified of batch expiry while a batch is still in-flight.

Public Interfaces

  • Add a new producer configuration delivery.timeout.ms with default value 120 seconds.
  • Change the default value of retries to MAX_INT.
  • request.timeout.ms–no changes in the meaning, but messages are not expired after this time. I.e., request.timeout.ms is no longer relevant for batch expiry This only adds a new producer configuration: delivery.timeout.ms.

Compatibility, Deprecation, and Migration Plan

  • Default value of delivery.timeout.ms = 120 seconds
  • Default value of retries will be changed to MAX_INT.

    Setting an explicit value of retries should be done with caution. If all the retries are exhausted, the request will fail and all the contained batches within the request (even if delivery.timeout.ms has not fully elapsed). In essence, batch expires when either delivery.timeout.ms has elapsed or the request containing the batch has failed, whichever happens

    first. The

    first. (Note: Due to change in the default value of retries from 0 to MAX_INT and the existing default value of max.in.flight.request.per.connection

    is

    ==5

    and therefore

    , reordering

    is

    becomes a possibility by default. To prevent reordering, set max.in.flight.request.per.connection==1

    .request.timeout.ms – current meaning, but messages are not expired after this time. I.e., request.timeout.ms is no longer relevant for batch expiry

    ).

    Validation

    This configuration is backwards compatible. Throw ConfigException for timeouts that don't make sense. (E.g., delivery.timeout.ms < linger.ms + request.timeout.ms + retry.backoff.ms).

    ...