Skip to content

Commit b1ec10f

Browse files
committed
Added support for fatal errors
1 parent dc16852 commit b1ec10f

File tree

3 files changed

+70
-7
lines changed

3 files changed

+70
-7
lines changed

confluent_kafka/kafkatest/verifiable_client.py

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -91,10 +91,6 @@ def set_config(conf, args):
9191
# "org.apache.kafka.clients.consumer.RangeAssignor" -> "range"
9292
conf[n] = re.sub(r'org.apache.kafka.clients.consumer.(\w+)Assignor',
9393
lambda x: x.group(1).lower(), v)
94-
95-
elif n == 'enable.idempotence':
96-
# Ignore idempotence for now, best-effortly.
97-
sys.stderr.write('%% WARN: Ignoring unsupported %s=%s\n' % (n, v))
9894
else:
9995
conf[n] = v
10096

confluent_kafka/src/confluent_kafka.c

Lines changed: 52 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -62,9 +62,12 @@ typedef struct {
6262
char *str; /* Human readable representation of error, if one
6363
* was provided by librdkafka.
6464
* Else falls back on err2str(). */
65+
int fatal; /**< Set to true if a fatal error. */
6566
} KafkaError;
6667

6768

69+
static void cfl_PyErr_Fatal (rd_kafka_resp_err_t err, const char *reason);
70+
6871
static PyObject *KafkaError_code (KafkaError *self, PyObject *ignore) {
6972
return cfl_PyInt_FromInt(self->code);
7073
}
@@ -81,6 +84,20 @@ static PyObject *KafkaError_name (KafkaError *self, PyObject *ignore) {
8184
return cfl_PyUnistr(_FromString(rd_kafka_err2name(self->code)));
8285
}
8386

87+
static PyObject *KafkaError_fatal (KafkaError *self, PyObject *ignore) {
88+
PyObject *ret = self->fatal ? Py_True : Py_False;
89+
Py_INCREF(ret);
90+
return ret;
91+
}
92+
93+
94+
static PyObject *KafkaError_test_raise_fatal (KafkaError *null,
95+
PyObject *ignore) {
96+
cfl_PyErr_Fatal(RD_KAFKA_RESP_ERR__INVALID_ARG,
97+
"This is a fatal exception for testing purposes");
98+
return NULL;
99+
}
100+
84101

85102
static PyMethodDef KafkaError_methods[] = {
86103
{ "code", (PyCFunction)KafkaError_code, METH_NOARGS,
@@ -105,6 +122,14 @@ static PyMethodDef KafkaError_methods[] = {
105122
" :rtype: str\n"
106123
"\n"
107124
},
125+
{ "fatal", (PyCFunction)KafkaError_fatal, METH_NOARGS,
126+
" :returns: True if this a fatal error, else False.\n"
127+
" :rtype: bool\n"
128+
"\n"
129+
},
130+
{ "_test_raise_fatal", (PyCFunction)KafkaError_test_raise_fatal,
131+
METH_NOARGS|METH_STATIC
132+
},
108133

109134
{ NULL }
110135
};
@@ -133,7 +158,8 @@ static int KafkaError_traverse (KafkaError *self,
133158
}
134159

135160
static PyObject *KafkaError_str0 (KafkaError *self) {
136-
return cfl_PyUnistr(_FromFormat("KafkaError{code=%s,val=%d,str=\"%s\"}",
161+
return cfl_PyUnistr(_FromFormat("KafkaError{%scode=%s,val=%d,str=\"%s\"}",
162+
self->fatal?"FATAL,":"",
137163
rd_kafka_err2name(self->code),
138164
self->code,
139165
self->str ? self->str :
@@ -247,6 +273,7 @@ static PyTypeObject KafkaErrorType = {
247273
static void KafkaError_init (KafkaError *self,
248274
rd_kafka_resp_err_t code, const char *str) {
249275
self->code = code;
276+
self->fatal = 0;
250277
if (str)
251278
self->str = strdup(str);
252279
else
@@ -292,6 +319,17 @@ PyObject *KafkaError_new0 (rd_kafka_resp_err_t err, const char *fmt, ...) {
292319
}
293320

294321

322+
/**
323+
* @brief Raise exception from fatal error.
324+
*/
325+
static void cfl_PyErr_Fatal (rd_kafka_resp_err_t err, const char *reason) {
326+
PyObject *eo = KafkaError_new0(err, "%s", reason);
327+
((KafkaError *)eo)->fatal = 1;
328+
PyErr_SetObject(KafkaException, eo);
329+
}
330+
331+
332+
295333

296334

297335
/****************************************************************************
@@ -1162,6 +1200,7 @@ PyObject *c_headers_to_py (rd_kafka_headers_t *headers) {
11621200
}
11631201
#endif
11641202

1203+
11651204
/****************************************************************************
11661205
*
11671206
*
@@ -1177,6 +1216,16 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque)
11771216
CallState *cs;
11781217

11791218
cs = CallState_get(h);
1219+
1220+
/* If the client raised a fatal error we'll raise an exception
1221+
* rather than calling the error callback. */
1222+
if (err == RD_KAFKA_RESP_ERR__FATAL) {
1223+
char errstr[512];
1224+
err = rd_kafka_fatal_error(rk, errstr, sizeof(errstr));
1225+
cfl_PyErr_Fatal(err, errstr);
1226+
goto crash;
1227+
}
1228+
11801229
if (!h->error_cb) {
11811230
/* No callback defined */
11821231
goto done;
@@ -1189,6 +1238,7 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque)
11891238
if (result)
11901239
Py_DECREF(result);
11911240
else {
1241+
crash:
11921242
CallState_crash(cs);
11931243
rd_kafka_yield(h->rk);
11941244
}
@@ -1766,8 +1816,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
17661816

17671817
Py_DECREF(confdict);
17681818

1769-
if (h->error_cb)
1770-
rd_kafka_conf_set_error_cb(conf, error_cb);
1819+
rd_kafka_conf_set_error_cb(conf, error_cb);
17711820

17721821
if (h->throttle_cb)
17731822
rd_kafka_conf_set_throttle_cb(conf, throttle_cb);

tests/test_KafkaError.py

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
#!/usr/bin/env python
22

3+
import pytest
34
from confluent_kafka import Producer, KafkaError, KafkaException
45
import time
56

@@ -8,6 +9,7 @@
89

910
def error_cb(err):
1011
print('error_cb', err)
12+
assert err.fatal() is False
1113
if err.code() == KafkaError._ALL_BROKERS_DOWN:
1214
global seen_all_brokers_down
1315
seen_all_brokers_down = True
@@ -31,6 +33,22 @@ def test_error_cb():
3133
assert seen_all_brokers_down
3234

3335

36+
def test_fatal():
37+
""" Test fatal exceptions """
38+
39+
# Configure an invalid broker and make sure the ALL_BROKERS_DOWN
40+
# error is seen in the error callback.
41+
p = Producer({'error_cb': error_cb})
42+
43+
with pytest.raises(KafkaException) as exc:
44+
KafkaError._test_raise_fatal()
45+
err = exc.value.args[0]
46+
assert isinstance(err, KafkaError)
47+
assert err.fatal() is True
48+
49+
p.poll(0) # Need some p use to avoid flake8 unused warning
50+
51+
3452
def test_subclassing():
3553
class MyExc(KafkaException):
3654
def a_method(self):

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