28
28
****************************************************************************/
29
29
30
30
31
- static int Consumer_clear (Consumer * self ) {
32
- if (self -> on_assign ) {
33
- Py_DECREF (self -> on_assign );
34
- self -> on_assign = NULL ;
31
+ static int Consumer_clear (Handle * self ) {
32
+ if (self -> u . Consumer . on_assign ) {
33
+ Py_DECREF (self -> u . Consumer . on_assign );
34
+ self -> u . Consumer . on_assign = NULL ;
35
35
}
36
- if (self -> on_revoke ) {
37
- Py_DECREF (self -> on_revoke );
38
- self -> on_revoke = NULL ;
36
+ if (self -> u . Consumer . on_revoke ) {
37
+ Py_DECREF (self -> u . Consumer . on_revoke );
38
+ self -> u . Consumer . on_revoke = NULL ;
39
39
}
40
- if (self -> on_commit ) {
41
- Py_DECREF (self -> on_commit );
42
- self -> on_commit = NULL ;
40
+ if (self -> u . Consumer . on_commit ) {
41
+ Py_DECREF (self -> u . Consumer . on_commit );
42
+ self -> u . Consumer . on_commit = NULL ;
43
43
}
44
+
45
+ Handle_clear (self );
46
+
44
47
return 0 ;
45
48
}
46
49
47
- static void Consumer_dealloc (Consumer * self ) {
50
+ static void Consumer_dealloc (Handle * self ) {
48
51
PyObject_GC_UnTrack (self );
49
52
50
53
Consumer_clear (self );
@@ -55,12 +58,15 @@ static void Consumer_dealloc (Consumer *self) {
55
58
Py_TYPE (self )-> tp_free ((PyObject * )self );
56
59
}
57
60
58
- static int Consumer_traverse (Consumer * self ,
59
- visitproc visit , void * arg ) {
60
- if (self -> on_assign )
61
- Py_VISIT (self -> on_assign );
62
- if (self -> on_revoke )
63
- Py_VISIT (self -> on_revoke );
61
+ static int Consumer_traverse (Handle * self ,
62
+ visitproc visit , void * arg ) {
63
+ if (self -> u .Consumer .on_assign )
64
+ Py_VISIT (self -> u .Consumer .on_assign );
65
+ if (self -> u .Consumer .on_revoke )
66
+ Py_VISIT (self -> u .Consumer .on_revoke );
67
+
68
+ Handle_traverse (self , visit , arg );
69
+
64
70
return 0 ;
65
71
}
66
72
@@ -69,7 +75,7 @@ static int Consumer_traverse (Consumer *self,
69
75
70
76
71
77
72
- static PyObject * Consumer_subscribe (Consumer * self , PyObject * args ,
78
+ static PyObject * Consumer_subscribe (Handle * self , PyObject * args ,
73
79
PyObject * kwargs ) {
74
80
75
81
rd_kafka_topic_partition_list_t * topics ;
@@ -130,29 +136,29 @@ static PyObject *Consumer_subscribe (Consumer *self, PyObject *args,
130
136
/*
131
137
* Update rebalance callbacks
132
138
*/
133
- if (self -> on_assign ) {
134
- Py_DECREF (self -> on_assign );
135
- self -> on_assign = NULL ;
139
+ if (self -> u . Consumer . on_assign ) {
140
+ Py_DECREF (self -> u . Consumer . on_assign );
141
+ self -> u . Consumer . on_assign = NULL ;
136
142
}
137
143
if (on_assign ) {
138
- self -> on_assign = on_assign ;
139
- Py_INCREF (self -> on_assign );
144
+ self -> u . Consumer . on_assign = on_assign ;
145
+ Py_INCREF (self -> u . Consumer . on_assign );
140
146
}
141
147
142
- if (self -> on_revoke ) {
143
- Py_DECREF (self -> on_revoke );
144
- self -> on_revoke = NULL ;
148
+ if (self -> u . Consumer . on_revoke ) {
149
+ Py_DECREF (self -> u . Consumer . on_revoke );
150
+ self -> u . Consumer . on_revoke = NULL ;
145
151
}
146
152
if (on_revoke ) {
147
- self -> on_revoke = on_revoke ;
148
- Py_INCREF (self -> on_revoke );
153
+ self -> u . Consumer . on_revoke = on_revoke ;
154
+ Py_INCREF (self -> u . Consumer . on_revoke );
149
155
}
150
156
151
157
Py_RETURN_NONE ;
152
158
}
153
159
154
160
155
- static PyObject * Consumer_unsubscribe (Consumer * self ,
161
+ static PyObject * Consumer_unsubscribe (Handle * self ,
156
162
PyObject * ignore ) {
157
163
158
164
rd_kafka_resp_err_t err ;
@@ -169,15 +175,15 @@ static PyObject *Consumer_unsubscribe (Consumer *self,
169
175
}
170
176
171
177
172
- static PyObject * Consumer_assign (Consumer * self , PyObject * tlist ) {
178
+ static PyObject * Consumer_assign (Handle * self , PyObject * tlist ) {
173
179
174
180
rd_kafka_topic_partition_list_t * c_parts ;
175
181
rd_kafka_resp_err_t err ;
176
182
177
183
if (!(c_parts = py_to_c_parts (tlist )))
178
184
return NULL ;
179
185
180
- self -> rebalance_assigned ++ ;
186
+ self -> u . Consumer . rebalance_assigned ++ ;
181
187
182
188
err = rd_kafka_assign (self -> rk , c_parts );
183
189
@@ -194,11 +200,11 @@ static PyObject *Consumer_assign (Consumer *self, PyObject *tlist) {
194
200
}
195
201
196
202
197
- static PyObject * Consumer_unassign (Consumer * self , PyObject * ignore ) {
203
+ static PyObject * Consumer_unassign (Handle * self , PyObject * ignore ) {
198
204
199
205
rd_kafka_resp_err_t err ;
200
206
201
- self -> rebalance_assigned ++ ;
207
+ self -> u . Consumer . rebalance_assigned ++ ;
202
208
203
209
err = rd_kafka_assign (self -> rk , NULL );
204
210
if (err ) {
@@ -213,7 +219,7 @@ static PyObject *Consumer_unassign (Consumer *self, PyObject *ignore) {
213
219
214
220
215
221
216
- static PyObject * Consumer_commit (Consumer * self , PyObject * args ,
222
+ static PyObject * Consumer_commit (Handle * self , PyObject * args ,
217
223
PyObject * kwargs ) {
218
224
219
225
rd_kafka_resp_err_t err ;
@@ -281,7 +287,7 @@ static PyObject *Consumer_commit (Consumer *self, PyObject *args,
281
287
282
288
283
289
284
- static PyObject * Consumer_committed (Consumer * self , PyObject * args ,
290
+ static PyObject * Consumer_committed (Handle * self , PyObject * args ,
285
291
PyObject * kwargs ) {
286
292
287
293
PyObject * plist ;
@@ -317,7 +323,7 @@ static PyObject *Consumer_committed (Consumer *self, PyObject *args,
317
323
}
318
324
319
325
320
- static PyObject * Consumer_position (Consumer * self , PyObject * args ,
326
+ static PyObject * Consumer_position (Handle * self , PyObject * args ,
321
327
PyObject * kwargs ) {
322
328
323
329
PyObject * plist ;
@@ -352,7 +358,7 @@ static PyObject *Consumer_position (Consumer *self, PyObject *args,
352
358
353
359
354
360
355
- static PyObject * Consumer_poll (Consumer * self , PyObject * args ,
361
+ static PyObject * Consumer_poll (Handle * self , PyObject * args ,
356
362
PyObject * kwargs ) {
357
363
double tmout = -1.0f ;
358
364
static char * kws [] = { "timeout" , NULL };
@@ -384,7 +390,7 @@ static PyObject *Consumer_poll (Consumer *self, PyObject *args,
384
390
}
385
391
386
392
387
- static PyObject * Consumer_close (Consumer * self , PyObject * ignore ) {
393
+ static PyObject * Consumer_close (Handle * self , PyObject * ignore ) {
388
394
self -> thread_state = PyEval_SaveThread ();
389
395
rd_kafka_consumer_close (self -> rk );
390
396
PyEval_RestoreThread (self -> thread_state );
@@ -523,7 +529,7 @@ static PyObject *Consumer_new (PyTypeObject *type, PyObject *args,
523
529
PyTypeObject ConsumerType = {
524
530
PyVarObject_HEAD_INIT (NULL , 0 )
525
531
"cimpl.Consumer" , /*tp_name*/
526
- sizeof (Consumer ), /*tp_basicsize*/
532
+ sizeof (Handle ), /*tp_basicsize*/
527
533
0 , /*tp_itemsize*/
528
534
(destructor )Consumer_dealloc , /*tp_dealloc*/
529
535
0 , /*tp_print*/
@@ -584,14 +590,16 @@ PyTypeObject ConsumerType = {
584
590
static void Consumer_rebalance_cb (rd_kafka_t * rk , rd_kafka_resp_err_t err ,
585
591
rd_kafka_topic_partition_list_t * c_parts ,
586
592
void * opaque ) {
587
- Consumer * self = opaque ;
593
+ Handle * self = opaque ;
588
594
589
595
PyEval_RestoreThread (self -> thread_state );
590
596
591
- self -> rebalance_assigned = 0 ;
597
+ self -> u . Consumer . rebalance_assigned = 0 ;
592
598
593
- if ((err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS && self -> on_assign ) ||
594
- (err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS && self -> on_revoke )) {
599
+ if ((err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS &&
600
+ self -> u .Consumer .on_assign ) ||
601
+ (err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS &&
602
+ self -> u .Consumer .on_revoke )) {
595
603
PyObject * parts ;
596
604
PyObject * args , * result ;
597
605
@@ -612,7 +620,8 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
612
620
613
621
result = PyObject_CallObject (
614
622
err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS ?
615
- self -> on_assign : self -> on_revoke , args );
623
+ self -> u .Consumer .on_assign :
624
+ self -> u .Consumer .on_revoke , args );
616
625
617
626
Py_DECREF (args );
618
627
@@ -628,7 +637,7 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
628
637
* to synchronize state, if the user did not do this from callback,
629
638
* or there was no callback, or the callback failed, then we perform
630
639
* that assign() call here instead. */
631
- if (!self -> rebalance_assigned ) {
640
+ if (!self -> u . Consumer . rebalance_assigned ) {
632
641
if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS )
633
642
rd_kafka_assign (rk , c_parts );
634
643
else
@@ -642,10 +651,10 @@ static void Consumer_rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
642
651
static void Consumer_offset_commit_cb (rd_kafka_t * rk , rd_kafka_resp_err_t err ,
643
652
rd_kafka_topic_partition_list_t * c_parts ,
644
653
void * opaque ) {
645
- Consumer * self = opaque ;
654
+ Handle * self = opaque ;
646
655
PyObject * parts , * k_err , * args , * result ;
647
656
648
- if (!self -> on_commit )
657
+ if (!self -> u . Consumer . on_commit )
649
658
return ;
650
659
651
660
PyEval_RestoreThread (self -> thread_state );
@@ -669,7 +678,7 @@ static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
669
678
return ;
670
679
}
671
680
672
- result = PyObject_CallObject (self -> on_commit , args );
681
+ result = PyObject_CallObject (self -> u . Consumer . on_commit , args );
673
682
674
683
Py_DECREF (args );
675
684
@@ -687,16 +696,16 @@ static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
687
696
688
697
static PyObject * Consumer_new (PyTypeObject * type , PyObject * args ,
689
698
PyObject * kwargs ) {
690
- Consumer * self ;
699
+ Handle * self ;
691
700
char errstr [256 ];
692
701
rd_kafka_conf_t * conf ;
693
702
694
- self = (Consumer * )ConsumerType .tp_alloc (& ConsumerType , 0 );
703
+ self = (Handle * )ConsumerType .tp_alloc (& ConsumerType , 0 );
695
704
if (!self )
696
705
return NULL ;
697
706
698
707
if (!(conf = common_conf_setup (RD_KAFKA_CONSUMER , self ,
699
- args , kwargs ))) {
708
+ args , kwargs ))) {
700
709
Py_DECREF (self );
701
710
return NULL ;
702
711
}
0 commit comments