@@ -156,10 +156,12 @@ rd_kafka_event_t* PollForEvent(
156
156
// Establish what attempt we are on
157
157
int attempt = 0 ;
158
158
159
- rd_kafka_event_t * event_response;
159
+ rd_kafka_event_t * event_response = nullptr ;
160
160
161
161
// Poll the event queue until we get it
162
162
do {
163
+ // free previously fetched event
164
+ rd_kafka_event_destroy (event_response);
163
165
// Increment attempt counter
164
166
attempt = attempt + 1 ;
165
167
event_response = rd_kafka_queue_poll (topic_rkqu, timeout_ms);
@@ -171,6 +173,7 @@ rd_kafka_event_t* PollForEvent(
171
173
// type, bail out with a null
172
174
if (event_response == NULL ||
173
175
rd_kafka_event_type (event_response) != event_type) {
176
+ rd_kafka_event_destroy (event_response);
174
177
return NULL ;
175
178
}
176
179
@@ -219,8 +222,9 @@ Baton AdminClient::CreateTopic(rd_kafka_NewTopic_t* topic, int timeout_ms) {
219
222
// Now we can get the error code from the event
220
223
if (rd_kafka_event_error (event_response)) {
221
224
// If we had a special error code, get out of here with it
222
- return Baton (static_cast <RdKafka::ErrorCode>(
223
- rd_kafka_event_error (event_response)));
225
+ const rd_kafka_resp_err_t errcode = rd_kafka_event_error (event_response);
226
+ rd_kafka_event_destroy (event_response);
227
+ return Baton (static_cast <RdKafka::ErrorCode>(errcode));
224
228
}
225
229
226
230
// get the created results
@@ -239,13 +243,17 @@ Baton AdminClient::CreateTopic(rd_kafka_NewTopic_t* topic, int timeout_ms) {
239
243
240
244
if (errcode != RD_KAFKA_RESP_ERR_NO_ERROR) {
241
245
if (errmsg) {
242
- return Baton (static_cast <RdKafka::ErrorCode>(errcode), std::string (errmsg)); // NOLINT
246
+ const std::string errormsg = std::string (errmsg);
247
+ rd_kafka_event_destroy (event_response);
248
+ return Baton (static_cast <RdKafka::ErrorCode>(errcode), errormsg); // NOLINT
243
249
} else {
250
+ rd_kafka_event_destroy (event_response);
244
251
return Baton (static_cast <RdKafka::ErrorCode>(errcode));
245
252
}
246
253
}
247
254
}
248
255
256
+ rd_kafka_event_destroy (event_response);
249
257
return Baton (RdKafka::ERR_NO_ERROR);
250
258
}
251
259
}
@@ -294,8 +302,9 @@ Baton AdminClient::DeleteTopic(rd_kafka_DeleteTopic_t* topic, int timeout_ms) {
294
302
// Now we can get the error code from the event
295
303
if (rd_kafka_event_error (event_response)) {
296
304
// If we had a special error code, get out of here with it
297
- return Baton (static_cast <RdKafka::ErrorCode>(
298
- rd_kafka_event_error (event_response)));
305
+ const rd_kafka_resp_err_t errcode = rd_kafka_event_error (event_response);
306
+ rd_kafka_event_destroy (event_response);
307
+ return Baton (static_cast <RdKafka::ErrorCode>(errcode));
299
308
}
300
309
301
310
// get the created results
@@ -312,10 +321,12 @@ Baton AdminClient::DeleteTopic(rd_kafka_DeleteTopic_t* topic, int timeout_ms) {
312
321
const rd_kafka_resp_err_t errcode = rd_kafka_topic_result_error (terr);
313
322
314
323
if (errcode != RD_KAFKA_RESP_ERR_NO_ERROR) {
324
+ rd_kafka_event_destroy (event_response);
315
325
return Baton (static_cast <RdKafka::ErrorCode>(errcode));
316
326
}
317
327
}
318
328
329
+ rd_kafka_event_destroy (event_response);
319
330
return Baton (RdKafka::ERR_NO_ERROR);
320
331
}
321
332
}
@@ -367,8 +378,9 @@ Baton AdminClient::CreatePartitions(
367
378
// Now we can get the error code from the event
368
379
if (rd_kafka_event_error (event_response)) {
369
380
// If we had a special error code, get out of here with it
370
- return Baton (static_cast <RdKafka::ErrorCode>(
371
- rd_kafka_event_error (event_response)));
381
+ const rd_kafka_resp_err_t errcode = rd_kafka_event_error (event_response);
382
+ rd_kafka_event_destroy (event_response);
383
+ return Baton (static_cast <RdKafka::ErrorCode>(errcode));
372
384
}
373
385
374
386
// get the created results
@@ -387,13 +399,17 @@ Baton AdminClient::CreatePartitions(
387
399
388
400
if (errcode != RD_KAFKA_RESP_ERR_NO_ERROR) {
389
401
if (errmsg) {
390
- return Baton (static_cast <RdKafka::ErrorCode>(errcode), std::string (errmsg)); // NOLINT
402
+ const std::string errormsg = std::string (errmsg);
403
+ rd_kafka_event_destroy (event_response);
404
+ return Baton (static_cast <RdKafka::ErrorCode>(errcode), errormsg); // NOLINT
391
405
} else {
406
+ rd_kafka_event_destroy (event_response);
392
407
return Baton (static_cast <RdKafka::ErrorCode>(errcode));
393
408
}
394
409
}
395
410
}
396
411
412
+ rd_kafka_event_destroy (event_response);
397
413
return Baton (RdKafka::ERR_NO_ERROR);
398
414
}
399
415
}
0 commit comments