From 79efd72c102c358a461b4602d8e843788a6290c3 Mon Sep 17 00:00:00 2001 From: Naxin Date: Fri, 31 Oct 2025 11:28:16 -0400 Subject: [PATCH 1/2] update --- src/confluent_kafka/src/Admin.c | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index 4b8363006..d48bcf83e 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -4536,7 +4536,16 @@ static PyObject * Admin_c_SingleGroupResult_to_py(const rd_kafka_group_result_t kwargs = PyDict_New(); - cfl_PyDict_SetString(kwargs, "group_id", rd_kafka_group_result_name(c_group_result_response)); + /* Safely handle potential NULL group name from librdkafka */ + const char *group_name = rd_kafka_group_result_name(c_group_result_response); + if (!group_name) { + cfl_PyErr_Format(RD_KAFKA_RESP_ERR__INVALID_ARG, + "Received NULL group name from librdkafka"); + Py_DECREF(kwargs); + Py_DECREF(GroupResult_type); + return NULL; + } + cfl_PyDict_SetString(kwargs, "group_id", group_name); c_topic_partition_offset_list = rd_kafka_group_result_partitions(c_group_result_response); if(c_topic_partition_offset_list) { From 2c74c27e2fa50a0899b4cddb5b0133106ef3cd1a Mon Sep 17 00:00:00 2001 From: Naxin Date: Fri, 31 Oct 2025 12:04:41 -0400 Subject: [PATCH 2/2] update --- src/confluent_kafka/src/Admin.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index d48bcf83e..3c4dfb961 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -4539,7 +4539,7 @@ static PyObject * Admin_c_SingleGroupResult_to_py(const rd_kafka_group_result_t /* Safely handle potential NULL group name from librdkafka */ const char *group_name = rd_kafka_group_result_name(c_group_result_response); if (!group_name) { - cfl_PyErr_Format(RD_KAFKA_RESP_ERR__INVALID_ARG, + cfl_PyErr_Format(RD_KAFKA_RESP_ERR__FAIL, "Received NULL group name from librdkafka"); Py_DECREF(kwargs); Py_DECREF(GroupResult_type);