Versions Compared

Key

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

...

There is an existing configuration TIMEOUT_CONFIG, which is really a server side timeout specifying the maximum amount of time brokers can wait before send back the response. In order to avoid future confusion, for KafkaProducer, we might also want to change TIMEOUT_CONFIG to REPLICATION_TIMEOUT_CONFIG.

...

Discussion on end state of timeouts in producer

...

We want to see if we can reduce the configurations exposed to users so we have less confusion meanwhile still have the flexibility to support different use cases.

The timeout might be needed in the following three scenarios. So we will have at least three timeout configurations.

A. when user calls producer.send()

B. batches in accumulator

C. after request is sent

Currently we might have the following timeout in producer configurations with scenario in the parenthesis.

  1. (A) metadata.fetch.timeout.ms - The maximum time producer will wait for metadata to be ready in producer.send()
  2. (B)
  3. max.send.block.ms - This timeout defines how long the producer.send() method can block (we will enforce this for both metadata and blocking on buffer full in a separate ticket)
  4. linger.ms - The maximum time the producer will buffer the records for batch before sending it out.
  5. (C) replication.timeout.ms - This is a server side configuration. It defines how long a server will wait for the records to be replicated to followers.
  6. (C) network.request.timeout.ms - This timeout is used when producer sends request to brokers through TCP connections. It specifies how long the producer should wait for the response.

From what I can see, network.request.timeout.ms and linger.ms have clear purposes and we probably want to keep them.

replication.timeout.ms is a legacy configuration, we might be able to get it away but let's keep it for now.

For (B) we need another timeout to expire the batches in accumulator in addition to batching purpose. That means we probably need the following configuration.

      5. (B) batch.timeout.ms - In some cases, the leader broker information of a batch could be unavailable after the batch is ready to be sent (i.e. batch is full or reached linger.ms). This configuration define the maximum time the producer will wait for the necessary broker information before it fail the batches

...

.

For (A) it makes sense to have a dedicated timeout for send() method to provide blocking bound. So it gives another configuration.

      6. (A) max.send.block

...

.ms - This timeout

...

defines how long the producer

...

.send() method can block (we will enforce this for both metadata and blocking on buffer full in a separate ticket).

With above, we have the following proposals and want to see which one makes more sense:

Proposal 1:

  • (A/B) metadata.fetch.timeout.ms - reuse metadata timeout as batch.timeout.ms because it is essentially metadata not available.
  • (B) linger.ms
  • (C) replication.timeout.ms
  • (C) network.request.timeout.ms

Proposal 2:

  1. (A) max.send.block.ms
  2. (B) linger.ms
  3. (B) batch.timeout.ms
  4. (C) replication.timeout.ms
  5. (C) network.request.timeout.ms

Proposal 1 has four configurations but does got guarantee of blocking time for send() when blocking on buffer full is turned on. So user needs to catch the exception in outside send and retry if buffer is full.

Proposal 2 has five configurations but guarantee the blocking time of send()

...

.

Proposed Changes

Because the NetworkClient works in an async way. We need to keep track of the send time for each request. So the implementation will the following:

...