Skip to content

Commit cecc139

Browse files
committed
Fix memory leak in msg.headers()
1 parent bb16e44 commit cecc139

File tree

1 file changed

+6
-0
lines changed

1 file changed

+6
-0
lines changed

confluent_kafka/src/confluent_kafka.c

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1001,13 +1001,15 @@ static int py_header_to_c (rd_kafka_headers_t *rd_headers,
10011001
if (cfl_PyBin(_AsStringAndSize(value, (char **)&v,
10021002
&vsize)) == -1) {
10031003
Py_DECREF(ks);
1004+
Py_XDECREF(ks8);
10041005
return 0;
10051006
}
10061007
} else if (cfl_PyUnistr(_Check(value))) {
10071008
/* Unicode string, translate to utf-8. */
10081009
v = cfl_PyUnistr_AsUTF8(value, &vo8);
10091010
if (!v) {
10101011
Py_DECREF(ks);
1012+
Py_XDECREF(ks8);
10111013
return 0;
10121014
}
10131015
vsize = (Py_ssize_t)strlen(v);
@@ -1017,6 +1019,8 @@ static int py_header_to_c (rd_kafka_headers_t *rd_headers,
10171019
"None, binary, or unicode string, not %s",
10181020
((PyTypeObject *)PyObject_Type(value))->
10191021
tp_name);
1022+
Py_DECREF(ks);
1023+
Py_XDECREF(ks8);
10201024
return 0;
10211025
}
10221026
}
@@ -1027,11 +1031,13 @@ static int py_header_to_c (rd_kafka_headers_t *rd_headers,
10271031
"%s",
10281032
k, rd_kafka_err2str(err));
10291033
Py_DECREF(ks);
1034+
Py_XDECREF(ks8);
10301035
Py_XDECREF(vo8);
10311036
return 0;
10321037
}
10331038

10341039
Py_DECREF(ks);
1040+
Py_XDECREF(ks8);
10351041
Py_XDECREF(vo8);
10361042

10371043
return 1;

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