Skip to content

Commit f279a32

Browse files
committed
kafkatest consumer fixes
* Use new commit() return value for offsets_committed rather than later-triggered offset_commit_cb to preserve event ordering to kafkatest driver. * fix various event ordering issues * retry commits for temporary failures * don't indiscriminately terminate on all consumer errors, most are benign * use store_offsets() to ensure messages are properly accounted for. * Print pid for tracking purposes * Properly handle keyboard interrupts
1 parent 411b7dc commit f279a32

File tree

1 file changed

+42
-12
lines changed

1 file changed

+42
-12
lines changed

confluent_kafka/kafkatest/verifiable_consumer.py

Lines changed: 42 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,8 @@
1616
#
1717

1818
import argparse
19+
import os
20+
import time
1921
from confluent_kafka import Consumer, KafkaError, KafkaException
2022
from verifiable_client import VerifiableClient
2123

@@ -95,10 +97,10 @@ def on_revoke(self, consumer, partitions):
9597
# Send final consumed records prior to rebalancing to make sure
9698
# latest consumed is in par with what is going to be committed.
9799
self.send_records_consumed(immediate=True)
100+
self.do_commit(immediate=True, async=False)
98101
self.assignment = list()
99102
self.assignment_dict = dict()
100103
self.send_assignment('revoked', partitions)
101-
self.do_commit(immediate=True)
102104

103105
def on_commit(self, err, partitions):
104106
""" Offsets Committed callback """
@@ -125,6 +127,10 @@ def on_commit(self, err, partitions):
125127
pd['error'] = str(p.error)
126128
d['offsets'].append(pd)
127129

130+
if len(self.assignment) == 0:
131+
self.dbg('Not sending offsets_committed: No current assignment: would be: %s' % d)
132+
return
133+
128134
self.send(d)
129135

130136
def do_commit(self, immediate=False, async=None):
@@ -149,15 +155,31 @@ def do_commit(self, immediate=False, async=None):
149155
(self.consumed_msgs - self.consumed_msgs_at_last_commit,
150156
async_mode))
151157

152-
try:
153-
self.consumer.commit(async=async_mode)
154-
except KafkaException as e:
155-
if e.args[0].code() == KafkaError._WAIT_COORD:
156-
self.dbg('Ignoring commit failure, still waiting for coordinator')
157-
elif e.args[0].code() == KafkaError._NO_OFFSET:
158-
self.dbg('No offsets to commit')
159-
else:
160-
raise
158+
retries = 3
159+
while True:
160+
try:
161+
self.dbg('Commit')
162+
offsets = self.consumer.commit(async=async_mode)
163+
self.dbg('Commit done: offsets %s' % offsets)
164+
165+
if not async_mode:
166+
self.on_commit(None, offsets)
167+
168+
break
169+
170+
except KafkaException as e:
171+
if e.args[0].code() == KafkaError._NO_OFFSET:
172+
self.dbg('No offsets to commit')
173+
break
174+
elif e.args[0].code() in (KafkaError.REQUEST_TIMED_OUT, KafkaError.NOT_COORDINATOR_FOR_GROUP, KafkaError._WAIT_COORD):
175+
self.dbg('Commit failed: %s (%d retries)' % (str(e), retries))
176+
if retries <= 0:
177+
raise
178+
retries -= 1
179+
time.sleep(1)
180+
continue
181+
else:
182+
raise
161183

162184
self.consumed_msgs_at_last_commit = self.consumed_msgs
163185

@@ -168,7 +190,7 @@ def msg_consume(self, msg):
168190
# ignore EOF
169191
pass
170192
else:
171-
self.err('Consume failed: %s' % msg.error(), term=True)
193+
self.err('Consume failed: %s' % msg.error(), term=False)
172194
return
173195

174196
if False:
@@ -192,6 +214,7 @@ def msg_consume(self, msg):
192214

193215
self.consumed_msgs += 1
194216

217+
self.consumer.store_offsets(message=msg)
195218
self.send_records_consumed(immediate=False)
196219
self.do_commit(immediate=False)
197220

@@ -229,7 +252,11 @@ def to_dict(self):
229252
args = vars(parser.parse_args())
230253

231254
conf = {'broker.version.fallback': '0.9.0',
232-
'default.topic.config': dict()}
255+
'default.topic.config': dict(),
256+
# Do explicit manual offset stores to avoid race conditions
257+
# where a message is consumed from librdkafka but not yet handled
258+
# by the Python code that keeps track of last consumed offset.
259+
'enable.auto.offset.store': False}
233260

234261
VerifiableClient.set_config(conf, args)
235262

@@ -239,6 +266,7 @@ def to_dict(self):
239266
vc.use_auto_commit = args['enable.auto.commit']
240267
vc.max_msgs = args['max_messages']
241268

269+
vc.dbg('Pid %d' % os.getpid())
242270
vc.dbg('Using config: %s' % conf)
243271

244272
vc.dbg('Subscribing to %s' % args['topic'])
@@ -261,6 +289,8 @@ def to_dict(self):
261289
vc.msg_consume(msg)
262290

263291
except KeyboardInterrupt:
292+
vc.dbg('KeyboardInterrupt')
293+
vc.run = False
264294
pass
265295

266296
vc.dbg('Closing consumer')

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