@@ -373,8 +373,8 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
373
373
return NULL ; /* Exception raised by options_to_c() */
374
374
375
375
/* 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. */
378
378
Py_INCREF (future );
379
379
380
380
/*
@@ -440,23 +440,23 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
440
440
}
441
441
442
442
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 );
446
446
447
447
/*
448
448
* Call CreateTopics.
449
449
*
450
450
* 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.
452
452
*/
453
453
CallState_begin (self , & cs );
454
454
rd_kafka_CreateTopics (self -> rk , c_objs , tcnt , c_options , rkqu );
455
455
CallState_end (self , & cs );
456
456
457
457
rd_kafka_NewTopic_destroy_array (c_objs , tcnt );
458
458
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 */
460
460
461
461
/* Increase refcount for the return value since
462
462
* it is currently a borrowed reference. */
@@ -511,8 +511,8 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
511
511
return NULL ; /* Exception raised by options_to_c() */
512
512
513
513
/* 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. */
516
516
Py_INCREF (future );
517
517
518
518
/*
@@ -543,9 +543,9 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
543
543
}
544
544
545
545
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 );
549
549
550
550
/*
551
551
* Call DeleteTopics.
@@ -559,7 +559,7 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
559
559
560
560
rd_kafka_DeleteTopic_destroy_array (c_objs , i );
561
561
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 */
563
563
564
564
/* Increase refcount for the return value since
565
565
* it is currently a borrowed reference. */
@@ -671,9 +671,9 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
671
671
}
672
672
673
673
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 );
677
677
678
678
/*
679
679
* Call CreatePartitions
@@ -687,7 +687,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
687
687
688
688
rd_kafka_NewPartitions_destroy_array (c_objs , tcnt );
689
689
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 */
691
691
692
692
/* Increase refcount for the return value since
693
693
* it is currently a borrowed reference. */
@@ -800,9 +800,9 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
800
800
}
801
801
802
802
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 );
806
806
807
807
/*
808
808
* Call DescribeConfigs
@@ -816,7 +816,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
816
816
817
817
rd_kafka_ConfigResource_destroy_array (c_objs , cnt );
818
818
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 */
820
820
821
821
Py_DECREF (ConfigResource_type ); /* from lookup() */
822
822
@@ -990,9 +990,9 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
990
990
}
991
991
992
992
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 );
996
996
997
997
/*
998
998
* Call AlterConfigs
@@ -1006,7 +1006,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
1006
1006
1007
1007
rd_kafka_ConfigResource_destroy_array (c_objs , cnt );
1008
1008
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 */
1010
1010
1011
1011
Py_DECREF (ConfigResource_type ); /* from lookup() */
1012
1012
@@ -1326,16 +1326,16 @@ Admin_c_ConfigResource_result_to_py (const rd_kafka_ConfigResource_t **c_resourc
1326
1326
1327
1327
1328
1328
/**
1329
- * @brief Event callback triggered from internal librdkafka thread
1329
+ * @brief Event callback triggered from librdkafka's background thread
1330
1330
* when Admin API results are ready.
1331
1331
*
1332
1332
* The rkev opaque (not \p opaque) is the future PyObject
1333
1333
* which we'll set the result on.
1334
1334
*
1335
- * @locality internal rdkafka thread
1335
+ * @locality background rdkafka thread
1336
1336
*/
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 ) {
1339
1339
PyObject * future = (PyObject * )rd_kafka_event_opaque (rkev );
1340
1340
const rd_kafka_topic_result_t * * c_topic_res ;
1341
1341
size_t c_topic_res_cnt ;
@@ -1532,7 +1532,7 @@ static int Admin_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) {
1532
1532
args , kwargs )))
1533
1533
return -1 ;
1534
1534
1535
- rd_kafka_conf_set_event_cb (conf , Admin_event_cb );
1535
+ rd_kafka_conf_set_background_event_cb (conf , Admin_background_event_cb );
1536
1536
1537
1537
/* There is no dedicated ADMIN client type in librdkafka, the Admin
1538
1538
* API can use either PRODUCER or CONSUMER.
0 commit comments