Skip to content

Add optional timeout_ms kwarg to consumer.close() / fix potential hang in test_group #2564

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 3 commits into from
Mar 24, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 4 additions & 2 deletions kafka/consumer/group.py
Original file line number Diff line number Diff line change
Expand Up @@ -470,19 +470,21 @@ def assignment(self):
"""
return self._subscription.assigned_partitions()

def close(self, autocommit=True):
def close(self, autocommit=True, timeout_ms=None):
"""Close the consumer, waiting indefinitely for any needed cleanup.

Keyword Arguments:
autocommit (bool): If auto-commit is configured for this consumer,
this optional flag causes the consumer to attempt to commit any
pending consumed offsets prior to close. Default: True
timeout_ms (num, optional): Milliseconds to wait for auto-commit.
Default: None
"""
if self._closed:
return
log.debug("Closing the KafkaConsumer.")
self._closed = True
self._coordinator.close(autocommit=autocommit)
self._coordinator.close(autocommit=autocommit, timeout_ms=timeout_ms)
self._metrics.close()
self._client.close()
try:
Expand Down
13 changes: 7 additions & 6 deletions test/test_consumer_group.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ def test_group(kafka_broker, topic):
consumers = {}
stop = {}
threads = {}
messages = collections.defaultdict(list)
messages = collections.defaultdict(lambda: collections.defaultdict(list))
group_id = 'test-group-' + random_string(6)
def consumer_thread(i):
assert i not in consumers
Expand All @@ -60,15 +60,15 @@ def consumer_thread(i):
api_version_auto_timeout_ms=5000,
heartbeat_interval_ms=500)
while not stop[i].is_set():
for tp, records in six.itervalues(consumers[i].poll(timeout_ms=200)):
for tp, records in six.iteritems(consumers[i].poll(timeout_ms=200)):
messages[i][tp].extend(records)
consumers[i].close()
consumers[i].close(timeout_ms=500)
consumers[i] = None
stop[i] = None

num_consumers = 4
for i in range(num_consumers):
t = threading.Thread(target=consumer_thread, args=(i,))
t = threading.Thread(target=consumer_thread, args=(i,), daemon=True)
t.start()
threads[i] = t

Expand Down Expand Up @@ -129,7 +129,8 @@ def consumer_thread(i):
for c in range(num_consumers):
logging.info('Stopping consumer %s', c)
stop[c].set()
threads[c].join()
threads[c].join(timeout=5)
assert not threads[c].is_alive()
threads[c] = None


Expand Down Expand Up @@ -179,4 +180,4 @@ def test_heartbeat_thread(kafka_broker, topic):
assert consumer._coordinator.heartbeat.last_poll == last_poll
consumer.poll(timeout_ms=100)
assert consumer._coordinator.heartbeat.last_poll > last_poll
consumer.close()
consumer.close(timeout_ms=100)
Loading