Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
  • Loading branch information
emasab committed Jul 25, 2024
1 parent 26841c2 commit 96d1055
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 3 deletions.
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

0 comments on commit 96d1055

Please sign in to comment.