Skip to content

Commit 6cf1fa4

Browse files
committed
Unlock GIL for Consumer_close's rd_kafka_destroy()
1 parent c7a6cf5 commit 6cf1fa4

File tree

1 file changed

+2
-5
lines changed

1 file changed

+2
-5
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -472,19 +472,16 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
472472

473473
static PyObject *Consumer_close (Handle *self, PyObject *ignore) {
474474
CallState cs;
475-
int raise = 0;
476475

477476
CallState_begin(self, &cs);
478477

479478
rd_kafka_consumer_close(self->rk);
480479

481-
raise = !CallState_end(self, &cs);
482-
483480
rd_kafka_destroy(self->rk);
484481
self->rk = NULL;
485482

486-
if (raise)
487-
return NULL;
483+
if (!CallState_end(self, &cs))
484+
return NULL;
488485

489486
Py_RETURN_NONE;
490487
}

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