Skip to content

Commit f669580

Browse files
committed
Fix call ordering to avoid callback crash on implicit close (confluentinc#265)
1 parent 4c6de0d commit f669580

File tree

3 files changed

+30
-22
lines changed

3 files changed

+30
-22
lines changed

confluent_kafka/src/Admin.c

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -40,8 +40,6 @@ static int Admin_clear (Handle *self) {
4040
static void Admin_dealloc (Handle *self) {
4141
PyObject_GC_UnTrack(self);
4242

43-
Admin_clear(self);
44-
4543
if (self->rk) {
4644
CallState cs;
4745
CallState_begin(self, &cs);
@@ -51,6 +49,8 @@ static void Admin_dealloc (Handle *self) {
5149
CallState_end(self, &cs);
5250
}
5351

52+
Admin_clear(self);
53+
5454
Py_TYPE(self)->tp_free((PyObject *)self);
5555
}
5656

confluent_kafka/src/Consumer.c

Lines changed: 10 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@
2828
****************************************************************************/
2929

3030

31-
static int Consumer_clear (Handle *self) {
31+
static void Consumer_clear0 (Handle *self) {
3232
if (self->u.Consumer.on_assign) {
3333
Py_DECREF(self->u.Consumer.on_assign);
3434
self->u.Consumer.on_assign = NULL;
@@ -45,18 +45,20 @@ static int Consumer_clear (Handle *self) {
4545
rd_kafka_queue_destroy(self->u.Consumer.rkqu);
4646
self->u.Consumer.rkqu = NULL;
4747
}
48+
}
4849

49-
Handle_clear(self);
50-
51-
return 0;
50+
static int Consumer_clear (Handle *self) {
51+
Consumer_clear0(self);
52+
Handle_clear(self);
53+
return 0;
5254
}
5355

5456
static void Consumer_dealloc (Handle *self) {
5557
PyObject_GC_UnTrack(self);
5658

57-
Consumer_clear(self);
59+
Consumer_clear0(self);
5860

59-
if (self->rk) {
61+
if (self->rk) {
6062
CallState cs;
6163

6264
CallState_begin(self, &cs);
@@ -71,6 +73,8 @@ static void Consumer_dealloc (Handle *self) {
7173
CallState_end(self, &cs);
7274
}
7375

76+
Handle_clear(self);
77+
7478
Py_TYPE(self)->tp_free((PyObject *)self);
7579
}
7680

confluent_kafka/src/Producer.c

Lines changed: 18 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -88,27 +88,29 @@ Producer_msgstate_destroy (struct Producer_msgstate *msgstate) {
8888
}
8989

9090

91-
static int Producer_clear (Handle *self) {
92-
if (self->u.Producer.default_dr_cb) {
93-
Py_DECREF(self->u.Producer.default_dr_cb);
94-
self->u.Producer.default_dr_cb = NULL;
95-
}
96-
if (self->u.Producer.partitioner_cb) {
97-
Py_DECREF(self->u.Producer.partitioner_cb);
98-
self->u.Producer.partitioner_cb = NULL;
99-
}
100-
101-
Handle_clear(self);
91+
static void Producer_clear0 (Handle *self) {
92+
if (self->u.Producer.default_dr_cb) {
93+
Py_DECREF(self->u.Producer.default_dr_cb);
94+
self->u.Producer.default_dr_cb = NULL;
95+
}
96+
if (self->u.Producer.partitioner_cb) {
97+
Py_DECREF(self->u.Producer.partitioner_cb);
98+
self->u.Producer.partitioner_cb = NULL;
99+
}
100+
}
102101

103-
return 0;
102+
static int Producer_clear (Handle *self) {
103+
Producer_clear0(self);
104+
Handle_clear(self);
105+
return 0;
104106
}
105107

106108
static void Producer_dealloc (Handle *self) {
107109
PyObject_GC_UnTrack(self);
108110

109-
Producer_clear(self);
111+
Producer_clear0(self);
110112

111-
if (self->rk) {
113+
if (self->rk) {
112114
CallState cs;
113115
CallState_begin(self, &cs);
114116

@@ -117,6 +119,8 @@ static void Producer_dealloc (Handle *self) {
117119
CallState_end(self, &cs);
118120
}
119121

122+
Handle_clear(self);
123+
120124
Py_TYPE(self)->tp_free((PyObject *)self);
121125
}
122126

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