Skip to content

Commit bf7abda

Browse files
committed
Add support for the new rd_kafka_error_t type
1 parent ac7691a commit bf7abda

File tree

2 files changed

+61
-1
lines changed

2 files changed

+61
-1
lines changed

confluent_kafka/src/confluent_kafka.c

Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,9 @@ typedef struct {
6363
* was provided by librdkafka.
6464
* Else falls back on err2str(). */
6565
int fatal; /**< Set to true if a fatal error. */
66+
int retriable; /**< Set to true if operation is retriable. */
67+
int txn_abortable; /**< Set to true if this is an abortable
68+
* transaction error. */
6669
} KafkaError;
6770

6871

@@ -90,6 +93,18 @@ static PyObject *KafkaError_fatal (KafkaError *self, PyObject *ignore) {
9093
return ret;
9194
}
9295

96+
static PyObject *KafkaError_retriable (KafkaError *self, PyObject *ignore) {
97+
PyObject *ret = self->retriable ? Py_True : Py_False;
98+
Py_INCREF(ret);
99+
return ret;
100+
}
101+
102+
static PyObject *KafkaError_txn_abortable (KafkaError *self, PyObject *ignore) {
103+
PyObject *ret = self->txn_abortable ? Py_True : Py_False;
104+
Py_INCREF(ret);
105+
return ret;
106+
}
107+
93108

94109
static PyObject *KafkaError_test_raise_fatal (KafkaError *null,
95110
PyObject *ignore) {
@@ -130,6 +145,21 @@ static PyMethodDef KafkaError_methods[] = {
130145
{ "_test_raise_fatal", (PyCFunction)KafkaError_test_raise_fatal,
131146
METH_NOARGS|METH_STATIC
132147
},
148+
{ "retriable", (PyCFunction)KafkaError_retriable, METH_NOARGS,
149+
" :returns: True if the operation that failed may be retried, "
150+
"else False.\n"
151+
" :rtype: bool\n"
152+
"\n"
153+
},
154+
{ "txn_abortable", (PyCFunction)KafkaError_txn_abortable, METH_NOARGS,
155+
" :returns: True if the error is an abortable transaction error, "
156+
"allowing the application to abort the current transaction with "
157+
"abort_transaction() and start a new transaction with "
158+
"begin_transaction(). This will only return true for errors from "
159+
"the transactional producer API.\n"
160+
" :rtype: bool\n"
161+
"\n"
162+
},
133163

134164
{ NULL }
135165
};
@@ -274,6 +304,8 @@ static void KafkaError_init (KafkaError *self,
274304
rd_kafka_resp_err_t code, const char *str) {
275305
self->code = code;
276306
self->fatal = 0;
307+
self->retriable = 0;
308+
self->txn_abortable = 0;
277309
if (str)
278310
self->str = strdup(str);
279311
else
@@ -319,6 +351,26 @@ PyObject *KafkaError_new0 (rd_kafka_resp_err_t err, const char *fmt, ...) {
319351
}
320352

321353

354+
/**
355+
* @brief Create a KafkaError object from an rd_kafka_error_t *
356+
* and destroy the C object when done.
357+
*/
358+
PyObject *KafkaError_new_from_error_destroy (rd_kafka_error_t *error) {
359+
KafkaError *kerr;
360+
361+
kerr = (KafkaError *)KafkaError_new0(rd_kafka_error_code(error),
362+
"%s",
363+
rd_kafka_error_string(error));
364+
365+
kerr->fatal = rd_kafka_error_is_fatal(error);
366+
kerr->retriable = rd_kafka_error_is_retriable(error);
367+
kerr->txn_abortable = rd_kafka_error_is_txn_abortable(error);
368+
rd_kafka_error_destroy(error);
369+
370+
return (PyObject *)kerr;
371+
}
372+
373+
322374
/**
323375
* @brief Raise exception from fatal error.
324376
*/

confluent_kafka/src/confluent_kafka.h

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -176,7 +176,7 @@ extern PyObject *KafkaException;
176176

177177
PyObject *KafkaError_new0 (rd_kafka_resp_err_t err, const char *fmt, ...);
178178
PyObject *KafkaError_new_or_None (rd_kafka_resp_err_t err, const char *str);
179-
179+
PyObject *KafkaError_new_from_error_destroy (rd_kafka_error_t *error);
180180

181181
/**
182182
* @brief Raise an exception using KafkaError.
@@ -188,6 +188,14 @@ PyObject *KafkaError_new_or_None (rd_kafka_resp_err_t err, const char *str);
188188
PyErr_SetObject(KafkaException, _eo); \
189189
} while (0)
190190

191+
/**
192+
* @brief Create a Python exception from an rd_kafka_error_t *
193+
* and destroy it the C object when done.
194+
*/
195+
#define cfl_PyErr_from_error_destroy(error) do { \
196+
PyObject *_eo = KafkaError_new_from_error_destroy(error); \
197+
PyErr_SetObject(KafkaException, _eo); \
198+
} while (0)
191199

192200

193201
/****************************************************************************

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