diff --git a/CHANGELOG.md b/CHANGELOG.md index 8b90a090a2..2de9f83148 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,9 @@ librdkafka v2.5.0 is a feature release. +* Fix segfault when using long client id because of erased segment when using flexver. (#4689) + + ## Enhancements * Update bundled lz4 (used when `./configure --disable-lz4-ext`) to @@ -9,6 +12,17 @@ librdkafka v2.5.0 is a feature release. bugfixes and performance improvements (#4726). +## Fixes + +### General fixes + +* Issues: [confluentinc/confluent-kafka-dotnet#2084](https://github.com/confluentinc/confluent-kafka-dotnet/issues/2084) + Fix segfault when a segment is erased and more data is written to the buffer. + Happens since 1.x when a portion of the buffer (segment) is erased for flexver or compression. + More likely to happen since 2.1.0, because of the upgrades to flexver, with certain string sizes like a long client id (#4689). + + + # librdkafka v2.4.0 librdkafka v2.4.0 is a feature release: diff --git a/src/rdbuf.c b/src/rdbuf.c index 6df64a9dee..427d632eb7 100644 --- a/src/rdbuf.c +++ b/src/rdbuf.c @@ -660,13 +660,16 @@ size_t rd_buf_erase(rd_buf_t *rbuf, size_t absof, size_t size) { segremains); seg->seg_of -= toerase; + seg->seg_erased += toerase; rbuf->rbuf_len -= toerase; of += toerase; /* If segment is now empty, remove it */ - if (seg->seg_of == 0) + if (seg->seg_of == 0) { + rbuf->rbuf_erased -= seg->seg_erased; rd_buf_destroy_segment(rbuf, seg); + } } /* Update absolute offset of remaining segments */ @@ -709,6 +712,7 @@ int rd_buf_write_seek(rd_buf_t *rbuf, size_t absof) { next != seg;) { rd_segment_t *this = next; next = TAILQ_PREV(this, rd_segment_head, seg_link); + rbuf->rbuf_erased -= this->seg_erased; rd_buf_destroy_segment(rbuf, this); } diff --git a/src/rdbuf.h b/src/rdbuf.h index 90d61401b0..d8f98422cc 100644 --- a/src/rdbuf.h +++ b/src/rdbuf.h @@ -70,6 +70,8 @@ typedef struct rd_segment_s { * beginning in the grand rd_buf_t */ void (*seg_free)(void *p); /**< Optional free function for seg_p */ int seg_flags; /**< Segment flags */ + size_t seg_erased; /** Total number of bytes erased from + * this segment. */ #define RD_SEGMENT_F_RDONLY 0x1 /**< Read-only segment */ #define RD_SEGMENT_F_FREE \ 0x2 /**< Free segment on destroy, \ diff --git a/tests/0011-produce_batch.c b/tests/0011-produce_batch.c index 1507d76f9e..f745a6d310 100644 --- a/tests/0011-produce_batch.c +++ b/tests/0011-produce_batch.c @@ -90,13 +90,21 @@ static void test_single_partition(void) { int failcnt = 0; int i; rd_kafka_message_t *rkmessages; - + char client_id[271]; SUB_TEST_QUICK(); msgid_next = 0; test_conf_init(&conf, &topic_conf, 20); + /* A long client id must not cause a segmentation fault + * because of an erased segment when using flexver. + * See: + * https://github.com/confluentinc/confluent-kafka-dotnet/issues/2084 */ + memset(client_id, 'c', sizeof(client_id) - 1); + client_id[sizeof(client_id) - 1] = '\0'; + rd_kafka_conf_set(conf, "client.id", client_id, NULL, 0); + /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_single_partition_cb);