Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix reading metadata with zeros #4795

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
22 changes: 22 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,3 +1,25 @@
# librdkafka v2.5.1 (can vary)

librdkafka v2.5.1 is a maintenance release.

* Fix for retrieving offset commit metadata when it contains
zeros and configured with `strndup` (#4795)


## Fixes

### Consumer fixes

* Issues: #4649.
When retrieving offset metadata, if the binary value contained zeros
and librdkafka was configured with `strndup`, part of
the buffer after first zero contained uninitialized data
instead of rest of metadata. Solved by avoiding to use
`strndup` for copying metadata.
Happens since: 0.9.0 (#4795).



# librdkafka v2.5.0

librdkafka v2.5.0 is a feature release.
Expand Down
12 changes: 9 additions & 3 deletions src/rdkafka_request.c
Original file line number Diff line number Diff line change
Expand Up @@ -1289,9 +1289,15 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk,
rktpar->metadata = NULL;
rktpar->metadata_size = 0;
} else {
rktpar->metadata = RD_KAFKAP_STR_DUP(&metadata);
rktpar->metadata_size =
RD_KAFKAP_STR_LEN(&metadata);
/* It cannot use strndup because
* it stops at first 0 occurrence. */
size_t len = RD_KAFKAP_STR_LEN(&metadata);
rktpar->metadata_size = len;
unsigned char *metadata_bytes =
rd_malloc(len + 1);
rktpar->metadata = metadata_bytes;
memcpy(rktpar->metadata, metadata.str, len);
metadata_bytes[len] = '\0';
}

/* Loose ref from get_toppar() */
Expand Down
13 changes: 9 additions & 4 deletions tests/0099-commit_metadata.c
Original file line number Diff line number Diff line change
Expand Up @@ -158,6 +158,7 @@ int main_0099_commit_metadata(int argc, char **argv) {
rd_kafka_topic_partition_list_t *expected_toppar;
const char *topic = test_mk_topic_name("0099-commit_metadata", 0);
char group_id[16];
char *metadata;

test_conf_init(NULL, NULL, 20 /*timeout*/);

Expand All @@ -171,10 +172,14 @@ int main_0099_commit_metadata(int argc, char **argv) {

expected_toppar = rd_kafka_topic_partition_list_copy(origin_toppar);

expected_toppar->elems[0].offset = 42;
expected_toppar->elems[0].metadata = rd_strdup("Hello world!");
expected_toppar->elems[0].metadata_size =
strlen(expected_toppar->elems[0].metadata);
metadata = rd_strdup("Hello world!");
expected_toppar->elems[0].offset = 42;
expected_toppar->elems[0].metadata = metadata;
expected_toppar->elems[0].metadata_size = strlen(metadata);
/* Make sure it's interpreted as bytes.
* To fail before the fix it needs to be configured
* with HAVE_STRNDUP */
metadata[5] = '\0';

get_committed_metadata(group_id, origin_toppar, origin_toppar);

Expand Down