Skip to content

Commit 868787d

Browse files
committed
librdkafka AdminOptions_new interface update
1 parent 4f0f69e commit 868787d

File tree

1 file changed

+12
-8
lines changed

1 file changed

+12
-8
lines changed

confluent_kafka/src/Admin.c

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -101,17 +101,19 @@ struct Admin_options {
101101
* which case an exception is raised.
102102
*/
103103
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,
105105
const struct Admin_options *options,
106106
PyObject *future) {
107107
rd_kafka_AdminOptions_t *c_options;
108108
rd_kafka_resp_err_t err;
109109
char errstr[512];
110110

111-
c_options = rd_kafka_AdminOptions_new(self->rk, forApi);
111+
c_options = rd_kafka_AdminOptions_new(self->rk, for_api);
112112
if (!c_options) {
113113
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());
115117
return NULL;
116118
}
117119

@@ -368,7 +370,8 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
368370
&options.validate_only))
369371
return NULL;
370372

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);
372375
if (!c_options)
373376
return NULL; /* Exception raised by options_to_c() */
374377

@@ -506,7 +509,8 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
506509
return NULL;
507510
}
508511

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);
510514
if (!c_options)
511515
return NULL; /* Exception raised by options_to_c() */
512516

@@ -616,7 +620,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
616620
&options.validate_only))
617621
return NULL;
618622

619-
c_options = Admin_options_to_c(self, "CreatePartitions",
623+
c_options = Admin_options_to_c(self, RD_KAFKA_ADMIN_OP_CREATEPARTITIONS,
620624
&options, future);
621625
if (!c_options)
622626
return NULL; /* Exception raised by options_to_c() */
@@ -738,7 +742,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
738742
return NULL;
739743
}
740744

741-
c_options = Admin_options_to_c(self, "DescribeConfigs",
745+
c_options = Admin_options_to_c(self, RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS,
742746
&options, future);
743747
if (!c_options)
744748
return NULL; /* Exception raised by options_to_c() */
@@ -888,7 +892,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
888892
return NULL;
889893

890894

891-
c_options = Admin_options_to_c(self, "AlterConfigs",
895+
c_options = Admin_options_to_c(self, RD_KAFKA_ADMIN_OP_ALTERCONFIGS,
892896
&options, future);
893897
if (!c_options)
894898
return NULL; /* Exception raised by options_to_c() */

0 commit comments

Comments
 (0)
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy