Skip to content

Commit b535590

Browse files
committed
Proper use of message error on Producer (confluentinc#129)
1 parent d415bb9 commit b535590

File tree

5 files changed

+39
-8
lines changed

5 files changed

+39
-8
lines changed

confluent_kafka/src/Consumer.c

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -463,7 +463,7 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
463463
if (!rkm)
464464
Py_RETURN_NONE;
465465

466-
msgobj = Message_new0(rkm);
466+
msgobj = Message_new0(self, rkm);
467467
rd_kafka_message_destroy(rkm);
468468

469469
return msgobj;
@@ -770,6 +770,8 @@ static int Consumer_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) {
770770
return -1;
771771
}
772772

773+
self->type = RD_KAFKA_CONSUMER;
774+
773775
if (!(conf = common_conf_setup(RD_KAFKA_CONSUMER, self,
774776
args, kwargs)))
775777
return -1; /* Exception raised by ..conf_setup() */

confluent_kafka/src/Producer.c

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -152,7 +152,7 @@ static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkm,
152152
goto done;
153153
}
154154

155-
msgobj = Message_new0(rkm);
155+
msgobj = Message_new0(self, rkm);
156156

157157
args = Py_BuildValue("(OO)",
158158
Message_error((Message *)msgobj, NULL),
@@ -527,6 +527,8 @@ static int Producer_init (PyObject *selfobj, PyObject *args, PyObject *kwargs) {
527527
return -1;
528528
}
529529

530+
self->type = RD_KAFKA_PRODUCER;
531+
530532
if (!(conf = common_conf_setup(RD_KAFKA_PRODUCER, self,
531533
args, kwargs)))
532534
return -1;

confluent_kafka/src/confluent_kafka.c

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -529,17 +529,19 @@ PyTypeObject MessageType = {
529529
/**
530530
* @brief Internal factory to create Message object from message_t
531531
*/
532-
PyObject *Message_new0 (const rd_kafka_message_t *rkm) {
532+
PyObject *Message_new0 (const Handle *handle, const rd_kafka_message_t *rkm) {
533533
Message *self;
534534

535535
self = (Message *)MessageType.tp_alloc(&MessageType, 0);
536536
if (!self)
537537
return NULL;
538538

539-
self->error = KafkaError_new_or_None(rkm->err,
540-
rkm->err ?
541-
rd_kafka_message_errstr(rkm) :
542-
NULL);
539+
/* Only use message error string on Consumer, for Producers
540+
* it will contain the original message payload. */
541+
self->error = KafkaError_new_or_None(
542+
rkm->err,
543+
(rkm->err && handle->type != RD_KAFKA_PRODUCER) ?
544+
rd_kafka_message_errstr(rkm) : NULL);
543545

544546
if (rkm->rkt)
545547
self->topic = cfl_PyUnistr(

confluent_kafka/src/confluent_kafka.h

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -125,6 +125,7 @@ typedef struct {
125125
PyObject *stats_cb;
126126
int initiated;
127127
int tlskey; /* Thread-Local-Storage key */
128+
rd_kafka_type_t type; /* Producer or consumer */
128129

129130
union {
130131
/**
@@ -255,7 +256,7 @@ typedef struct {
255256

256257
extern PyTypeObject MessageType;
257258

258-
PyObject *Message_new0 (const rd_kafka_message_t *rkm);
259+
PyObject *Message_new0 (const Handle *handle, const rd_kafka_message_t *rkm);
259260
PyObject *Message_error (Message *self, PyObject *ignore);
260261

261262

tests/test_Producer.py

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -74,3 +74,27 @@ def produce_hi (self):
7474
sp = SubProducer({'log.thread.name': True}, 'mytopic')
7575
sp.produce('someother', value='not hello')
7676
sp.produce_hi()
77+
78+
79+
def test_dr_msg_errstr():
80+
"""
81+
Test that the error string for failed messages works (issue #129).
82+
The underlying problem is that librdkafka reuses the message payload
83+
for error value on Consumer messages, but on Producer messages the
84+
payload is the original payload and no rich error string exists.
85+
"""
86+
p = Producer({"default.topic.config":{"message.timeout.ms":10}})
87+
88+
def handle_dr (err, msg):
89+
# Neither message payloads must not affect the error string.
90+
assert err is not None
91+
assert err.code() == KafkaError._MSG_TIMED_OUT
92+
assert "Message timed out" in err.str()
93+
94+
# Unicode safe string
95+
p.produce('mytopic', "This is the message payload", on_delivery=handle_dr)
96+
97+
# Invalid unicode sequence
98+
p.produce('mytopic', "\xc2\xc2", on_delivery=handle_dr)
99+
100+
p.flush()

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