Skip to content

Commit bccac7b

Browse files
authored
Change coordinator.needs_join_prepare -> coordinator.rejoining; fix consumer group test (dpkp#795)
1 parent 68c8fa4 commit bccac7b

File tree

2 files changed

+10
-5
lines changed

2 files changed

+10
-5
lines changed

kafka/coordinator/base.py

+4-4
Original file line numberDiff line numberDiff line change
@@ -89,7 +89,7 @@ def __init__(self, client, metrics, **configs):
8989
self.group_id = self.config['group_id']
9090
self.coordinator_id = None
9191
self.rejoin_needed = True
92-
self.needs_join_prepare = True
92+
self.rejoining = False
9393
self.heartbeat = Heartbeat(**self.config)
9494
self.heartbeat_task = HeartbeatTask(weakref.proxy(self))
9595
self.sensors = GroupCoordinatorMetrics(self.heartbeat, metrics,
@@ -235,9 +235,9 @@ def ensure_active_group(self):
235235
if not self.need_rejoin():
236236
return
237237

238-
if self.needs_join_prepare:
238+
if not self.rejoining:
239239
self._on_join_prepare(self.generation, self.member_id)
240-
self.needs_join_prepare = False
240+
self.rejoining = True
241241

242242
while self.need_rejoin():
243243
self.ensure_coordinator_known()
@@ -249,7 +249,7 @@ def ensure_active_group(self):
249249
member_assignment_bytes = future.value
250250
self._on_join_complete(self.generation, self.member_id,
251251
self.protocol, member_assignment_bytes)
252-
self.needs_join_prepare = True
252+
self.rejoining = False
253253
self.heartbeat_task.reset()
254254
else:
255255
assert future.failed()

test/test_consumer_group.py

+6-1
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,12 @@ def consumer_thread(i):
9595
generations = set([consumer._coordinator.generation
9696
for consumer in list(consumers.values())])
9797

98-
if len(generations) == 1:
98+
# New generation assignment is not complete until
99+
# coordinator.rejoining = False
100+
rejoining = any([consumer._coordinator.rejoining
101+
for consumer in list(consumers.values())])
102+
103+
if not rejoining and len(generations) == 1:
99104
for c, consumer in list(consumers.items()):
100105
logging.info("[%s] %s %s: %s", c,
101106
consumer._coordinator.generation,

0 commit comments

Comments
 (0)