@@ -82,6 +82,8 @@ struct Admin_options {
82
82
rd_kafka_IsolationLevel_t isolation_level ;
83
83
rd_kafka_consumer_group_state_t * states ;
84
84
int states_cnt ;
85
+ rd_kafka_consumer_group_type_t * types ;
86
+ int types_cnt ;
85
87
};
86
88
87
89
/**@brief "unset" value initializers for Admin_options
@@ -96,6 +98,8 @@ struct Admin_options {
96
98
Admin_options_def_int, \
97
99
Admin_options_def_ptr, \
98
100
Admin_options_def_cnt, \
101
+ Admin_options_def_ptr, \
102
+ Admin_options_def_cnt, \
99
103
}
100
104
101
105
#define Admin_options_is_set_int (v ) ((v) != Admin_options_def_int)
@@ -185,6 +189,13 @@ Admin_options_to_c (Handle *self, rd_kafka_admin_op_t for_api,
185
189
goto err ;
186
190
}
187
191
192
+ if (Admin_options_is_set_ptr (options -> types ) &&
193
+ (err_obj = rd_kafka_AdminOptions_set_match_consumer_group_types (
194
+ c_options , options -> types , options -> types_cnt ))) {
195
+ snprintf (errstr , sizeof (errstr ), "%s" , rd_kafka_error_string (err_obj ));
196
+ goto err ;
197
+ }
198
+
188
199
return c_options ;
189
200
190
201
err :
@@ -1698,24 +1709,28 @@ static const char Admin_delete_acls_doc[] = PyDoc_STR(
1698
1709
* @brief List consumer groups
1699
1710
*/
1700
1711
PyObject * Admin_list_consumer_groups (Handle * self , PyObject * args , PyObject * kwargs ) {
1701
- PyObject * future , * states_int = NULL ;
1712
+ PyObject * future , * states_int = NULL , * types_int = NULL ;
1702
1713
struct Admin_options options = Admin_options_INITIALIZER ;
1703
1714
rd_kafka_AdminOptions_t * c_options = NULL ;
1704
1715
CallState cs ;
1705
1716
rd_kafka_queue_t * rkqu ;
1706
1717
rd_kafka_consumer_group_state_t * c_states = NULL ;
1718
+ rd_kafka_consumer_group_type_t * c_types = NULL ;
1707
1719
int states_cnt = 0 ;
1720
+ int types_cnt = 0 ;
1708
1721
int i = 0 ;
1709
1722
1710
1723
static char * kws [] = {"future" ,
1711
1724
/* options */
1712
1725
"states_int" ,
1726
+ "types_int" ,
1713
1727
"request_timeout" ,
1714
1728
NULL };
1715
1729
1716
- if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|Of " , kws ,
1730
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|OOf " , kws ,
1717
1731
& future ,
1718
1732
& states_int ,
1733
+ & types_int ,
1719
1734
& options .request_timeout )) {
1720
1735
goto err ;
1721
1736
}
@@ -1736,7 +1751,7 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
1736
1751
PyObject * state = PyList_GET_ITEM (states_int , i );
1737
1752
if (!cfl_PyInt_Check (state )) {
1738
1753
PyErr_SetString (PyExc_ValueError ,
1739
- "Element of states must be a valid state " );
1754
+ "Element of states must be valid states " );
1740
1755
goto err ;
1741
1756
}
1742
1757
c_states [i ] = (rd_kafka_consumer_group_state_t ) cfl_PyInt_AsInt (state );
@@ -1746,6 +1761,33 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
1746
1761
}
1747
1762
}
1748
1763
1764
+ if (types_int != NULL && types_int != Py_None ) {
1765
+ if (!PyList_Check (types_int )) {
1766
+ PyErr_SetString (PyExc_ValueError ,
1767
+ "types must of type list" );
1768
+ goto err ;
1769
+ }
1770
+
1771
+ types_cnt = (int )PyList_Size (types_int );
1772
+
1773
+ if (types_cnt > 0 ) {
1774
+ c_types = (rd_kafka_consumer_group_type_t * )
1775
+ malloc (types_cnt *
1776
+ sizeof (rd_kafka_consumer_group_type_t ));
1777
+ for (i = 0 ; i < types_cnt ; i ++ ) {
1778
+ PyObject * type = PyList_GET_ITEM (types_int , i );
1779
+ if (!cfl_PyInt_Check (type )) {
1780
+ PyErr_SetString (PyExc_ValueError ,
1781
+ "Element of types must be valid group types" );
1782
+ goto err ;
1783
+ }
1784
+ c_types [i ] = (rd_kafka_consumer_group_type_t ) cfl_PyInt_AsInt (type );
1785
+ }
1786
+ options .types = c_types ;
1787
+ options .types_cnt = types_cnt ;
1788
+ }
1789
+ }
1790
+
1749
1791
c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS ,
1750
1792
& options , future );
1751
1793
if (!c_options ) {
@@ -1774,22 +1816,27 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
1774
1816
if (c_states ) {
1775
1817
free (c_states );
1776
1818
}
1819
+ if (c_types ) {
1820
+ free (c_types );
1821
+ }
1777
1822
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
1778
1823
rd_kafka_AdminOptions_destroy (c_options );
1779
-
1780
1824
Py_RETURN_NONE ;
1781
1825
err :
1782
1826
if (c_states ) {
1783
1827
free (c_states );
1784
1828
}
1829
+ if (c_types ) {
1830
+ free (c_types );
1831
+ }
1785
1832
if (c_options ) {
1786
1833
rd_kafka_AdminOptions_destroy (c_options );
1787
1834
Py_DECREF (future );
1788
1835
}
1789
1836
return NULL ;
1790
1837
}
1791
1838
const char Admin_list_consumer_groups_doc [] = PyDoc_STR (
1792
- ".. py:function:: list_consumer_groups(future, [states_int], [request_timeout])\n"
1839
+ ".. py:function:: list_consumer_groups(future, [states_int], [types_int], [ request_timeout])\n"
1793
1840
"\n"
1794
1841
" List all the consumer groups.\n"
1795
1842
"\n"
@@ -3711,6 +3758,8 @@ static PyObject *Admin_c_ListConsumerGroupsResults_to_py(
3711
3758
3712
3759
cfl_PyDict_SetInt (kwargs , "state" , rd_kafka_ConsumerGroupListing_state (c_valid_responses [i ]));
3713
3760
3761
+ cfl_PyDict_SetInt (kwargs , "type" , rd_kafka_ConsumerGroupListing_type (c_valid_responses [i ]));
3762
+
3714
3763
args = PyTuple_New (0 );
3715
3764
3716
3765
valid_result = PyObject_Call (ConsumerGroupListing_type , args , kwargs );
0 commit comments