Merge pull request #516 from dpkp/group_id_none

Support group_id=None to disable offset commits and group membership
This commit is contained in:
Dana Powers
2016-01-24 21:24:21 -08:00
3 changed files with 46 additions and 28 deletions

View File

@@ -42,9 +42,11 @@ class KafkaConsumer(six.Iterator):
server-side log entries that correspond to this client. Also server-side log entries that correspond to this client. Also
submitted to GroupCoordinator for logging with respect to submitted to GroupCoordinator for logging with respect to
consumer group administration. Default: 'kafka-python-{version}' consumer group administration. Default: 'kafka-python-{version}'
group_id (str): name of the consumer group to join for dynamic group_id (str or None): name of the consumer group to join for dynamic
partition assignment (if enabled), and to use for fetching and partition assignment (if enabled), and to use for fetching and
committing offsets. Default: 'kafka-python-default-group' committing offsets. If None, auto-partition assignment (via
group coordinator) and offset commits are disabled.
Default: 'kafka-python-default-group'
key_deserializer (callable): Any callable that takes a key_deserializer (callable): Any callable that takes a
raw message key and returns a deserialized key. raw message key and returns a deserialized key.
value_deserializer (callable, optional): Any callable that takes a value_deserializer (callable, optional): Any callable that takes a
@@ -283,7 +285,8 @@ class KafkaConsumer(six.Iterator):
Returns: Returns:
kafka.future.Future kafka.future.Future
""" """
assert self.config['api_version'] >= (0, 8, 1) assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1'
assert self.config['group_id'] is not None, 'Requires group_id'
if offsets is None: if offsets is None:
offsets = self._subscription.all_consumed_offsets() offsets = self._subscription.all_consumed_offsets()
log.debug("Committing offsets: %s", offsets) log.debug("Committing offsets: %s", offsets)
@@ -309,7 +312,8 @@ class KafkaConsumer(six.Iterator):
to commit with the configured group_id. Defaults to current to commit with the configured group_id. Defaults to current
consumed offsets for all subscribed partitions. consumed offsets for all subscribed partitions.
""" """
assert self.config['api_version'] >= (0, 8, 1) assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1'
assert self.config['group_id'] is not None, 'Requires group_id'
if offsets is None: if offsets is None:
offsets = self._subscription.all_consumed_offsets() offsets = self._subscription.all_consumed_offsets()
self._coordinator.commit_offsets_sync(offsets) self._coordinator.commit_offsets_sync(offsets)
@@ -330,7 +334,8 @@ class KafkaConsumer(six.Iterator):
Returns: Returns:
The last committed offset, or None if there was no prior commit. The last committed offset, or None if there was no prior commit.
""" """
assert self.config['api_version'] >= (0, 8, 1) assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1'
assert self.config['group_id'] is not None, 'Requires group_id'
if self._subscription.is_assigned(partition): if self._subscription.is_assigned(partition):
committed = self._subscription.assignment[partition].committed committed = self._subscription.assignment[partition].committed
if committed is None: if committed is None:
@@ -418,14 +423,14 @@ class KafkaConsumer(six.Iterator):
Returns: Returns:
dict: map of topic to list of records (may be empty) dict: map of topic to list of records (may be empty)
""" """
if self.config['api_version'] >= (0, 8, 2): if self.config['group_id'] is not None:
# TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) if self.config['api_version'] >= (0, 8, 2):
self._coordinator.ensure_coordinator_known() self._coordinator.ensure_coordinator_known()
if self.config['api_version'] >= (0, 9): if self.config['api_version'] >= (0, 9):
# ensure we have partitions assigned if we expect to # ensure we have partitions assigned if we expect to
if self._subscription.partitions_auto_assigned(): if self._subscription.partitions_auto_assigned():
self._coordinator.ensure_active_group() self._coordinator.ensure_active_group()
# fetch positions if we have partitions we're subscribed to that we # fetch positions if we have partitions we're subscribed to that we
# don't know the offset for # don't know the offset for
@@ -603,7 +608,9 @@ class KafkaConsumer(six.Iterator):
NoOffsetForPartitionError: If no offset is stored for a given NoOffsetForPartitionError: If no offset is stored for a given
partition and no offset reset policy is defined partition and no offset reset policy is defined
""" """
if self.config['api_version'] >= (0, 8, 1): if (self.config['api_version'] >= (0, 8, 1)
and self.config['group_id'] is not None):
# refresh commits for all assigned partitions # refresh commits for all assigned partitions
self._coordinator.refresh_committed_offsets_if_needed() self._coordinator.refresh_committed_offsets_if_needed()
@@ -613,13 +620,14 @@ class KafkaConsumer(six.Iterator):
def _message_generator(self): def _message_generator(self):
assert self.assignment() or self.subscription() is not None assert self.assignment() or self.subscription() is not None
while time.time() < self._consumer_timeout: while time.time() < self._consumer_timeout:
if self.config['api_version'] >= (0, 8, 2): if self.config['group_id'] is not None:
self._coordinator.ensure_coordinator_known() if self.config['api_version'] >= (0, 8, 2):
self._coordinator.ensure_coordinator_known()
if self.config['api_version'] >= (0, 9): if self.config['api_version'] >= (0, 9):
# ensure we have partitions assigned if we expect to # ensure we have partitions assigned if we expect to
if self._subscription.partitions_auto_assigned(): if self._subscription.partitions_auto_assigned():
self._coordinator.ensure_active_group() self._coordinator.ensure_active_group()
# fetch positions if we have partitions we're subscribed to that we # fetch positions if we have partitions we're subscribed to that we
# don't know the offset for # don't know the offset for
@@ -634,7 +642,7 @@ class KafkaConsumer(six.Iterator):
self._client.cluster.ttl() / 1000.0 + time.time()) self._client.cluster.ttl() / 1000.0 + time.time())
if self.config['api_version'] >= (0, 9): if self.config['api_version'] >= (0, 9):
if not self.assignment(): if self.config['group_id'] is not None and not self.assignment():
sleep_time = time.time() - timeout_at sleep_time = time.time() - timeout_at
log.debug('No partitions assigned; sleeping for %s', sleep_time) log.debug('No partitions assigned; sleeping for %s', sleep_time)
time.sleep(sleep_time) time.sleep(sleep_time)

View File

@@ -75,18 +75,24 @@ class ConsumerCoordinator(BaseCoordinator):
if key in configs: if key in configs:
self.config[key] = configs[key] self.config[key] = configs[key]
self._cluster = client.cluster if self.config['api_version'] >= (0, 9) and self.config['group_id'] is not None:
assert self.config['assignors'], 'Coordinator requires assignors'
self._subscription = subscription self._subscription = subscription
self._partitions_per_topic = {} self._partitions_per_topic = {}
self._auto_commit_task = None self._cluster = client.cluster
if self.config['api_version'] >= (0, 9):
assert self.config['assignors'], 'Coordinator require assignors'
self._cluster.request_update() self._cluster.request_update()
self._cluster.add_listener(self._handle_metadata_update) self._cluster.add_listener(self._handle_metadata_update)
if self.config['api_version'] >= (0, 8, 1): self._auto_commit_task = None
if self.config['enable_auto_commit']: if self.config['enable_auto_commit']:
if self.config['api_version'] < (0, 8, 1):
log.warning('Broker version (%s) does not support offset'
' commits; disabling auto-commit.',
self.config['api_version'])
elif self.config['group_id'] is None:
log.warning('group_id is None: disabling auto-commit.')
else:
interval = self.config['auto_commit_interval_ms'] / 1000.0 interval = self.config['auto_commit_interval_ms'] / 1000.0
self._auto_commit_task = AutoCommitTask(self, interval) self._auto_commit_task = AutoCommitTask(self, interval)
@@ -127,7 +133,10 @@ class ConsumerCoordinator(BaseCoordinator):
# check if there are any changes to the metadata which should trigger # check if there are any changes to the metadata which should trigger
# a rebalance # a rebalance
if self._subscription_metadata_changed(): if self._subscription_metadata_changed():
if self.config['api_version'] >= (0, 9):
if (self.config['api_version'] >= (0, 9)
and self.config['group_id'] is not None):
self._subscription.mark_for_reassignment() self._subscription.mark_for_reassignment()
# If we haven't got group coordinator support, # If we haven't got group coordinator support,

View File

@@ -12,6 +12,7 @@ def test_end_to_end(kafka_broker):
max_block_ms=10000, max_block_ms=10000,
value_serializer=str.encode) value_serializer=str.encode)
consumer = KafkaConsumer(bootstrap_servers=connect_str, consumer = KafkaConsumer(bootstrap_servers=connect_str,
group_id=None,
consumer_timeout_ms=10000, consumer_timeout_ms=10000,
auto_offset_reset='earliest', auto_offset_reset='earliest',
value_deserializer=bytes.decode) value_deserializer=bytes.decode)