Skip to content

Commit f5b6b03

Browse files
authored
Merge pull request confluentinc#262 from johnistan/raise-runtime-error-after-consumer-close
Raise RuntimeError for all consumer methods after close
2 parents 3257fea + e006829 commit f5b6b03

File tree

2 files changed

+135
-0
lines changed

2 files changed

+135
-0
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 77 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -98,6 +98,12 @@ static PyObject *Consumer_subscribe (Handle *self, PyObject *args,
9898
Py_ssize_t pos = 0;
9999
rd_kafka_resp_err_t err;
100100

101+
if (!self->rk) {
102+
PyErr_SetString(PyExc_RuntimeError,
103+
"Consumer closed");
104+
return NULL;
105+
}
106+
101107
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|OO", kws,
102108
&tlist, &on_assign, &on_revoke))
103109
return NULL;
@@ -178,6 +184,12 @@ static PyObject *Consumer_unsubscribe (Handle *self,
178184

179185
rd_kafka_resp_err_t err;
180186

187+
if (!self->rk) {
188+
PyErr_SetString(PyExc_RuntimeError,
189+
"Consumer closed");
190+
return NULL;
191+
}
192+
181193
err = rd_kafka_unsubscribe(self->rk);
182194
if (err) {
183195
cfl_PyErr_Format(err,
@@ -195,6 +207,12 @@ static PyObject *Consumer_assign (Handle *self, PyObject *tlist) {
195207
rd_kafka_topic_partition_list_t *c_parts;
196208
rd_kafka_resp_err_t err;
197209

210+
if (!self->rk) {
211+
PyErr_SetString(PyExc_RuntimeError,
212+
"Consumer closed");
213+
return NULL;
214+
}
215+
198216
if (!(c_parts = py_to_c_parts(tlist)))
199217
return NULL;
200218

@@ -219,6 +237,12 @@ static PyObject *Consumer_unassign (Handle *self, PyObject *ignore) {
219237

220238
rd_kafka_resp_err_t err;
221239

240+
if (!self->rk) {
241+
PyErr_SetString(PyExc_RuntimeError,
242+
"Consumer closed");
243+
return NULL;
244+
}
245+
222246
self->u.Consumer.rebalance_assigned++;
223247

224248
err = rd_kafka_assign(self->rk, NULL);
@@ -239,6 +263,12 @@ static PyObject *Consumer_assignment (Handle *self, PyObject *args,
239263
rd_kafka_topic_partition_list_t *c_parts;
240264
rd_kafka_resp_err_t err;
241265

266+
if (!self->rk) {
267+
PyErr_SetString(PyExc_RuntimeError,
268+
"Consumer closed");
269+
return NULL;
270+
}
271+
242272
err = rd_kafka_assignment(self->rk, &c_parts);
243273
if (err) {
244274
cfl_PyErr_Format(err,
@@ -265,6 +295,12 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
265295
int async = 1;
266296
static char *kws[] = { "message", "offsets", "async",NULL };
267297

298+
if (!self->rk) {
299+
PyErr_SetString(PyExc_RuntimeError,
300+
"Consumer closed");
301+
return NULL;
302+
}
303+
268304
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|OOO", kws,
269305
&msg, &offsets, &async_o))
270306
return NULL;
@@ -342,6 +378,11 @@ static PyObject *Consumer_store_offsets (Handle *self, PyObject *args,
342378
rd_kafka_topic_partition_list_t *c_offsets;
343379
static char *kws[] = { "message", "offsets", NULL };
344380

381+
if (!self->rk) {
382+
PyErr_SetString(PyExc_RuntimeError,
383+
"Consumer closed");
384+
return NULL;
385+
}
345386

346387
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|OO", kws,
347388
&msg, &offsets))
@@ -410,6 +451,12 @@ static PyObject *Consumer_committed (Handle *self, PyObject *args,
410451
double tmout = -1.0f;
411452
static char *kws[] = { "partitions", "timeout", NULL };
412453

454+
if (!self->rk) {
455+
PyErr_SetString(PyExc_RuntimeError,
456+
"Consumer closed");
457+
return NULL;
458+
}
459+
413460
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|d", kws,
414461
&plist, &tmout))
415462
return NULL;
@@ -445,6 +492,12 @@ static PyObject *Consumer_position (Handle *self, PyObject *args,
445492
rd_kafka_resp_err_t err;
446493
static char *kws[] = { "partitions", NULL };
447494

495+
if (!self->rk) {
496+
PyErr_SetString(PyExc_RuntimeError,
497+
"Consumer closed");
498+
return NULL;
499+
}
500+
448501
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O", kws,
449502
&plist))
450503
return NULL;
@@ -482,6 +535,12 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
482535
static char *kws[] = { "partition", "timeout", "cached", NULL };
483536
PyObject *rtup;
484537

538+
if (!self->rk) {
539+
PyErr_SetString(PyExc_RuntimeError,
540+
"Consumer closed");
541+
return NULL;
542+
}
543+
485544
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|db", kws,
486545
(PyObject **)&tp, &tmout, &cached))
487546
return NULL;
@@ -528,6 +587,12 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
528587
PyObject *msgobj;
529588
CallState cs;
530589

590+
if (!self->rk) {
591+
PyErr_SetString(PyExc_RuntimeError,
592+
"Consumer closed");
593+
return NULL;
594+
}
595+
531596
if (!PyArg_ParseTupleAndKeywords(args, kwargs, "|d", kws, &tmout))
532597
return NULL;
533598

@@ -594,6 +659,7 @@ static PyMethodDef Consumer_methods[] = {
594659
"rebalance operation.\n"
595660
"\n"
596661
" :raises KafkaException:\n"
662+
" :raises: RuntimeError if called on a closed consumer\n"
597663
"\n"
598664
"\n"
599665
".. py:function:: on_assign(consumer, partitions)\n"
@@ -605,7 +671,9 @@ static PyMethodDef Consumer_methods[] = {
605671
},
606672
{ "unsubscribe", (PyCFunction)Consumer_unsubscribe, METH_NOARGS,
607673
" Remove current subscription.\n"
674+
"\n"
608675
" :raises: KafkaException\n"
676+
" :raises: RuntimeError if called on a closed consumer\n"
609677
"\n"
610678
},
611679
{ "poll", (PyCFunction)Consumer_poll,
@@ -625,6 +693,7 @@ static PyMethodDef Consumer_methods[] = {
625693
" :param float timeout: Maximum time to block waiting for message, event or callback.\n"
626694
" :returns: A Message object or None on timeout\n"
627695
" :rtype: :py:class:`Message` or None\n"
696+
" :raises: RuntimeError if called on a closed consumer\n"
628697
"\n"
629698
},
630699
{ "assign", (PyCFunction)Consumer_assign, METH_O,
@@ -634,11 +703,13 @@ static PyMethodDef Consumer_methods[] = {
634703
":py:class:`TopicPartition` and starts consuming.\n"
635704
"\n"
636705
" :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"
637707
"\n"
638708
},
639709
{ "unassign", (PyCFunction)Consumer_unassign, METH_NOARGS,
640710
" Removes the current partition assignment and stops consuming.\n"
641711
" :raises: KafkaException\n"
712+
" :raises: RuntimeError if called on a closed consumer\n"
642713
"\n"
643714
},
644715
{ "assignment", (PyCFunction)Consumer_assignment,
@@ -650,6 +721,7 @@ static PyMethodDef Consumer_methods[] = {
650721
" :returns: List of assigned topic+partitions.\n"
651722
" :rtype: list(TopicPartition)\n"
652723
" :raises: KafkaException\n"
724+
" :raises: RuntimeError if called on a closed consumer\n"
653725
"\n"
654726
},
655727
{ "store_offsets", (PyCFunction)Consumer_store_offsets, METH_VARARGS|METH_KEYWORDS,
@@ -666,6 +738,7 @@ static PyMethodDef Consumer_methods[] = {
666738
" :param list(TopicPartition) offsets: List of topic+partitions+offsets to store.\n"
667739
" :rtype: None\n"
668740
" :raises: KafkaException\n"
741+
" :raises: RuntimeError if called on a closed consumer\n"
669742
"\n"
670743
},
671744
{ "commit", (PyCFunction)Consumer_commit, METH_VARARGS|METH_KEYWORDS,
@@ -682,6 +755,7 @@ static PyMethodDef Consumer_methods[] = {
682755
" :param bool async: Asynchronous commit, return immediately.\n"
683756
" :rtype: None\n"
684757
" :raises: KafkaException\n"
758+
" :raises: RuntimeError if called on a closed consumer\n"
685759
"\n"
686760
},
687761
{ "committed", (PyCFunction)Consumer_committed,
@@ -696,6 +770,7 @@ static PyMethodDef Consumer_methods[] = {
696770
" :returns: List of topic+partitions with offset and possibly error set.\n"
697771
" :rtype: list(TopicPartition)\n"
698772
" :raises: KafkaException\n"
773+
" :raises: RuntimeError if called on a closed consumer\n"
699774
"\n"
700775
},
701776
{ "position", (PyCFunction)Consumer_position,
@@ -710,6 +785,7 @@ static PyMethodDef Consumer_methods[] = {
710785
" :returns: List of topic+partitions with offset and possibly error set.\n"
711786
" :rtype: list(TopicPartition)\n"
712787
" :raises: KafkaException\n"
788+
" :raises: RuntimeError if called on a closed consumer\n"
713789
"\n"
714790
},
715791
{ "get_watermark_offsets", (PyCFunction)Consumer_get_watermark_offsets,
@@ -726,6 +802,7 @@ static PyMethodDef Consumer_methods[] = {
726802
" :returns: Tuple of (low,high) on success or None on timeout.\n"
727803
" :rtype: tuple(int,int)\n"
728804
" :raises: KafkaException\n"
805+
" :raises: RuntimeError if called on a closed consumer\n"
729806
"\n"
730807
},
731808
{ "close", (PyCFunction)Consumer_close, METH_NOARGS,

tests/test_Consumer.py

Lines changed: 58 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -177,3 +177,61 @@ def test_multiple_close_throw_exception():
177177
with pytest.raises(RuntimeError) as ex:
178178
c.close()
179179
assert 'Consumer already closed' == str(ex.value)
180+
181+
182+
def test_any_method_after_close_throws_exception():
183+
""" Calling any consumer method after close should thorw a RuntimeError
184+
"""
185+
c = Consumer({'group.id': 'test',
186+
'enable.auto.commit': True,
187+
'enable.auto.offset.store': False,
188+
'socket.timeout.ms': 50,
189+
'session.timeout.ms': 100})
190+
191+
c.subscribe(["test"])
192+
c.unsubscribe()
193+
c.close()
194+
195+
with pytest.raises(RuntimeError) as ex:
196+
c.subscribe(['test'])
197+
assert 'Consumer already closed' == str(ex.value)
198+
199+
with pytest.raises(RuntimeError) as ex:
200+
c.unsubscribe()
201+
assert 'Consumer already closed' == str(ex.value)
202+
203+
with pytest.raises(RuntimeError) as ex:
204+
c.poll()
205+
assert 'Consumer already closed' == str(ex.value)
206+
207+
with pytest.raises(RuntimeError) as ex:
208+
c.assign([TopicPartition('test', 0)])
209+
assert 'Consumer already closed' == str(ex.value)
210+
211+
with pytest.raises(RuntimeError) as ex:
212+
c.unassign()
213+
assert 'Consumer already closed' == str(ex.value)
214+
215+
with pytest.raises(RuntimeError) as ex:
216+
c.assignment()
217+
assert 'Consumer already closed' == str(ex.value)
218+
219+
with pytest.raises(RuntimeError) as ex:
220+
c.store_offsets(offsets=[TopicPartition("test", 0, 42)])
221+
assert 'Consumer already closed' == str(ex.value)
222+
223+
with pytest.raises(RuntimeError) as ex:
224+
c.commit()
225+
assert 'Consumer already closed' == str(ex.value)
226+
227+
with pytest.raises(RuntimeError) as ex:
228+
c.committed([TopicPartition("test", 0)])
229+
assert 'Consumer already closed' == str(ex.value)
230+
231+
with pytest.raises(RuntimeError) as ex:
232+
c.position([TopicPartition("test", 0)])
233+
assert 'Consumer already closed' == str(ex.value)
234+
235+
with pytest.raises(RuntimeError) as ex:
236+
lo, hi = c.get_watermark_offsets(TopicPartition("test", 0))
237+
assert 'Consumer already closed' == str(ex.value)

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