@@ -101,17 +101,19 @@ struct Admin_options {
101
101
* which case an exception is raised.
102
102
*/
103
103
static rd_kafka_AdminOptions_t *
104
- Admin_options_to_c (Handle * self , const char * forApi ,
104
+ Admin_options_to_c (Handle * self , rd_kafka_admin_op_t for_api ,
105
105
const struct Admin_options * options ,
106
106
PyObject * future ) {
107
107
rd_kafka_AdminOptions_t * c_options ;
108
108
rd_kafka_resp_err_t err ;
109
109
char errstr [512 ];
110
110
111
- c_options = rd_kafka_AdminOptions_new (self -> rk , forApi );
111
+ c_options = rd_kafka_AdminOptions_new (self -> rk , for_api );
112
112
if (!c_options ) {
113
113
PyErr_Format (PyExc_RuntimeError ,
114
- "Admin API %s unsupported by librdkafka" , forApi );
114
+ "This Admin API method "
115
+ "is unsupported by librdkafka %s" ,
116
+ rd_kafka_version_str ());
115
117
return NULL ;
116
118
}
117
119
@@ -368,7 +370,8 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
368
370
& options .validate_only ))
369
371
return NULL ;
370
372
371
- c_options = Admin_options_to_c (self , "CreateTopics" , & options , future );
373
+ c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_CREATETOPICS ,
374
+ & options , future );
372
375
if (!c_options )
373
376
return NULL ; /* Exception raised by options_to_c() */
374
377
@@ -506,7 +509,8 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
506
509
return NULL ;
507
510
}
508
511
509
- c_options = Admin_options_to_c (self , "DeleteTopics" , & options , future );
512
+ c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_DELETETOPICS ,
513
+ & options , future );
510
514
if (!c_options )
511
515
return NULL ; /* Exception raised by options_to_c() */
512
516
@@ -616,7 +620,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
616
620
& options .validate_only ))
617
621
return NULL ;
618
622
619
- c_options = Admin_options_to_c (self , "CreatePartitions" ,
623
+ c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_CREATEPARTITIONS ,
620
624
& options , future );
621
625
if (!c_options )
622
626
return NULL ; /* Exception raised by options_to_c() */
@@ -738,7 +742,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
738
742
return NULL ;
739
743
}
740
744
741
- c_options = Admin_options_to_c (self , "DescribeConfigs" ,
745
+ c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS ,
742
746
& options , future );
743
747
if (!c_options )
744
748
return NULL ; /* Exception raised by options_to_c() */
@@ -888,7 +892,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
888
892
return NULL ;
889
893
890
894
891
- c_options = Admin_options_to_c (self , "AlterConfigs" ,
895
+ c_options = Admin_options_to_c (self , RD_KAFKA_ADMIN_OP_ALTERCONFIGS ,
892
896
& options , future );
893
897
if (!c_options )
894
898
return NULL ; /* Exception raised by options_to_c() */
0 commit comments