22import logging
33import re
44import sys
5+ from typing import Iterable , Optional , Pattern
56import traceback
67import warnings
78
@@ -133,6 +134,27 @@ class AIOKafkaConsumer:
133134 group member. If API methods block waiting for messages, that time
134135 does not count against this timeout. See `KIP-62`_ for more
135136 information. Default 300000
137+ listener (ConsumerRebalanceListener): Optionally include listener
138+ callback, which will be called before and after each rebalance
139+ operation.
140+ As part of group management, the consumer will keep track of
141+ the list of consumers that belong to a particular group and
142+ will trigger a rebalance operation if one of the following
143+ events trigger:
144+
145+ * Number of partitions change for any of the subscribed topics
146+ * Topic is created or deleted
147+ * An existing member of the consumer group dies
148+ * A new member is added to the consumer group
149+
150+ When any of these events are triggered, the provided listener
151+ will be invoked first to indicate that the consumer's
152+ assignment has been revoked, and then again when the new
153+ assignment has been received. Note that this listener will
154+ immediately override any listener set in a previous call
155+ to subscribe. It is guaranteed, however, that the partitions
156+ revoked/assigned
157+ through this interface are from topics subscribed in this call.
136158 rebalance_timeout_ms (int): The maximum time server will wait for this
137159 consumer to rejoin the group in a case of rebalance. In Java client
138160 this behaviour is bound to `max.poll.interval.ms` configuration,
@@ -242,6 +264,7 @@ def __init__(self, *topics, loop=None,
242264 metadata_max_age_ms = 5 * 60 * 1000 ,
243265 partition_assignment_strategy = (RoundRobinPartitionAssignor ,),
244266 max_poll_interval_ms = 300000 ,
267+ listener : Optional [ConsumerRebalanceListener ] = None ,
245268 rebalance_timeout_ms = None ,
246269 session_timeout_ms = 10000 ,
247270 heartbeat_interval_ms = 3000 ,
@@ -321,10 +344,14 @@ def __init__(self, *topics, loop=None,
321344 self ._source_traceback = traceback .extract_stack (sys ._getframe (1 ))
322345 self ._closed = False
323346
347+ if listener is not None :
348+ listener .consumer = self
349+ self .listener = listener
350+
324351 if topics :
325352 topics = self ._validate_topics (topics )
326353 self ._client .set_topics (topics )
327- self ._subscription . subscribe (topics = topics )
354+ self .subscribe (topics )
328355
329356 def __del__ (self , _warnings = warnings ):
330357 if self ._closed is False :
@@ -1008,7 +1035,12 @@ async def end_offsets(self, partitions):
10081035 partitions , self ._request_timeout_ms )
10091036 return offsets
10101037
1011- def subscribe (self , topics = (), pattern = None , listener = None ):
1038+ def subscribe (
1039+ self ,
1040+ topics : Iterable [str ] = (),
1041+ pattern : Optional [Pattern ] = None ,
1042+ listener : Optional [ConsumerRebalanceListener ] = None
1043+ ) -> None :
10121044 """ Subscribe to a list of topics, or a topic regex pattern.
10131045
10141046 Partitions will be dynamically assigned via a group coordinator.
@@ -1046,26 +1078,27 @@ def subscribe(self, topics=(), pattern=None, listener=None):
10461078 IllegalStateError: if called after previously calling :meth:`assign`
10471079 ValueError: if neither topics or pattern is provided or both
10481080 are provided
1049- TypeError: if listener is not a :class:`.ConsumerRebalanceListener`
10501081 """
10511082 if not (topics or pattern ):
10521083 raise ValueError (
10531084 "You should provide either `topics` or `pattern`" )
10541085 if topics and pattern :
10551086 raise ValueError (
10561087 "You can't provide both `topics` and `pattern`" )
1057- if listener is not None and \
1058- not isinstance (listener , ConsumerRebalanceListener ):
1059- raise TypeError (
1060- "listener should be an instance of ConsumerRebalanceListener" )
1088+
1089+ # Override the self.listener as the user wants this
1090+ if listener is not None :
1091+ listener .consumer = self
1092+ self .listener = listener
1093+
10611094 if pattern is not None :
10621095 try :
10631096 pattern = re .compile (pattern )
10641097 except re .error as err :
10651098 raise ValueError (
10661099 f"{ pattern !r} is not a valid pattern: { err } " )
10671100 self ._subscription .subscribe_pattern (
1068- pattern = pattern , listener = listener )
1101+ pattern = pattern , listener = self . listener )
10691102 # NOTE: set_topics will trigger a rebalance, so the coordinator
10701103 # will get the initial subscription shortly by ``metadata_changed``
10711104 # handler.
@@ -1074,7 +1107,7 @@ def subscribe(self, topics=(), pattern=None, listener=None):
10741107 elif topics :
10751108 topics = self ._validate_topics (topics )
10761109 self ._subscription .subscribe (
1077- topics = topics , listener = listener )
1110+ topics = topics , listener = self . listener )
10781111 self ._client .set_topics (self ._subscription .subscription .topics )
10791112 if self ._group_id is None :
10801113 # We have reset the assignment, but client.set_topics will
0 commit comments