Check api_version in ConsumerCoordinator

- Full group support in 0.9
  - Kafka-storage offsets w/ GroupCoordinator in 0.8.2
  - Zookeeper-storage offsets in 0.8.1
  - Assign all partitions locally if < 0.9
This commit is contained in:
Dana Powers
2016-01-03 16:03:30 -08:00
parent fae1a227b1
commit 5c45ec13f3

View File

@@ -9,7 +9,9 @@ from .base import BaseCoordinator
import kafka.common as Errors import kafka.common as Errors
from kafka.common import OffsetAndMetadata, TopicPartition from kafka.common import OffsetAndMetadata, TopicPartition
from kafka.future import Future from kafka.future import Future
from kafka.protocol.commit import OffsetCommitRequest_v2, OffsetFetchRequest_v1 from kafka.protocol.commit import (
OffsetCommitRequest_v2, OffsetCommitRequest_v1, OffsetCommitRequest_v0,
OffsetFetchRequest_v0, OffsetFetchRequest_v1)
from kafka.protocol.struct import Struct from kafka.protocol.struct import Struct
from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String
@@ -55,6 +57,7 @@ class ConsumerCoordinator(BaseCoordinator):
'session_timeout_ms': 30000, 'session_timeout_ms': 30000,
'heartbeat_interval_ms': 3000, 'heartbeat_interval_ms': 3000,
'retry_backoff_ms': 100, 'retry_backoff_ms': 100,
'api_version': (0, 9),
} }
def __init__(self, client, subscription, **configs): def __init__(self, client, subscription, **configs):
@@ -99,14 +102,16 @@ class ConsumerCoordinator(BaseCoordinator):
self._subscription = subscription self._subscription = subscription
self._partitions_per_topic = {} self._partitions_per_topic = {}
self._auto_commit_task = None self._auto_commit_task = None
assert self.config['assignors'], 'Coordinator require assignors' 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['enable_auto_commit']: if self.config['api_version'] >= (0, 8, 1):
interval = self.config['auto_commit_interval_ms'] / 1000.0 if self.config['enable_auto_commit']:
self._auto_commit_task = AutoCommitTask(self, interval) interval = self.config['auto_commit_interval_ms'] / 1000.0
self._auto_commit_task = AutoCommitTask(self, interval)
# metrics=None, # metrics=None,
# metric_group_prefix=None, # metric_group_prefix=None,
@@ -143,7 +148,17 @@ class ConsumerCoordinator(BaseCoordinator):
# check if there are any changes to the metadata which should trigger a rebalance # check if there are any changes to the metadata which should trigger a rebalance
if self._subscription_metadata_changed(): if self._subscription_metadata_changed():
self._subscription.mark_for_reassignment() if self.config['api_version'] >= (0, 9):
self._subscription.mark_for_reassignment()
# If we haven't got group coordinator support,
# just assign all partitions locally
else:
self._subscription.assign_from_subscribed([
TopicPartition(topic, partition)
for topic in self._subscription.subscription
for partition in self._partitions_per_topic[topic]
])
def _subscription_metadata_changed(self): def _subscription_metadata_changed(self):
if not self._subscription.partitions_auto_assigned(): if not self._subscription.partitions_auto_assigned():
@@ -273,7 +288,8 @@ class ConsumerCoordinator(BaseCoordinator):
dict: {TopicPartition: OffsetAndMetadata} dict: {TopicPartition: OffsetAndMetadata}
""" """
while True: while True:
self.ensure_coordinator_known() if self.config['api_version'] >= (0, 8, 2):
self.ensure_coordinator_known()
# contact coordinator to fetch committed offsets # contact coordinator to fetch committed offsets
future = self._send_offset_fetch_request(partitions) future = self._send_offset_fetch_request(partitions)
@@ -331,7 +347,8 @@ class ConsumerCoordinator(BaseCoordinator):
return return
while True: while True:
self.ensure_coordinator_known() if self.config['api_version'] >= (0, 8, 2):
self.ensure_coordinator_known()
future = self._send_offset_commit_request(offsets) future = self._send_offset_commit_request(offsets)
self._client.poll(future=future) self._client.poll(future=future)
@@ -345,6 +362,8 @@ class ConsumerCoordinator(BaseCoordinator):
time.sleep(self.config['retry_backoff_ms'] / 1000.0) time.sleep(self.config['retry_backoff_ms'] / 1000.0)
def _maybe_auto_commit_offsets_sync(self): def _maybe_auto_commit_offsets_sync(self):
if self.config['api_version'] < (0, 8, 1):
return
if self.config['enable_auto_commit']: if self.config['enable_auto_commit']:
# disable periodic commits prior to committing synchronously. note that they will # disable periodic commits prior to committing synchronously. note that they will
# be re-enabled after a rebalance completes # be re-enabled after a rebalance completes
@@ -379,8 +398,12 @@ class ConsumerCoordinator(BaseCoordinator):
Returns: Returns:
Future: indicating whether the commit was successful or not Future: indicating whether the commit was successful or not
""" """
if self.coordinator_unknown(): if self.config['api_version'] >= (0, 8, 2):
return Future().failure(Errors.GroupCoordinatorNotAvailableError) if self.coordinator_unknown():
return Future().failure(Errors.GroupCoordinatorNotAvailableError)
node_id = self.coordinator_id
else:
node_id = self._client.least_loaded_node()
if not offsets: if not offsets:
return Future().failure(None) return Future().failure(None)
@@ -390,25 +413,49 @@ class ConsumerCoordinator(BaseCoordinator):
for tp, offset in six.iteritems(offsets): for tp, offset in six.iteritems(offsets):
offset_data[tp.topic][tp.partition] = offset offset_data[tp.topic][tp.partition] = offset
request = OffsetCommitRequest_v2( if self.config['api_version'] >= (0, 9):
self.group_id, request = OffsetCommitRequest_v2(
self.generation, self.group_id,
self.member_id, self.generation,
OffsetCommitRequest_v2.DEFAULT_RETENTION_TIME, self.member_id,
[( OffsetCommitRequest_v2.DEFAULT_RETENTION_TIME,
topic, [( [(
partition, topic, [(
offset.offset, partition,
offset.metadata offset.offset,
) for partition, offset in six.iteritems(partitions)] offset.metadata
) for topic, partitions in six.iteritems(offset_data)] ) for partition, offset in six.iteritems(partitions)]
) ) for topic, partitions in six.iteritems(offset_data)]
)
elif self.config['api_version'] >= (0, 8, 2):
request = OffsetCommitRequest_v1(
self.group_id, -1, '',
[(
topic, [(
partition,
offset.offset,
-1,
offset.metadata
) for partition, offset in six.iteritems(partitions)]
) for topic, partitions in six.iteritems(offset_data)]
)
elif self.config['api_version'] >= (0, 8, 1):
request = OffsetCommitRequest_v0(
self.group_id,
[(
topic, [(
partition,
offset.offset,
offset.metadata
) for partition, offset in six.iteritems(partitions)]
) for topic, partitions in six.iteritems(offset_data)]
)
log.debug("Sending offset-commit request with %s to %s", log.debug("Sending offset-commit request with %s to %s",
offsets, self.coordinator_id) offsets, node_id)
future = Future() future = Future()
_f = self._client.send(self.coordinator_id, request) _f = self._client.send(node_id, request)
_f.add_callback(self._handle_offset_commit_response, offsets, future) _f.add_callback(self._handle_offset_commit_response, offsets, future)
_f.add_errback(self._failed_request, future) _f.add_errback(self._failed_request, future)
return future return future
@@ -495,22 +542,33 @@ class ConsumerCoordinator(BaseCoordinator):
Returns: Returns:
Future: resolves to dict of offsets: {TopicPartition: int} Future: resolves to dict of offsets: {TopicPartition: int}
""" """
if self.coordinator_unknown(): if self.config['api_version'] >= (0, 8, 2):
return Future().failure(Errors.GroupCoordinatorNotAvailableError) if self.coordinator_unknown():
return Future().failure(Errors.GroupCoordinatorNotAvailableError)
node_id = self.coordinator_id
else:
node_id = self._client.least_loaded_node()
log.debug("Fetching committed offsets for partitions: %s", partitions) log.debug("Fetching committed offsets for partitions: %s", partitions)
# construct the request # construct the request
topic_partitions = collections.defaultdict(set) topic_partitions = collections.defaultdict(set)
for tp in partitions: for tp in partitions:
topic_partitions[tp.topic].add(tp.partition) topic_partitions[tp.topic].add(tp.partition)
request = OffsetFetchRequest_v1(
self.group_id, if self.config['api_version'] >= (0, 8, 2):
list(topic_partitions.items()) request = OffsetFetchRequest_v1(
) self.group_id,
list(topic_partitions.items())
)
else:
request = OffsetFetchRequest_v0(
self.group_id,
list(topic_partitions.items())
)
# send the request with a callback # send the request with a callback
future = Future() future = Future()
_f = self._client.send(self.coordinator_id, request) _f = self._client.send(node_id, request)
_f.add_callback(self._handle_offset_fetch_response, future) _f.add_callback(self._handle_offset_fetch_response, future)
_f.add_errback(self._failed_request, future) _f.add_errback(self._failed_request, future)
return future return future
@@ -536,6 +594,10 @@ class ConsumerCoordinator(BaseCoordinator):
# need to re-join group # need to re-join group
self._subscription.mark_for_reassignment() self._subscription.mark_for_reassignment()
future.failure(error) future.failure(error)
elif error_type is Errors.UnknownTopicOrPartitionError:
log.warning("OffsetFetchRequest -- unknown topic %s",
topic)
continue
else: else:
log.error("Unknown error fetching offsets for %s: %s", log.error("Unknown error fetching offsets for %s: %s",
tp, error) tp, error)