@@ -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 * group_types ;
86
+ int group_types_cnt ;
85
87
};
86
88
87
89
/**@brief "unset" value initializers for Admin_options
@@ -185,6 +187,13 @@ Admin_options_to_c (Handle *self, rd_kafka_admin_op_t for_api,
185
187
goto err ;
186
188
}
187
189
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
+
188
197
return c_options ;
189
198
190
199
err :
@@ -1698,24 +1707,28 @@ static const char Admin_delete_acls_doc[] = PyDoc_STR(
1698
1707
* @brief List consumer groups
1699
1708
*/
1700
1709
PyObject * 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 ;
1702
1711
struct Admin_options options = Admin_options_INITIALIZER ;
1703
1712
rd_kafka_AdminOptions_t * c_options = NULL ;
1704
1713
CallState cs ;
1705
1714
rd_kafka_queue_t * rkqu ;
1706
1715
rd_kafka_consumer_group_state_t * c_states = NULL ;
1716
+ rd_kafka_consumer_group_type_t * c_group_types = NULL ;
1707
1717
int states_cnt = 0 ;
1718
+ int group_types_cnt = 0 ;
1708
1719
int i = 0 ;
1709
1720
1710
1721
static char * kws [] = {"future" ,
1711
1722
/* options */
1712
1723
"states_int" ,
1724
+ "group_types_int" ,
1713
1725
"request_timeout" ,
1714
1726
NULL };
1715
1727
1716
- if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|Of " , kws ,
1728
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|OOf " , kws ,
1717
1729
& future ,
1718
1730
& states_int ,
1731
+ & group_types_int ,
1719
1732
& options .request_timeout )) {
1720
1733
goto err ;
1721
1734
}
@@ -1746,6 +1759,32 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
1746
1759
}
1747
1760
}
1748
1761
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
+
1749
1788
c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS ,
1750
1789
& options , future );
1751
1790
if (!c_options ) {
@@ -1760,7 +1799,6 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
1760
1799
/* Use librdkafka's background thread queue to automatically dispatch
1761
1800
* Admin_background_event_cb() when the admin operation is finished. */
1762
1801
rkqu = rd_kafka_queue_get_background (self -> rk );
1763
-
1764
1802
/*
1765
1803
* Call ListConsumerGroupOffsets
1766
1804
*
@@ -1774,22 +1812,27 @@ PyObject *Admin_list_consumer_groups (Handle *self, PyObject *args, PyObject *kw
1774
1812
if (c_states ) {
1775
1813
free (c_states );
1776
1814
}
1815
+ if (c_group_types ) {
1816
+ free (c_group_types );
1817
+ }
1777
1818
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
1778
1819
rd_kafka_AdminOptions_destroy (c_options );
1779
-
1780
1820
Py_RETURN_NONE ;
1781
1821
err :
1782
1822
if (c_states ) {
1783
1823
free (c_states );
1784
1824
}
1825
+ if (c_group_types ) {
1826
+ free (c_group_types );
1827
+ }
1785
1828
if (c_options ) {
1786
1829
rd_kafka_AdminOptions_destroy (c_options );
1787
1830
Py_DECREF (future );
1788
1831
}
1789
1832
return NULL ;
1790
1833
}
1791
1834
const 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"
1793
1836
"\n"
1794
1837
" List all the consumer groups.\n"
1795
1838
"\n"
@@ -3566,7 +3609,6 @@ static PyObject *Admin_c_ListConsumerGroupsResults_to_py(
3566
3609
size_t valid_cnt ,
3567
3610
const rd_kafka_error_t * * c_errors_responses ,
3568
3611
size_t errors_cnt ) {
3569
-
3570
3612
PyObject * result = NULL ;
3571
3613
PyObject * ListConsumerGroupsResult_type = NULL ;
3572
3614
PyObject * ConsumerGroupListing_type = NULL ;
@@ -3609,6 +3651,8 @@ static PyObject *Admin_c_ListConsumerGroupsResults_to_py(
3609
3651
3610
3652
cfl_PyDict_SetInt (kwargs , "state" , rd_kafka_ConsumerGroupListing_state (c_valid_responses [i ]));
3611
3653
3654
+ cfl_PyDict_SetInt (kwargs , "group_type" , rd_kafka_ConsumerGroupListing_type (c_valid_responses [i ]));
3655
+
3612
3656
args = PyTuple_New (0 );
3613
3657
3614
3658
valid_result = PyObject_Call (ConsumerGroupListing_type , args , kwargs );
0 commit comments