diff --git a/confluent_kafka/src/Consumer.c b/confluent_kafka/src/Consumer.c index 3ecc977e3..f6dcc143a 100644 --- a/confluent_kafka/src/Consumer.c +++ b/confluent_kafka/src/Consumer.c @@ -981,11 +981,8 @@ static PyObject *Consumer_consume (Handle *self, PyObject *args, static PyObject *Consumer_close (Handle *self, PyObject *ignore) { CallState cs; - if (!self->rk) { - PyErr_SetString(PyExc_RuntimeError, - "Consumer already closed"); - return NULL; - } + if (!self->rk) + Py_RETURN_NONE; CallState_begin(self, &cs); @@ -1275,7 +1272,6 @@ static PyMethodDef Consumer_methods[] = { "see :py:func::`poll()` for more info.\n" "\n" " :rtype: None\n" - " :raises: RuntimeError if called on a closed consumer\n" "\n" }, { "list_topics", (PyCFunction)list_topics, METH_VARARGS|METH_KEYWORDS, diff --git a/tests/test_Consumer.py b/tests/test_Consumer.py index 0c89f9d9b..7d7cb485d 100644 --- a/tests/test_Consumer.py +++ b/tests/test_Consumer.py @@ -209,8 +209,8 @@ def test_offsets_for_times(): c.close() -def test_multiple_close_throw_exception(): - """ Calling Consumer.close() multiple times should throw Runtime Exception +def test_multiple_close_does_not_throw_exception(): + """ Calling Consumer.close() multiple times should not throw Runtime Exception """ c = Consumer({'group.id': 'test', 'enable.auto.commit': True, @@ -222,10 +222,7 @@ def test_multiple_close_throw_exception(): c.unsubscribe() c.close() - - with pytest.raises(RuntimeError) as ex: - c.close() - assert ex.match('Consumer already closed') + c.close() def test_any_method_after_close_throws_exception():