Skip to content

Commit 6f5f7a3

Browse files
piochelepiotrclaude
andcommitted
admin: fix use-after-free in coord_request error path
When rd_kafka_admin_coord_request()'s request() call fails, the error path called rd_kafka_admin_common_worker_destroy() which freed the eonce object. However, the caller (rd_kafka_coord_req_fsm) still holds a reference to the eonce and passes it to rd_kafka_coord_req_fail(), which enqueues a dummy error response carrying the (now-freed) eonce as opaque. When rd_kafka_admin_coord_response_parse() later processes that response and calls rd_kafka_enq_once_del_source_return(), it accesses freed memory, triggering an assertion failure and abort: rd_kafka_enq_once_del_source_return: Assertion `eonce->refcnt > 0' Fix by not calling worker_destroy() in the error path of rd_kafka_admin_coord_request(). Instead, let the error propagate through the normal coord_req_fail -> coord_response_parse path, which already handles cleanup correctly. This affects all coordinator-targeted Admin API operations: DescribeConsumerGroups, DeleteConsumerGroupOffsets, ListConsumerGroupOffsets, and similar. Fixes #4605 Fixes #3663 Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
1 parent f1b831e commit 6f5f7a3

File tree

2 files changed

+27
-6
lines changed

2 files changed

+27
-6
lines changed

CHANGELOG.md

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,19 @@
1+
# librdkafka v2.14.1 (unreleased)
2+
3+
## Fixes
4+
5+
### Admin client fixes
6+
7+
* Fix use-after-free in `rd_kafka_admin_coord_request()` error path
8+
that caused an assertion failure
9+
(`rd_kafka_enq_once_del_source_return: Assertion 'eonce->refcnt > 0' failed`)
10+
and process abort when a coordinator-targeted Admin API request
11+
(e.g., DescribeConsumerGroups, DeleteConsumerGroupOffsets,
12+
ListConsumerGroupOffsets) failed to send.
13+
The error path prematurely destroyed the eonce while the caller
14+
(coord_req_fsm) still held a reference to it (#4605, #3663).
15+
16+
117
# librdkafka v2.14.0
218

319
librdkafka v2.14.0 is a feature release:

src/rdkafka_admin.c

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -480,7 +480,6 @@ rd_kafka_admin_coord_request(rd_kafka_broker_t *rkb,
480480
rd_kafka_replyq_t replyq,
481481
rd_kafka_resp_cb_t *resp_cb,
482482
void *opaque) {
483-
rd_kafka_t *rk = rkb->rkb_rk;
484483
rd_kafka_enq_once_t *eonce = opaque;
485484
rd_kafka_op_t *rko;
486485
char errstr[512];
@@ -500,11 +499,17 @@ rd_kafka_admin_coord_request(rd_kafka_broker_t *rkb,
500499
rd_kafka_admin_handle_response, eonce);
501500

502501
if (err) {
503-
rd_kafka_admin_result_fail(
504-
rko, err, "%s worker failed to send request: %s",
505-
rd_kafka_op2str(rko->rko_type), errstr);
506-
rd_kafka_admin_common_worker_destroy(rk, rko,
507-
rd_true /*destroy*/);
502+
/* Do not call worker_destroy() here. The caller
503+
* (rd_kafka_coord_req_fsm) will call coord_req_fail()
504+
* which enqueues a dummy error response carrying the eonce
505+
* as opaque. coord_response_parse() will then pick up the
506+
* rko via del_source_return("coordinator response"),
507+
* see the error, and call worker_destroy() itself.
508+
*
509+
* Calling worker_destroy() here would free the eonce while
510+
* the caller still holds a reference to it, leading to a
511+
* use-after-free / assertion failure in
512+
* rd_kafka_enq_once_del_source_return(). */
508513
}
509514
return err;
510515
}

0 commit comments

Comments
 (0)