Skip to content

Commit

Permalink
Change retry.backoff.* to apply to both producer and consumer (#4561)
Browse files Browse the repository at this point in the history
* Change retry.backoff.* to apply to both producer and consumer

* Style fixes
  • Loading branch information
milindl authored May 8, 2024
1 parent a4bf487 commit 2587cac
Show file tree
Hide file tree
Showing 2 changed files with 4 additions and 5 deletions.
4 changes: 2 additions & 2 deletions CONFIGURATION.md
Original file line number Diff line number Diff line change
Expand Up @@ -144,8 +144,8 @@ queue.buffering.max.ms | P | 0 .. 900000 | 5
linger.ms | P | 0 .. 900000 | 5 | high | Alias for `queue.buffering.max.ms`: Delay in milliseconds to wait for messages in the producer queue to accumulate before constructing message batches (MessageSets) to transmit to brokers. A higher value allows larger and more effective (less overhead, improved compression) batches of messages to accumulate at the expense of increased message delivery latency. <br>*Type: float*
message.send.max.retries | P | 0 .. 2147483647 | 2147483647 | high | How many times to retry sending a failing Message. **Note:** retrying may cause reordering unless `enable.idempotence` is set to true. <br>*Type: integer*
retries | P | 0 .. 2147483647 | 2147483647 | high | Alias for `message.send.max.retries`: How many times to retry sending a failing Message. **Note:** retrying may cause reordering unless `enable.idempotence` is set to true. <br>*Type: integer*
retry.backoff.ms | P | 1 .. 300000 | 100 | medium | The backoff time in milliseconds before retrying a protocol request, this is the first backoff time, and will be backed off exponentially until number of retries is exhausted, and it's capped by retry.backoff.max.ms. <br>*Type: integer*
retry.backoff.max.ms | P | 1 .. 300000 | 1000 | medium | The max backoff time in milliseconds before retrying a protocol request, this is the atmost backoff allowed for exponentially backed off requests. <br>*Type: integer*
retry.backoff.ms | * | 1 .. 300000 | 100 | medium | The backoff time in milliseconds before retrying a protocol request, this is the first backoff time, and will be backed off exponentially until number of retries is exhausted, and it's capped by retry.backoff.max.ms. <br>*Type: integer*
retry.backoff.max.ms | * | 1 .. 300000 | 1000 | medium | The max backoff time in milliseconds before retrying a protocol request, this is the atmost backoff allowed for exponentially backed off requests. <br>*Type: integer*
queue.buffering.backpressure.threshold | P | 1 .. 1000000 | 1 | low | The threshold of outstanding not yet transmitted broker requests needed to backpressure the producer's message accumulator. If the number of not yet transmitted requests equals or exceeds this number, produce request creation that would have otherwise been triggered (for example, in accordance with linger.ms) will be delayed. A lower number yields larger and more effective batches. A higher value can improve latency when using compression on slow machines. <br>*Type: integer*
compression.codec | P | none, gzip, snappy, lz4, zstd | none | medium | compression codec to use for compressing message sets. This is the default value for all topics, may be overridden by the topic configuration property `compression.codec`. <br>*Type: enum value*
compression.type | P | none, gzip, snappy, lz4, zstd | none | medium | Alias for `compression.codec`: compression codec to use for compressing message sets. This is the default value for all topics, may be overridden by the topic configuration property `compression.codec`. <br>*Type: enum value*
Expand Down
5 changes: 2 additions & 3 deletions src/rdkafka_conf.c
Original file line number Diff line number Diff line change
Expand Up @@ -1393,15 +1393,14 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
{_RK_GLOBAL | _RK_PRODUCER, "retries", _RK_C_ALIAS,
.sdef = "message.send.max.retries"},

{_RK_GLOBAL | _RK_PRODUCER | _RK_MED, "retry.backoff.ms", _RK_C_INT,
_RK(retry_backoff_ms),
{_RK_GLOBAL | _RK_MED, "retry.backoff.ms", _RK_C_INT, _RK(retry_backoff_ms),
"The backoff time in milliseconds before retrying a protocol request, "
"this is the first backoff time, "
"and will be backed off exponentially until number of retries is "
"exhausted, and it's capped by retry.backoff.max.ms.",
1, 300 * 1000, 100},

{_RK_GLOBAL | _RK_PRODUCER | _RK_MED, "retry.backoff.max.ms", _RK_C_INT,
{_RK_GLOBAL | _RK_MED, "retry.backoff.max.ms", _RK_C_INT,
_RK(retry_backoff_max_ms),
"The max backoff time in milliseconds before retrying a protocol request, "
"this is the atmost backoff allowed for exponentially backed off "
Expand Down

0 comments on commit 2587cac

Please sign in to comment.