@@ -98,6 +98,12 @@ static PyObject *Consumer_subscribe (Handle *self, PyObject *args,
98
98
Py_ssize_t pos = 0 ;
99
99
rd_kafka_resp_err_t err ;
100
100
101
+ if (!self -> rk ) {
102
+ PyErr_SetString (PyExc_RuntimeError ,
103
+ "Consumer already closed" );
104
+ return NULL ;
105
+ }
106
+
101
107
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|OO" , kws ,
102
108
& tlist , & on_assign , & on_revoke ))
103
109
return NULL ;
@@ -178,6 +184,12 @@ static PyObject *Consumer_unsubscribe (Handle *self,
178
184
179
185
rd_kafka_resp_err_t err ;
180
186
187
+ if (!self -> rk ) {
188
+ PyErr_SetString (PyExc_RuntimeError ,
189
+ "Consumer already closed" );
190
+ return NULL ;
191
+ }
192
+
181
193
err = rd_kafka_unsubscribe (self -> rk );
182
194
if (err ) {
183
195
cfl_PyErr_Format (err ,
@@ -195,6 +207,12 @@ static PyObject *Consumer_assign (Handle *self, PyObject *tlist) {
195
207
rd_kafka_topic_partition_list_t * c_parts ;
196
208
rd_kafka_resp_err_t err ;
197
209
210
+ if (!self -> rk ) {
211
+ PyErr_SetString (PyExc_RuntimeError ,
212
+ "Consumer already closed" );
213
+ return NULL ;
214
+ }
215
+
198
216
if (!(c_parts = py_to_c_parts (tlist )))
199
217
return NULL ;
200
218
@@ -219,6 +237,12 @@ static PyObject *Consumer_unassign (Handle *self, PyObject *ignore) {
219
237
220
238
rd_kafka_resp_err_t err ;
221
239
240
+ if (!self -> rk ) {
241
+ PyErr_SetString (PyExc_RuntimeError ,
242
+ "Consumer already closed" );
243
+ return NULL ;
244
+ }
245
+
222
246
self -> u .Consumer .rebalance_assigned ++ ;
223
247
224
248
err = rd_kafka_assign (self -> rk , NULL );
@@ -239,6 +263,12 @@ static PyObject *Consumer_assignment (Handle *self, PyObject *args,
239
263
rd_kafka_topic_partition_list_t * c_parts ;
240
264
rd_kafka_resp_err_t err ;
241
265
266
+ if (!self -> rk ) {
267
+ PyErr_SetString (PyExc_RuntimeError ,
268
+ "Consumer already closed" );
269
+ return NULL ;
270
+ }
271
+
242
272
err = rd_kafka_assignment (self -> rk , & c_parts );
243
273
if (err ) {
244
274
cfl_PyErr_Format (err ,
@@ -265,6 +295,12 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
265
295
int async = 1 ;
266
296
static char * kws [] = { "message" , "offsets" , "async" ,NULL };
267
297
298
+ if (!self -> rk ) {
299
+ PyErr_SetString (PyExc_RuntimeError ,
300
+ "Consumer already closed" );
301
+ return NULL ;
302
+ }
303
+
268
304
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "|OOO" , kws ,
269
305
& msg , & offsets , & async_o ))
270
306
return NULL ;
@@ -342,6 +378,11 @@ static PyObject *Consumer_store_offsets (Handle *self, PyObject *args,
342
378
rd_kafka_topic_partition_list_t * c_offsets ;
343
379
static char * kws [] = { "message" , "offsets" , NULL };
344
380
381
+ if (!self -> rk ) {
382
+ PyErr_SetString (PyExc_RuntimeError ,
383
+ "Consumer already closed" );
384
+ return NULL ;
385
+ }
345
386
346
387
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "|OO" , kws ,
347
388
& msg , & offsets ))
@@ -410,6 +451,12 @@ static PyObject *Consumer_committed (Handle *self, PyObject *args,
410
451
double tmout = -1.0f ;
411
452
static char * kws [] = { "partitions" , "timeout" , NULL };
412
453
454
+ if (!self -> rk ) {
455
+ PyErr_SetString (PyExc_RuntimeError ,
456
+ "Consumer already closed" );
457
+ return NULL ;
458
+ }
459
+
413
460
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|d" , kws ,
414
461
& plist , & tmout ))
415
462
return NULL ;
@@ -445,6 +492,12 @@ static PyObject *Consumer_position (Handle *self, PyObject *args,
445
492
rd_kafka_resp_err_t err ;
446
493
static char * kws [] = { "partitions" , NULL };
447
494
495
+ if (!self -> rk ) {
496
+ PyErr_SetString (PyExc_RuntimeError ,
497
+ "Consumer already closed" );
498
+ return NULL ;
499
+ }
500
+
448
501
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O" , kws ,
449
502
& plist ))
450
503
return NULL ;
@@ -482,6 +535,12 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
482
535
static char * kws [] = { "partition" , "timeout" , "cached" , NULL };
483
536
PyObject * rtup ;
484
537
538
+ if (!self -> rk ) {
539
+ PyErr_SetString (PyExc_RuntimeError ,
540
+ "Consumer already closed" );
541
+ return NULL ;
542
+ }
543
+
485
544
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|db" , kws ,
486
545
(PyObject * * )& tp , & tmout , & cached ))
487
546
return NULL ;
@@ -528,6 +587,12 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
528
587
PyObject * msgobj ;
529
588
CallState cs ;
530
589
590
+ if (!self -> rk ) {
591
+ PyErr_SetString (PyExc_RuntimeError ,
592
+ "Consumer already closed" );
593
+ return NULL ;
594
+ }
595
+
531
596
if (!PyArg_ParseTupleAndKeywords (args , kwargs , "|d" , kws , & tmout ))
532
597
return NULL ;
533
598
@@ -594,6 +659,7 @@ static PyMethodDef Consumer_methods[] = {
594
659
"rebalance operation.\n"
595
660
"\n"
596
661
" :raises KafkaException:\n"
662
+ " :raises: RuntimeError if called on a closed consumer\n"
597
663
"\n"
598
664
"\n"
599
665
".. py:function:: on_assign(consumer, partitions)\n"
@@ -605,7 +671,9 @@ static PyMethodDef Consumer_methods[] = {
605
671
},
606
672
{ "unsubscribe" , (PyCFunction )Consumer_unsubscribe , METH_NOARGS ,
607
673
" Remove current subscription.\n"
674
+ "\n"
608
675
" :raises: KafkaException\n"
676
+ " :raises: RuntimeError if called on a closed consumer\n"
609
677
"\n"
610
678
},
611
679
{ "poll" , (PyCFunction )Consumer_poll ,
@@ -625,6 +693,7 @@ static PyMethodDef Consumer_methods[] = {
625
693
" :param float timeout: Maximum time to block waiting for message, event or callback.\n"
626
694
" :returns: A Message object or None on timeout\n"
627
695
" :rtype: :py:class:`Message` or None\n"
696
+ " :raises: RuntimeError if called on a closed consumer\n"
628
697
"\n"
629
698
},
630
699
{ "assign" , (PyCFunction )Consumer_assign , METH_O ,
@@ -634,11 +703,13 @@ static PyMethodDef Consumer_methods[] = {
634
703
":py:class:`TopicPartition` and starts consuming.\n"
635
704
"\n"
636
705
" :param list(TopicPartition) partitions: List of topic+partitions and optionally initial offsets to start consuming.\n"
706
+ " :raises: RuntimeError if called on a closed consumer\n"
637
707
"\n"
638
708
},
639
709
{ "unassign" , (PyCFunction )Consumer_unassign , METH_NOARGS ,
640
710
" Removes the current partition assignment and stops consuming.\n"
641
711
" :raises: KafkaException\n"
712
+ " :raises: RuntimeError if called on a closed consumer\n"
642
713
"\n"
643
714
},
644
715
{ "assignment" , (PyCFunction )Consumer_assignment ,
@@ -650,6 +721,7 @@ static PyMethodDef Consumer_methods[] = {
650
721
" :returns: List of assigned topic+partitions.\n"
651
722
" :rtype: list(TopicPartition)\n"
652
723
" :raises: KafkaException\n"
724
+ " :raises: RuntimeError if called on a closed consumer\n"
653
725
"\n"
654
726
},
655
727
{ "store_offsets" , (PyCFunction )Consumer_store_offsets , METH_VARARGS |METH_KEYWORDS ,
@@ -666,6 +738,7 @@ static PyMethodDef Consumer_methods[] = {
666
738
" :param list(TopicPartition) offsets: List of topic+partitions+offsets to store.\n"
667
739
" :rtype: None\n"
668
740
" :raises: KafkaException\n"
741
+ " :raises: RuntimeError if called on a closed consumer\n"
669
742
"\n"
670
743
},
671
744
{ "commit" , (PyCFunction )Consumer_commit , METH_VARARGS |METH_KEYWORDS ,
@@ -682,6 +755,7 @@ static PyMethodDef Consumer_methods[] = {
682
755
" :param bool async: Asynchronous commit, return immediately.\n"
683
756
" :rtype: None\n"
684
757
" :raises: KafkaException\n"
758
+ " :raises: RuntimeError if called on a closed consumer\n"
685
759
"\n"
686
760
},
687
761
{ "committed" , (PyCFunction )Consumer_committed ,
@@ -696,6 +770,7 @@ static PyMethodDef Consumer_methods[] = {
696
770
" :returns: List of topic+partitions with offset and possibly error set.\n"
697
771
" :rtype: list(TopicPartition)\n"
698
772
" :raises: KafkaException\n"
773
+ " :raises: RuntimeError if called on a closed consumer\n"
699
774
"\n"
700
775
},
701
776
{ "position" , (PyCFunction )Consumer_position ,
@@ -710,6 +785,7 @@ static PyMethodDef Consumer_methods[] = {
710
785
" :returns: List of topic+partitions with offset and possibly error set.\n"
711
786
" :rtype: list(TopicPartition)\n"
712
787
" :raises: KafkaException\n"
788
+ " :raises: RuntimeError if called on a closed consumer\n"
713
789
"\n"
714
790
},
715
791
{ "get_watermark_offsets" , (PyCFunction )Consumer_get_watermark_offsets ,
@@ -726,6 +802,7 @@ static PyMethodDef Consumer_methods[] = {
726
802
" :returns: Tuple of (low,high) on success or None on timeout.\n"
727
803
" :rtype: tuple(int,int)\n"
728
804
" :raises: KafkaException\n"
805
+ " :raises: RuntimeError if called on a closed consumer\n"
729
806
"\n"
730
807
},
731
808
{ "close" , (PyCFunction )Consumer_close , METH_NOARGS ,
0 commit comments