Skip to content

Commit 244e0e6

Browse files
committed
Fixed a segfault when 'commit' or 'store_offsets' consumer method is called incorrectly with errored Message object
1 parent f360bdf commit 244e0e6

File tree

2 files changed

+21
-0
lines changed

2 files changed

+21
-0
lines changed

CHANGELOG.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ v2.4.1 is a maintenance release with the following fixes and enhancements:
66

77
- Removed usage of `strcpy` to enhance security of the client (#1745)
88
- Fixed invalid write in `OAUTHBEARER/OIDC` extensions copy (#1745)
9+
- Fixed a segfault when `commit` or `store_offsets` consumer method is called incorrectly with errored Message object
910

1011
confluent-kafka-python is based on librdkafka v2.4.1, see the
1112
[librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.4.1)

src/confluent_kafka/src/Consumer.c

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -496,6 +496,16 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
496496
}
497497

498498
m = (Message *)msg;
499+
500+
if (m->error != Py_None) {
501+
PyObject *error = Message_error(m, NULL);
502+
PyObject *errstr = PyObject_CallMethod(error, "str", NULL);
503+
cfl_PyErr_Format(RD_KAFKA_RESP_ERR__INVALID_ARG,
504+
"Cannot commit offsets for message with error: '%s'" , PyUnicode_AsUTF8(errstr));
505+
Py_DECREF(error);
506+
Py_DECREF(errstr);
507+
return NULL;
508+
}
499509

500510
c_offsets = rd_kafka_topic_partition_list_new(1);
501511
rktpar = rd_kafka_topic_partition_list_add(
@@ -627,6 +637,16 @@ static PyObject *Consumer_store_offsets (Handle *self, PyObject *args,
627637

628638
m = (Message *)msg;
629639

640+
if (m->error != Py_None) {
641+
PyObject *error = Message_error(m, NULL);
642+
PyObject *errstr = PyObject_CallMethod(error, "str", NULL);
643+
cfl_PyErr_Format(RD_KAFKA_RESP_ERR__INVALID_ARG,
644+
"Cannot store offsets for message with error: '%s'" , PyUnicode_AsUTF8(errstr));
645+
Py_DECREF(error);
646+
Py_DECREF(errstr);
647+
return NULL;
648+
}
649+
630650
c_offsets = rd_kafka_topic_partition_list_new(1);
631651
rktpar = rd_kafka_topic_partition_list_add(
632652
c_offsets, cfl_PyUnistr_AsUTF8(m->topic, &uo8),

0 commit comments

Comments
 (0)