Skip to content

Commit 4859e7f

Browse files
authored
Merge pull request confluentinc#140 from confluentinc/drerr84
Handle delivery.report.only.error in Python (confluentinc#84)
2 parents ece3df4 + d5c86ec commit 4859e7f

File tree

4 files changed

+103
-1
lines changed

4 files changed

+103
-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: 77 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import uuid
2626
import sys
2727
import json
28+
import gc
2829
from copy import copy
2930

3031
try:
@@ -134,6 +135,82 @@ def verify_producer():
134135
# Block until all messages are delivered/failed
135136
p.flush()
136137

138+
#
139+
# Additional isolated tests
140+
#
141+
test_producer_dr_only_error()
142+
143+
144+
145+
# Global variable to track garbage collection of suppressed on_delivery callbacks
146+
DrOnlyTestSuccess_gced = 0
147+
148+
def test_producer_dr_only_error():
149+
"""
150+
The C delivery.report.only.error configuration property
151+
can't be used with the Python client since the Python client
152+
allocates a msgstate for each produced message that has a callback,
153+
and on success (with delivery.report.only.error=true) the delivery report
154+
will not be called and the msgstate will thus never be freed.
155+
156+
Since a proper broker is required for messages to be succesfully sent
157+
this test must be run from the integration tests rather than
158+
the unit tests.
159+
"""
160+
p = confluent_kafka.Producer({"bootstrap.servers": bootstrap_servers,
161+
'broker.address.family':'v4',
162+
"delivery.report.only.error":True})
163+
164+
class DrOnlyTestErr (object):
165+
def __init__ (self):
166+
self.remaining = 1
167+
168+
def handle_err (self, err, msg):
169+
""" This delivery handler should only get called for errored msgs """
170+
assert "BAD:" in msg.value().decode('utf-8')
171+
assert err is not None
172+
self.remaining -= 1
173+
174+
class DrOnlyTestSuccess (object):
175+
def handle_success (self, err, msg):
176+
""" This delivery handler should never get called """
177+
# FIXME: Can we verify that it is actually garbage collected?
178+
assert "GOOD:" in msg.value().decode('utf-8')
179+
assert err is None
180+
assert False, "should never come here"
181+
182+
def __del__ (self):
183+
# Indicate that gc has hit this object.
184+
global DrOnlyTestSuccess_gced
185+
DrOnlyTestSuccess_gced = 1
186+
187+
print('only.error: Verifying delivery.report.only.error')
188+
189+
state = DrOnlyTestErr()
190+
p.produce(topic, "BAD: This message will make not make it".encode('utf-8'),
191+
partition=99, on_delivery=state.handle_err)
192+
193+
not_called_state = DrOnlyTestSuccess()
194+
p.produce(topic, "GOOD: This message will make make it".encode('utf-8'),
195+
on_delivery=not_called_state.handle_success)
196+
197+
# Garbage collection should not kick in yet for not_called_state
198+
# since there is a on_delivery reference to it.
199+
not_called_state = None
200+
gc.collect()
201+
global DrOnlyTestSuccess_gced
202+
assert DrOnlyTestSuccess_gced == 0
203+
204+
print('only.error: Waiting for flush of %d messages' % len(p))
205+
p.flush(10000)
206+
207+
print('only.error: Remaining messages now %d' % state.remaining)
208+
assert state.remaining == 0
209+
210+
# Now with all messages flushed the reference to not_called_state should be gone.
211+
gc.collect()
212+
assert DrOnlyTestSuccess_gced == 1
213+
137214

138215
def verify_avro():
139216
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