diff --git a/src/confluent_kafka/src/Admin.c b/src/confluent_kafka/src/Admin.c index c58166d6e..ad9ccf3b7 100644 --- a/src/confluent_kafka/src/Admin.c +++ b/src/confluent_kafka/src/Admin.c @@ -2305,8 +2305,8 @@ PyObject *Admin_describe_topics (Handle *self, PyObject *args, PyObject *kwargs) int topics_cnt = 0; int i = 0; - static char *kws[] = {"future", - "topic_names", + static char *kws[] = {"topic_names", + "future", /* options */ "request_timeout", "include_authorized_operations", @@ -2650,10 +2650,15 @@ PyObject *Admin_list_consumer_group_offsets (Handle *self, PyObject *args, PyObj * admin operation is finished, so we need to keep our own refcount. */ Py_INCREF(future); - if (PyList_Check(request) && - (requests_cnt = (int)PyList_Size(request)) != 1) { - PyErr_SetString(PyExc_ValueError, - "Currently we support listing only 1 consumer groups offset information"); + if (PyList_Check(request)) { + if ((int)PyList_Size(request) != 1) { + PyErr_SetString(PyExc_ValueError, + "Currently we support listing only 1 consumer groups offset inforation"); + goto err; + } + } else { + PyErr_SetString(PyExc_TypeError, + "Expected 'request' to be a list"); goto err; } @@ -2789,10 +2794,15 @@ PyObject *Admin_alter_consumer_group_offsets (Handle *self, PyObject *args, PyOb * admin operation is finished, so we need to keep our own refcount. */ Py_INCREF(future); - if (PyList_Check(request) && - (requests_cnt = (int)PyList_Size(request)) != 1) { - PyErr_SetString(PyExc_ValueError, - "Currently we support alter consumer groups offset request for 1 group only"); + if (PyList_Check(request)) { + if ((int)PyList_Size(request) != 1) { + PyErr_SetString(PyExc_ValueError, + "Currently we support alter consumer groups offset request for 1 group only"); + goto err; + } + } else { + PyErr_SetString(PyExc_TypeError, + "Expected 'request' to be a list"); goto err; } diff --git a/src/confluent_kafka/src/confluent_kafka.c b/src/confluent_kafka/src/confluent_kafka.c index 506c9a995..1f7723d69 100644 --- a/src/confluent_kafka/src/confluent_kafka.c +++ b/src/confluent_kafka/src/confluent_kafka.c @@ -1330,8 +1330,8 @@ rd_kafka_topic_partition_list_t *py_to_c_parts (PyObject *plist) { TopicPartition *tp = (TopicPartition *) PyList_GetItem(plist, i); - if (PyObject_Type((PyObject *)tp) != - (PyObject *)&TopicPartitionType) { + if (PyObject_TypeCheck((PyObject *)tp, + (PyObject *)&TopicPartitionType) == 0) { PyErr_Format(PyExc_TypeError, "expected %s", TopicPartitionType.tp_name);