|
25 | 25 | import uuid
|
26 | 26 | import sys
|
27 | 27 | import json
|
| 28 | +import gc |
28 | 29 | from copy import copy
|
29 | 30 |
|
30 | 31 | try:
|
@@ -134,6 +135,82 @@ def verify_producer():
|
134 | 135 | # Block until all messages are delivered/failed
|
135 | 136 | p.flush()
|
136 | 137 |
|
| 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 | + |
137 | 214 |
|
138 | 215 | def verify_avro():
|
139 | 216 | from confluent_kafka import avro
|
|
0 commit comments