diff --git a/CHANGELOG.md b/CHANGELOG.md index 0fe274f4a..c6e1463e1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,6 +5,8 @@ librdkafka v2.6.1 is a maintenance release: * Fix for a Fetch regression when connecting to Apache Kafka < 2.7 (#4871). * Fix for an infinite loop happening with cooperative-sticky assignor under some particular conditions (#4800). +* Fix for retrieving offset commit metadata when it contains + zeros and configured with `strndup` (#4876) ## Fixes @@ -23,6 +25,13 @@ librdkafka v2.6.1 is a maintenance release: assignable partitions. Solved by removing the infinite loop cause. Happening since: 1.6.0 (#4800). +* 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. + Happening since: 0.9.0 (#4876). diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 8e43fd15e..985d8d19b 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() */ diff --git a/tests/0099-commit_metadata.c b/tests/0099-commit_metadata.c index 9acdb07f5..b5e639a1b 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);