Skip to content

Commit 06b2fcb

Browse files
Handle integer overflow in rd_kafka_broker_buf_retry (#5157)
1 parent 48a7c7d commit 06b2fcb

File tree

1 file changed

+11
-3
lines changed

1 file changed

+11
-3
lines changed

src/rdkafka_broker.c

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -2966,10 +2966,18 @@ void rd_kafka_broker_buf_retry(rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) {
29662966
rd_atomic64_add(&rkb->rkb_c.tx_retries, 1);
29672967
/* In some cases, failed Produce requests do not increment the retry
29682968
* count, see rd_kafka_handle_Produce_error. */
2969-
if (rkbuf->rkbuf_retries > 0)
2970-
backoff = (1 << (rkbuf->rkbuf_retries - 1)) *
2969+
if (rkbuf->rkbuf_retries > 0) {
2970+
int shift = rkbuf->rkbuf_retries - 1;
2971+
2972+
/* Cap shift at 34 to prevent overflow in final calculation.
2973+
* Accounts for multiplication by retry_backoff_ms (max 300000)
2974+
* and jitter (max 1200). */
2975+
if (shift > 34)
2976+
shift = 34;
2977+
2978+
backoff = ((int64_t)1 << shift) *
29712979
(rkb->rkb_rk->rk_conf.retry_backoff_ms);
2972-
else
2980+
} else
29732981
backoff = rkb->rkb_rk->rk_conf.retry_backoff_ms;
29742982

29752983
/* We are multiplying by 10 as (backoff_ms * percent * 1000)/100 ->

0 commit comments

Comments
 (0)