@@ -193,7 +193,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
193
193
return 0 ;
194
194
}
195
195
196
- c_replicas = alloca (sizeof (* c_replicas ) *
196
+ c_replicas = malloc (sizeof (* c_replicas ) *
197
197
replica_cnt );
198
198
199
199
for (ri = 0 ; ri < replica_cnt ; ri ++ ) {
@@ -206,6 +206,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
206
206
"replica_assignment must be "
207
207
"a list of int lists with an "
208
208
"outer size of %s" , err_count_desc );
209
+ free (c_replicas );
209
210
return 0 ;
210
211
}
211
212
@@ -231,6 +232,8 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
231
232
"Unsupported forApi %s" , forApi );
232
233
}
233
234
235
+ free (c_replicas );
236
+
234
237
if (err ) {
235
238
PyErr_SetString (
236
239
PyExc_ValueError , errstr );
@@ -255,6 +258,7 @@ Admin_config_dict_to_c (void *c_obj, PyObject *dict, const char *op_name) {
255
258
256
259
while (PyDict_Next (dict , & pos , & ko , & vo )) {
257
260
PyObject * ks , * ks8 ;
261
+ PyObject * vs = NULL , * vs8 = NULL ;
258
262
const char * k ;
259
263
const char * v ;
260
264
rd_kafka_resp_err_t err ;
@@ -268,8 +272,6 @@ Admin_config_dict_to_c (void *c_obj, PyObject *dict, const char *op_name) {
268
272
269
273
k = cfl_PyUnistr_AsUTF8 (ks , & ks8 );
270
274
271
-
272
- PyObject * vs = NULL , * vs8 = NULL ;
273
275
if (!(vs = cfl_PyObject_Unistr (vo )) ||
274
276
!(v = cfl_PyUnistr_AsUTF8 (vs , & vs8 ))) {
275
277
PyErr_Format (PyExc_ValueError ,
@@ -367,7 +369,7 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
367
369
/*
368
370
* Parse the list of NewTopics and convert to corresponding C types.
369
371
*/
370
- c_objs = alloca (sizeof (* c_objs ) * tcnt );
372
+ c_objs = malloc (sizeof (* c_objs ) * tcnt );
371
373
372
374
for (i = 0 ; i < tcnt ; i ++ ) {
373
375
NewTopic * newt = (NewTopic * )PyList_GET_ITEM (topics , i );
@@ -443,13 +445,15 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
443
445
444
446
rd_kafka_NewTopic_destroy_array (c_objs , tcnt );
445
447
rd_kafka_AdminOptions_destroy (c_options );
448
+ free (c_objs );
446
449
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
447
450
448
451
Py_RETURN_NONE ;
449
452
450
453
err :
451
454
rd_kafka_NewTopic_destroy_array (c_objs , i );
452
455
rd_kafka_AdminOptions_destroy (c_options );
456
+ free (c_objs );
453
457
Py_DECREF (future ); /* from options_to_c() */
454
458
455
459
return NULL ;
@@ -503,7 +507,7 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
503
507
/*
504
508
* Parse the list of strings and convert to corresponding C types.
505
509
*/
506
- c_objs = alloca (sizeof (* c_objs ) * tcnt );
510
+ c_objs = malloc (sizeof (* c_objs ) * tcnt );
507
511
508
512
for (i = 0 ; i < tcnt ; i ++ ) {
509
513
PyObject * topic = PyList_GET_ITEM (topics , i );
@@ -544,13 +548,15 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
544
548
545
549
rd_kafka_DeleteTopic_destroy_array (c_objs , i );
546
550
rd_kafka_AdminOptions_destroy (c_options );
551
+ free (c_objs );
547
552
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
548
553
549
554
Py_RETURN_NONE ;
550
555
551
556
err :
552
557
rd_kafka_DeleteTopic_destroy_array (c_objs , i );
553
558
rd_kafka_AdminOptions_destroy (c_options );
559
+ free (c_objs );
554
560
Py_DECREF (future ); /* from options_to_c() */
555
561
556
562
return NULL ;
@@ -611,7 +617,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
611
617
/*
612
618
* Parse the list of NewPartitions and convert to corresponding C types.
613
619
*/
614
- c_objs = alloca (sizeof (* c_objs ) * tcnt );
620
+ c_objs = malloc (sizeof (* c_objs ) * tcnt );
615
621
616
622
for (i = 0 ; i < tcnt ; i ++ ) {
617
623
NewPartitions * newp = (NewPartitions * )PyList_GET_ITEM (topics ,
@@ -669,13 +675,15 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
669
675
670
676
rd_kafka_NewPartitions_destroy_array (c_objs , tcnt );
671
677
rd_kafka_AdminOptions_destroy (c_options );
678
+ free (c_objs );
672
679
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
673
680
674
681
Py_RETURN_NONE ;
675
682
676
683
err :
677
684
rd_kafka_NewPartitions_destroy_array (c_objs , i );
678
685
rd_kafka_AdminOptions_destroy (c_options );
686
+ free (c_objs );
679
687
Py_DECREF (future ); /* from options_to_c() */
680
688
681
689
return NULL ;
@@ -742,7 +750,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
742
750
* Parse the list of ConfigResources and convert to
743
751
* corresponding C types.
744
752
*/
745
- c_objs = alloca (sizeof (* c_objs ) * cnt );
753
+ c_objs = malloc (sizeof (* c_objs ) * cnt );
746
754
747
755
for (i = 0 ; i < cnt ; i ++ ) {
748
756
PyObject * res = PyList_GET_ITEM (resources , i );
@@ -795,6 +803,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
795
803
796
804
rd_kafka_ConfigResource_destroy_array (c_objs , cnt );
797
805
rd_kafka_AdminOptions_destroy (c_options );
806
+ free (c_objs );
798
807
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
799
808
800
809
Py_DECREF (ConfigResource_type ); /* from lookup() */
@@ -804,6 +813,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
804
813
err :
805
814
rd_kafka_ConfigResource_destroy_array (c_objs , i );
806
815
rd_kafka_AdminOptions_destroy (c_options );
816
+ free (c_objs );
807
817
Py_DECREF (ConfigResource_type ); /* from lookup() */
808
818
Py_DECREF (future ); /* from options_to_c() */
809
819
@@ -881,7 +891,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
881
891
* Parse the list of ConfigResources and convert to
882
892
* corresponding C types.
883
893
*/
884
- c_objs = alloca (sizeof (* c_objs ) * cnt );
894
+ c_objs = malloc (sizeof (* c_objs ) * cnt );
885
895
886
896
for (i = 0 ; i < cnt ; i ++ ) {
887
897
PyObject * res = PyList_GET_ITEM (resources , i );
@@ -950,6 +960,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
950
960
951
961
rd_kafka_ConfigResource_destroy_array (c_objs , cnt );
952
962
rd_kafka_AdminOptions_destroy (c_options );
963
+ free (c_objs );
953
964
rd_kafka_queue_destroy (rkqu ); /* drop reference from get_background */
954
965
955
966
Py_DECREF (ConfigResource_type ); /* from lookup() */
@@ -959,6 +970,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
959
970
err :
960
971
rd_kafka_ConfigResource_destroy_array (c_objs , i );
961
972
rd_kafka_AdminOptions_destroy (c_options );
973
+ free (c_objs );
962
974
Py_DECREF (ConfigResource_type ); /* from lookup() */
963
975
Py_DECREF (future ); /* from options_to_c() */
964
976
0 commit comments