@@ -82,6 +82,8 @@ struct Admin_options {
8282 rd_kafka_IsolationLevel_t isolation_level ;
8383 rd_kafka_consumer_group_state_t * states ;
8484 int states_cnt ;
85+ rd_kafka_consumer_group_type_t * group_types ;
86+ int group_types_cnt ;
8587};
8688
8789/**@brief "unset" value initializers for Admin_options
@@ -185,6 +187,13 @@ Admin_options_to_c (Handle *self, rd_kafka_admin_op_t for_api,
185187 goto err ;
186188 }
187189
190+ if (Admin_options_is_set_ptr (options -> group_types ) &&
191+ (err_obj = rd_kafka_AdminOptions_set_match_consumer_group_types (
192+ c_options , options -> group_types , options -> group_types_cnt ))) {
193+ snprintf (errstr , sizeof (errstr ), "%s" , rd_kafka_error_string (err_obj ));
194+ goto err ;
195+ }
196+
188197 return c_options ;
189198
190199 err :
@@ -1698,24 +1707,28 @@ static const char Admin_delete_acls_doc[] = PyDoc_STR(
16981707 * @brief List consumer groups
16991708 */
17001709PyObject * Admin_list_consumer_groups (Handle * self , PyObject * args , PyObject * kwargs ) {
1701- PyObject * future , * states_int = NULL ;
1710+ PyObject * future , * states_int , * group_types_int = NULL ;
17021711 struct Admin_options options = Admin_options_INITIALIZER ;
17031712 rd_kafka_AdminOptions_t * c_options = NULL ;
17041713 CallState cs ;
17051714 rd_kafka_queue_t * rkqu ;
17061715 rd_kafka_consumer_group_state_t * c_states = NULL ;
1716+ rd_kafka_consumer_group_type_t * c_group_types = NULL ;
17071717 int states_cnt = 0 ;
1718+ int group_types_cnt = 0 ;
17081719 int i = 0 ;
17091720
17101721 static char * kws [] = {"future" ,
17111722 /* options */
17121723 "states_int" ,
1724+ "group_types_int" ,
17131725 "request_timeout" ,
17141726 NULL };
17151727
1716- if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|Of " , kws ,
1728+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|OOf " , kws ,
17171729 & future ,
17181730 & states_int ,
1731+ & group_types_int ,
17191732 & options .request_timeout )) {
17201733 goto err ;
17211734 }
@@ -1746,6 +1759,32 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
17461759 }
17471760 }
17481761
1762+ if (group_types_int != NULL && group_types_int != Py_None ) {
1763+ if (!PyList_Check (group_types_int )) {
1764+ PyErr_SetString (PyExc_ValueError ,
1765+ "group_types must of type list" );
1766+ goto err ;
1767+ }
1768+
1769+ group_types_cnt = (int )PyList_Size (group_types_int );
1770+
1771+ if (group_types_cnt > 0 ) {
1772+ c_group_types = (rd_kafka_consumer_group_type_t * )
1773+ malloc (group_types_cnt * sizeof (rd_kafka_consumer_group_type_t ));
1774+ for (i = 0 ; i < group_types_cnt ; i ++ ) {
1775+ PyObject * group_type = PyList_GET_ITEM (group_types_int , i );
1776+ if (!cfl_PyInt_Check (group_type )) {
1777+ PyErr_SetString (PyExc_ValueError ,
1778+ "Element of group_types must be a valid group type" );
1779+ goto err ;
1780+ }
1781+ c_group_types [i ] = (rd_kafka_consumer_group_type_t ) cfl_PyInt_AsInt (group_type );
1782+ }
1783+ options .group_types = c_group_types ;
1784+ options .group_types_cnt = group_types_cnt ;
1785+ }
1786+ }
1787+
17491788 c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS ,
17501789 & options , future );
17511790 if (!c_options ) {
@@ -1760,7 +1799,6 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
17601799 /* Use librdkafka's background thread queue to automatically dispatch
17611800 * Admin_background_event_cb() when the admin operation is finished. */
17621801 rkqu = rd_kafka_queue_get_background (self -> rk );
1763-
17641802 /*
17651803 * Call ListConsumerGroupOffsets
17661804 *
@@ -1774,22 +1812,27 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
17741812 if (c_states ) {
17751813 free (c_states );
17761814 }
1815+ if (c_group_types ) {
1816+ free (c_group_types );
1817+ }
17771818 rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
17781819 rd_kafka_AdminOptions_destroy (c_options );
1779-
17801820 Py_RETURN_NONE ;
17811821err :
17821822 if (c_states ) {
17831823 free (c_states );
17841824 }
1825+ if (c_group_types ) {
1826+ free (c_group_types );
1827+ }
17851828 if (c_options ) {
17861829 rd_kafka_AdminOptions_destroy (c_options );
17871830 Py_DECREF (future );
17881831 }
17891832 return NULL ;
17901833}
17911834const char Admin_list_consumer_groups_doc [] = PyDoc_STR (
1792- ".. py:function:: list_consumer_groups(future, [states_int], [request_timeout])\n"
1835+ ".. py:function:: list_consumer_groups(future, [states_int], [group_types_int], [ request_timeout])\n"
17931836 "\n"
17941837 " List all the consumer groups.\n"
17951838 "\n"
@@ -3466,7 +3509,6 @@ static PyObject *Admin_c_ListConsumerGroupsResults_to_py(
34663509 size_t valid_cnt ,
34673510 const rd_kafka_error_t * * c_errors_responses ,
34683511 size_t errors_cnt ) {
3469-
34703512 PyObject * result = NULL ;
34713513 PyObject * ListConsumerGroupsResult_type = NULL ;
34723514 PyObject * ConsumerGroupListing_type = NULL ;
@@ -3509,6 +3551,8 @@ static PyObject *Admin_c_ListConsumerGroupsResults_to_py(
35093551
35103552 cfl_PyDict_SetInt (kwargs , "state" , rd_kafka_ConsumerGroupListing_state (c_valid_responses [i ]));
35113553
3554+ cfl_PyDict_SetInt (kwargs , "group_type" , rd_kafka_ConsumerGroupListing_type (c_valid_responses [i ]));
3555+
35123556 args = PyTuple_New (0 );
35133557
35143558 valid_result = PyObject_Call (ConsumerGroupListing_type , args , kwargs );
0 commit comments