From 009764d55860f077c00105417dd4be3e8f7d0525 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Fri, 10 May 2024 14:28:46 +0530 Subject: [PATCH 1/3] Fixed security vulnerability related to strcpy --- CHANGELOG.md | 11 +++++++++++ src/confluent_kafka/src/Admin.c | 8 ++++---- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c194809d4..50ef7a3e5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,16 @@ # Confluent's Python client for Apache Kafka +## v2.4.1 + +v2.4.1 is a maintenance release with the following fixes and enhancements: + + - Fix security vulnerability related to `strcpy` + +confluent-kafka-python is based on librdkafka v2.4.1, see the +[librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.4.1) +for a complete list of changes, enhancements, fixes and upgrade considerations. + + ## v2.4.0 v2.4.0 is a feature release with the following features, fixes and enhancements: diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index 8515c2aa8..5d59d71d2 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -160,28 +160,28 @@ Admin_options_to_c (Handle *self, rd_kafka_admin_op_t for_api, if (Admin_options_is_set_int(options->require_stable_offsets) && (err_obj = rd_kafka_AdminOptions_set_require_stable_offsets( c_options, options->require_stable_offsets))) { - strcpy(errstr, rd_kafka_error_string(err_obj)); + snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj)); goto err; } if (Admin_options_is_set_int(options->include_authorized_operations) && (err_obj = rd_kafka_AdminOptions_set_include_authorized_operations( c_options, options->include_authorized_operations))) { - strcpy(errstr, rd_kafka_error_string(err_obj)); + snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj)); goto err; } if (Admin_options_is_set_int((int)options->isolation_level) && (err_obj = rd_kafka_AdminOptions_set_isolation_level( c_options,options->isolation_level))) { - strcpy(errstr, rd_kafka_error_string(err_obj)); + snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj)); goto err; } if (Admin_options_is_set_ptr(options->states) && (err_obj = rd_kafka_AdminOptions_set_match_consumer_group_states( c_options, options->states, options->states_cnt))) { - strcpy(errstr, rd_kafka_error_string(err_obj)); + snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj)); goto err; } From 897e63ec42af0569ab75890a5acbe46fbe963029 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Wed, 15 May 2024 14:09:27 +0530 Subject: [PATCH 2/3] Changelog changes --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 50ef7a3e5..05c2d0849 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,7 +4,7 @@ v2.4.1 is a maintenance release with the following fixes and enhancements: - - Fix security vulnerability related to `strcpy` + - Removed usage of `strcpy` to enhance security of the client (#1745) confluent-kafka-python is based on librdkafka v2.4.1, see the [librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.4.1) From 2097064bb80f143c29a11e2c11e46b94f1243ad3 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Thu, 16 May 2024 18:29:33 +0530 Subject: [PATCH 3/3] Fixed invalid write in OAUTHBEARER/OIDC extensions copy. Removed strcpy use from there as well. --- CHANGELOG.md | 1 + src/confluent_kafka/src/confluent_kafka.c | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 05c2d0849..9b81d9e16 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,6 +5,7 @@ v2.4.1 is a maintenance release with the following fixes and enhancements: - Removed usage of `strcpy` to enhance security of the client (#1745) + - Fixed invalid write in `OAUTHBEARER/OIDC` extensions copy (#1745) confluent-kafka-python is based on librdkafka v2.4.1, see the [librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.4.1) diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index 8bfa999f1..506c9a995 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -1893,10 +1893,10 @@ static int py_extensions_to_c (char **extensions, Py_ssize_t idx, return 0; } - extensions[idx] = (char*)malloc(ksize); - strcpy(extensions[idx], k); - extensions[idx + 1] = (char*)malloc(vsize); - strcpy(extensions[idx + 1], v); + extensions[idx] = (char*)malloc(ksize + 1); + snprintf(extensions[idx], ksize + 1, "%s", k); + extensions[idx + 1] = (char*)malloc(vsize + 1); + snprintf(extensions[idx + 1], vsize + 1, "%s", v); Py_DECREF(ks); Py_XDECREF(ks8);