Skip to content

Commit 9eaefa5

Browse files
committed
Handle delivery.report.only.error in Python (confluentinc#84)
1 parent ece3df4 commit 9eaefa5

File tree

4 files changed

+82
-1
lines changed

4 files changed

+82
-1
lines changed

confluent_kafka/src/Producer.c

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -152,6 +152,10 @@ static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkm,
152152
goto done;
153153
}
154154

155+
/* Skip callback if delivery.report.only.error=true */
156+
if (self->u.Producer.dr_only_error && !rkm->err)
157+
goto done;
158+
155159
msgobj = Message_new0(self, rkm);
156160

157161
args = Py_BuildValue("(OO)",

confluent_kafka/src/confluent_kafka.c

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1124,7 +1124,26 @@ static int producer_conf_set_special (Handle *self, rd_kafka_conf_t *conf,
11241124
}
11251125

11261126
return 1;
1127-
}
1127+
1128+
} else if (!strcmp(name, "delivery.report.only.error")) {
1129+
/* Since we allocate msgstate for each produced message
1130+
* with a callback we can't use delivery.report.only.error
1131+
* as-is, as we wouldn't be able to ever free those msgstates.
1132+
* Instead we shortcut this setting in the Python client,
1133+
* providing the same functionality from dr_msg_cb trampoline.
1134+
*/
1135+
1136+
if (!PyBool_Check(valobj)) {
1137+
cfl_PyErr_Format(
1138+
RD_KAFKA_RESP_ERR__INVALID_ARG,
1139+
"%s requires bool", name);
1140+
return -1;
1141+
}
1142+
1143+
self->u.Producer.dr_only_error = valobj == Py_True;
1144+
1145+
return 1;
1146+
}
11281147

11291148
return 0; /* Not handled */
11301149
}

confluent_kafka/src/confluent_kafka.h

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,8 @@ typedef struct {
138138
const rd_kafka_topic_t *,
139139
const void *, size_t, int32_t,
140140
void *, void *); /**< Fallback C partitioner*/
141+
142+
int dr_only_error; /**< delivery.report.only.error */
141143
} Producer;
142144

143145
/**

examples/integration_test.py

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -134,6 +134,62 @@ def verify_producer():
134134
# Block until all messages are delivered/failed
135135
p.flush()
136136

137+
#
138+
# Additional isolated tests
139+
#
140+
test_producer_dr_only_error()
141+
142+
143+
144+
def test_producer_dr_only_error():
145+
"""
146+
The C delivery.report.only.error configuration property
147+
can't be used with the Python client since the Python client
148+
allocates a msgstate for each produced message that has a callback,
149+
and on success (with delivery.report.only.error=true) the delivery report
150+
will not be called and the msgstate will thus never be freed.
151+
152+
Since a proper broker is required for messages to be succesfully sent
153+
this test must be run from the integration tests rather than
154+
the unit tests.
155+
"""
156+
p = confluent_kafka.Producer({"bootstrap.servers": bootstrap_servers,
157+
'broker.address.family':'v4',
158+
"delivery.report.only.error":True})
159+
160+
class DrOnlyTest (object):
161+
def __init__ (self):
162+
self.remaining = 1
163+
164+
def handle_err (self, err, msg):
165+
""" This delivery handler should only get called for errored msgs """
166+
assert "BAD:" in msg.value().decode('utf-8')
167+
assert err is not None
168+
self.remaining -= 1
169+
170+
def handle_success (self, err, msg):
171+
""" This delivery handler should never get called """
172+
# FIXME: Can we verify that it is actually garbage collected?
173+
assert "GOOD:" in msg.value().decode('utf-8')
174+
assert err is None
175+
assert False, "should never come here"
176+
177+
state = DrOnlyTest()
178+
179+
print('only.error: Verifying delivery.report.only.error')
180+
p.produce(topic, "BAD: This message will make not make it".encode('utf-8'),
181+
partition=99, on_delivery=state.handle_err)
182+
183+
p.produce(topic, "GOOD: This message will make make it".encode('utf-8'),
184+
on_delivery=state.handle_success)
185+
186+
print('only.error: Waiting for flush')
187+
p.flush(10000)
188+
189+
print('only.error: Remaining messages now %d' % state.remaining)
190+
assert state.remaining == 0
191+
192+
137193

138194
def verify_avro():
139195
from confluent_kafka import avro

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