Skip to content

Commit

Permalink
Fix segfault when a fetch from follower
Browse files Browse the repository at this point in the history
lease expires and the partition is waiting for
a list offsets result

closes #4195
  • Loading branch information
emasab committed Apr 19, 2023
1 parent 2013f79 commit f7a9335
Show file tree
Hide file tree
Showing 2 changed files with 21 additions and 3 deletions.
21 changes: 21 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,3 +1,24 @@
# librdkafka v2.1.1

librdkafka v2.1.1 is a bugfix release:

* Fix a segmentation fault when fetching from follower and the partition lease
expires while waiting for the result of a list offsets operation (#4254).


## Fixes

### Consumer fixes

* When fetching from follower, if the partition lease expires after 5 minutes,
and a list offsets operation was requested to retrieve the earliest
or latest offset, it resulted in segmentation fault. This was fixed by
allowing threads different from the main one to call
the `rd_kafka_toppar_set_fetch_state` function, given they hold
the lock on the `rktp`.



# librdkafka v2.1.0

librdkafka v2.1.0 is a feature release:
Expand Down
3 changes: 0 additions & 3 deletions src/rdkafka_partition.c
Original file line number Diff line number Diff line change
Expand Up @@ -359,9 +359,6 @@ void rd_kafka_toppar_destroy_final(rd_kafka_toppar_t *rktp) {
* @locks_required rd_kafka_toppar_lock() MUST be held.
*/
void rd_kafka_toppar_set_fetch_state(rd_kafka_toppar_t *rktp, int fetch_state) {
rd_kafka_assert(NULL,
thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread));

if ((int)rktp->rktp_fetch_state == fetch_state)
return;

Expand Down

0 comments on commit f7a9335

Please sign in to comment.