Skip to content

Commit

Permalink
Fix number or bytes skipped when receiving tags.
Browse files Browse the repository at this point in the history
Not being for a nullable field those unsigned varints
don't start from 1

fixes #4253
  • Loading branch information
emasab committed May 11, 2023
1 parent c282ba2 commit 9071963
Show file tree
Hide file tree
Showing 2 changed files with 20 additions and 3 deletions.
18 changes: 18 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -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:
Expand Down
5 changes: 2 additions & 3 deletions src/rdkafka_buf.h
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down

0 comments on commit 9071963

Please sign in to comment.