Skip to content

Commit c257b7e

Browse files
committed
Fixed security vulnerability related to strcpy
1 parent b3bde5c commit c257b7e

File tree

2 files changed

+15
-4
lines changed

2 files changed

+15
-4
lines changed

CHANGELOG.md

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,16 @@
11
# Confluent's Python client for Apache Kafka
22

3+
## v2.4.1
4+
5+
v2.4.1 is a maintenance release with the following fixes and enhancements:
6+
7+
- Fix security vulnerability related to `strcpy`
8+
9+
confluent-kafka-python is based on librdkafka v2.4.1, see the
10+
[librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.4.1)
11+
for a complete list of changes, enhancements, fixes and upgrade considerations.
12+
13+
314
## v2.4.0
415

516
v2.4.0 is a feature release with the following features, fixes and enhancements:

src/confluent_kafka/src/Admin.c

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -160,28 +160,28 @@ Admin_options_to_c (Handle *self, rd_kafka_admin_op_t for_api,
160160
if (Admin_options_is_set_int(options->require_stable_offsets) &&
161161
(err_obj = rd_kafka_AdminOptions_set_require_stable_offsets(
162162
c_options, options->require_stable_offsets))) {
163-
strcpy(errstr, rd_kafka_error_string(err_obj));
163+
snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj));
164164
goto err;
165165
}
166166

167167
if (Admin_options_is_set_int(options->include_authorized_operations) &&
168168
(err_obj = rd_kafka_AdminOptions_set_include_authorized_operations(
169169
c_options, options->include_authorized_operations))) {
170-
strcpy(errstr, rd_kafka_error_string(err_obj));
170+
snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj));
171171
goto err;
172172
}
173173

174174
if (Admin_options_is_set_int((int)options->isolation_level) &&
175175
(err_obj = rd_kafka_AdminOptions_set_isolation_level(
176176
c_options,options->isolation_level))) {
177-
strcpy(errstr, rd_kafka_error_string(err_obj));
177+
snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj));
178178
goto err;
179179
}
180180

181181
if (Admin_options_is_set_ptr(options->states) &&
182182
(err_obj = rd_kafka_AdminOptions_set_match_consumer_group_states(
183183
c_options, options->states, options->states_cnt))) {
184-
strcpy(errstr, rd_kafka_error_string(err_obj));
184+
snprintf(errstr, sizeof(errstr), "%s", rd_kafka_error_string(err_obj));
185185
goto err;
186186
}
187187

0 commit comments

Comments
 (0)