Skip to content

Commit bd72bf6

Browse files
committed
Updated for librdkafka's internal -> background queue rename
1 parent 3b4f3fb commit bd72bf6

File tree

1 file changed

+30
-30
lines changed

1 file changed

+30
-30
lines changed

confluent_kafka/src/Admin.c

Lines changed: 30 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -373,8 +373,8 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
373373
return NULL; /* Exception raised by options_to_c() */
374374

375375
/* options_to_c() sets future as the opaque, which is used in the
376-
* event_cb to set the results on the future as the admin operation
377-
* is finished, so we need to keep our own refcount. */
376+
* background_event_cb to set the results on the future as the
377+
* admin operation is finished, so we need to keep our own refcount. */
378378
Py_INCREF(future);
379379

380380
/*
@@ -440,23 +440,23 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
440440
}
441441

442442

443-
/* Use librdkafka's internal main thread queue to dispatch
444-
* Admin_event_cb() when the admin operation is finished. */
445-
rkqu = rd_kafka_queue_get_internal(self->rk);
443+
/* Use librdkafka's background thread queue to automatically dispatch
444+
* Admin_background_event_cb() when the admin operation is finished. */
445+
rkqu = rd_kafka_queue_get_background(self->rk);
446446

447447
/*
448448
* Call CreateTopics.
449449
*
450450
* We need to set up a CallState and release GIL here since
451-
* the event_cb may be triggered immediately.
451+
* the background_event_cb may be triggered immediately.
452452
*/
453453
CallState_begin(self, &cs);
454454
rd_kafka_CreateTopics(self->rk, c_objs, tcnt, c_options, rkqu);
455455
CallState_end(self, &cs);
456456

457457
rd_kafka_NewTopic_destroy_array(c_objs, tcnt);
458458
rd_kafka_AdminOptions_destroy(c_options);
459-
rd_kafka_queue_destroy(rkqu); /* drop our reference from get_internal */
459+
rd_kafka_queue_destroy(rkqu); /* drop reference from get_background */
460460

461461
/* Increase refcount for the return value since
462462
* it is currently a borrowed reference. */
@@ -511,8 +511,8 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
511511
return NULL; /* Exception raised by options_to_c() */
512512

513513
/* options_to_c() sets opaque to the future object, which is used in the
514-
* event_cb to set the results on the future as the admin operation
515-
* is finished, so we need to keep our own refcount. */
514+
* background_event_cb to set the results on the future as the
515+
* admin operation is finished, so we need to keep our own refcount. */
516516
Py_INCREF(future);
517517

518518
/*
@@ -543,9 +543,9 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
543543
}
544544

545545

546-
/* Use librdkafka's internal main thread queue to dispatch
547-
* Admin_event_cb() when the admin operation is finished. */
548-
rkqu = rd_kafka_queue_get_internal(self->rk);
546+
/* Use librdkafka's background thread queue to automatically dispatch
547+
* Admin_background_event_cb() when the admin operation is finished. */
548+
rkqu = rd_kafka_queue_get_background(self->rk);
549549

550550
/*
551551
* Call DeleteTopics.
@@ -559,7 +559,7 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
559559

560560
rd_kafka_DeleteTopic_destroy_array(c_objs, i);
561561
rd_kafka_AdminOptions_destroy(c_options);
562-
rd_kafka_queue_destroy(rkqu); /* drop our reference from get_internal */
562+
rd_kafka_queue_destroy(rkqu); /* drop reference from get_background */
563563

564564
/* Increase refcount for the return value since
565565
* it is currently a borrowed reference. */
@@ -671,9 +671,9 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
671671
}
672672

673673

674-
/* Use librdkafka's internal main thread queue to dispatch
675-
* Admin_event_cb() when the admin operation is finished. */
676-
rkqu = rd_kafka_queue_get_internal(self->rk);
674+
/* Use librdkafka's background thread queue to automatically dispatch
675+
* Admin_background_event_cb() when the admin operation is finished. */
676+
rkqu = rd_kafka_queue_get_background(self->rk);
677677

678678
/*
679679
* Call CreatePartitions
@@ -687,7 +687,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
687687

688688
rd_kafka_NewPartitions_destroy_array(c_objs, tcnt);
689689
rd_kafka_AdminOptions_destroy(c_options);
690-
rd_kafka_queue_destroy(rkqu); /* drop our reference from get_internal */
690+
rd_kafka_queue_destroy(rkqu); /* drop reference from get_background */
691691

692692
/* Increase refcount for the return value since
693693
* it is currently a borrowed reference. */
@@ -800,9 +800,9 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
800800
}
801801

802802

803-
/* Use librdkafka's internal main thread queue to dispatch
804-
* Admin_event_cb() when the admin operation is finished. */
805-
rkqu = rd_kafka_queue_get_internal(self->rk);
803+
/* Use librdkafka's background thread queue to automatically dispatch
804+
* Admin_background_event_cb() when the admin operation is finished. */
805+
rkqu = rd_kafka_queue_get_background(self->rk);
806806

807807
/*
808808
* Call DescribeConfigs
@@ -816,7 +816,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
816816

817817
rd_kafka_ConfigResource_destroy_array(c_objs, cnt);
818818
rd_kafka_AdminOptions_destroy(c_options);
819-
rd_kafka_queue_destroy(rkqu); /* drop our reference from get_internal */
819+
rd_kafka_queue_destroy(rkqu); /* drop reference from get_background */
820820

821821
Py_DECREF(ConfigResource_type); /* from lookup() */
822822

@@ -990,9 +990,9 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
990990
}
991991

992992

993-
/* Use librdkafka's internal main thread queue to dispatch
994-
* Admin_event_cb() when the admin operation is finished. */
995-
rkqu = rd_kafka_queue_get_internal(self->rk);
993+
/* Use librdkafka's background thread queue to automatically dispatch
994+
* Admin_background_event_cb() when the admin operation is finished. */
995+
rkqu = rd_kafka_queue_get_background(self->rk);
996996

997997
/*
998998
* Call AlterConfigs
@@ -1006,7 +1006,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
10061006

10071007
rd_kafka_ConfigResource_destroy_array(c_objs, cnt);
10081008
rd_kafka_AdminOptions_destroy(c_options);
1009-
rd_kafka_queue_destroy(rkqu); /* drop our reference from get_internal */
1009+
rd_kafka_queue_destroy(rkqu); /* drop reference from get_background */
10101010

10111011
Py_DECREF(ConfigResource_type); /* from lookup() */
10121012

@@ -1326,16 +1326,16 @@ Admin_c_ConfigResource_result_to_py (const rd_kafka_ConfigResource_t **c_resourc
13261326

13271327

13281328
/**
1329-
* @brief Event callback triggered from internal librdkafka thread
1329+
* @brief Event callback triggered from librdkafka's background thread
13301330
* when Admin API results are ready.
13311331
*
13321332
* The rkev opaque (not \p opaque) is the future PyObject
13331333
* which we'll set the result on.
13341334
*
1335-
* @locality internal rdkafka thread
1335+
* @locality background rdkafka thread
13361336
*/
1337-
static void Admin_event_cb (rd_kafka_t *rk, rd_kafka_event_t *rkev,
1338-
void *opaque) {
1337+
static void Admin_background_event_cb (rd_kafka_t *rk, rd_kafka_event_t *rkev,
1338+
void *opaque) {
13391339
PyObject *future = (PyObject *)rd_kafka_event_opaque(rkev);
13401340
const rd_kafka_topic_result_t **c_topic_res;
13411341
size_t c_topic_res_cnt;
@@ -1532,7 +1532,7 @@ static int Admin_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) {
15321532
args, kwargs)))
15331533
return -1;
15341534

1535-
rd_kafka_conf_set_event_cb(conf, Admin_event_cb);
1535+
rd_kafka_conf_set_background_event_cb(conf, Admin_background_event_cb);
15361536

15371537
/* There is no dedicated ADMIN client type in librdkafka, the Admin
15381538
* API can use either PRODUCER or CONSUMER.

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