From 26841c28a6613cefbd35ca82fef7e46335e6a197 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 25 Jul 2024 12:56:12 +0200 Subject: [PATCH 1/2] Failing test when configured with HAVE_STRNDUP --- tests/0099-commit_metadata.c | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/tests/0099-commit_metadata.c b/tests/0099-commit_metadata.c index 9acdb07f55..b5e639a1b7 100644 --- a/tests/0099-commit_metadata.c +++ b/tests/0099-commit_metadata.c @@ -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*/); @@ -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); From 96d1055f48ea4af41fc99cb06783463283727dab Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 25 Jul 2024 13:55:58 +0200 Subject: [PATCH 2/2] Closes #4649 --- CHANGELOG.md | 22 ++++++++++++++++++++++ src/rdkafka_request.c | 12 +++++++++--- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 68142d0d3c..707423c37c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -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. diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 8623be97d3..d3f57abdc0 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -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() */