Skip to content

Commit 135f17b

Browse files
committed
Unlock GIL on Producer's librdkafka destroy call (confluentinc#107)
1 parent b74eba0 commit 135f17b

File tree

1 file changed

+8
-2
lines changed

1 file changed

+8
-2
lines changed

confluent_kafka/src/Producer.c

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -108,8 +108,14 @@ static void Producer_dealloc (Handle *self) {
108108

109109
Producer_clear(self);
110110

111-
if (self->rk)
112-
rd_kafka_destroy(self->rk);
111+
if (self->rk) {
112+
CallState cs;
113+
CallState_begin(self, &cs);
114+
115+
rd_kafka_destroy(self->rk);
116+
117+
CallState_end(self, &cs);
118+
}
113119

114120
Py_TYPE(self)->tp_free((PyObject *)self);
115121
}

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