diff --git a/CHANGELOG.md b/CHANGELOG.md index 9d6405a879..95f875bf0c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,21 @@ +# librdkafka v2.1.2 + +librdkafka v2.1.2 is a maintenance release: + + * Fix a bug that happens when skipping tags, causing buffer underflow in + MetadataResponse (#4278). + +## Fixes + +### General fixes + + * Fix a bug that happens when skipping tags, causing buffer underflow in + MetadataResponse. This is triggered since RPC version 9 (v2.1.0), + when using Confluent Platform, only when racks are set, + observers are activated and there is more than one partition. + Fixed by skipping the correct amount of bytes when tags are received. + + # librdkafka v2.1.1 librdkafka v2.1.1 is a maintenance release: diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h index b4f606317b..737780578a 100644 --- a/src/rdkafka_buf.h +++ b/src/rdkafka_buf.h @@ -784,9 +784,8 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */ uint64_t _tagtype, _taglen; \ rd_kafka_buf_read_uvarint(rkbuf, &_tagtype); \ rd_kafka_buf_read_uvarint(rkbuf, &_taglen); \ - if (_taglen > 1) \ - rd_kafka_buf_skip(rkbuf, \ - (size_t)(_taglen - 1)); \ + if (_taglen > 0) \ + rd_kafka_buf_skip(rkbuf, (size_t)(_taglen)); \ } \ } while (0)