From fa6738b8f49735df1812e2e9068e227fcaca961d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Sep 2014 00:12:38 -0700 Subject: [PATCH 01/33] A simpler kafka consumer: ``` kafka = KafkaConsumer('topic1') for m in kafka: print m kafka = KafkaConsumer('topic1', 'topic2', group_id='my_consumer_group', auto_commit_enable=True, auto_commit_interval_ms=30 * 1000, auto_offset_reset='smallest') for m in kafka: process_message(m) kafka.task_done(m) ``` --- kafka/common.py | 7 + kafka/consumer/new.py | 432 ++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 439 insertions(+) create mode 100644 kafka/consumer/new.py diff --git a/kafka/common.py b/kafka/common.py index 008736c33..2e817cb79 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -63,6 +63,9 @@ TopicAndPartition = namedtuple("TopicAndPartition", ["topic", "partition"]) +KafkaMessage = namedtuple("KafkaMessage", + ["topic", "partition", "offset", "key", "value"]) + ################# # Exceptions # @@ -182,6 +185,10 @@ class ConsumerNoMoreData(KafkaError): pass +class ConsumerTimeout(KafkaError): + pass + + class ProtocolError(KafkaError): pass diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py new file mode 100644 index 000000000..5790a31d4 --- /dev/null +++ b/kafka/consumer/new.py @@ -0,0 +1,432 @@ +from collections import defaultdict, namedtuple +from copy import deepcopy +import logging +import sys +import time + +from kafka.client import KafkaClient +from kafka.common import ( + OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, + check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, + OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout +) + +logger = logging.getLogger(__name__) + +OffsetsStruct = namedtuple("OffsetsStruct", ["fetch", "highwater", "commit", "task_done"]) + + +class KafkaConsumer(object): + """ + A simpler kafka consumer + + ``` + # A very basic 'tail' consumer, with no stored offset management + kafka = KafkaConsumer('topic1') + for m in kafka: + print m + + # Alternate interface: next() + while True: + print kafka.next() + + # Alternate interface: batch iteration + while True: + for m in kafka.fetch_messages(): + print m + print "Done with batch - let's do another!" + ``` + + ``` + # more advanced consumer -- multiple topics w/ auto commit offset management + kafka = KafkaConsumer('topic1', 'topic2', + group_id='my_consumer_group', + auto_commit_enable=True, + auto_commit_interval_ms=30 * 1000, + auto_offset_reset='smallest') + + # Infinite iteration + for m in kafka: + process_message(m) + kafka.task_done(m) + + # Alternate interface: next() + while True: + m = kafka.next() + process_message(m) + kafka.task_done(m) + + # Batch process interface does not auto_commit! + while True: + for m in kafka.fetch_messages(): + process_message(m) + kafka.task_done(m) + kafka.commit() + ``` + + messages (m) are namedtuples with attributes: + m.topic: topic name (str) + m.partition: partition number (int) + m.offset: message offset on topic-partition log (int) + m.key: key (bytes - can be None) + m.value: message (output of deserializer_class - default is event object) + + Configuration settings can be passed to constructor, + otherwise defaults will be used: + client_id='kafka.consumer.XXX', + group_id=None, + fetch_message_max_bytes=1024*1024, + fetch_min_bytes=1, + fetch_wait_max_ms=100, + refresh_leader_backoff_ms=200, + metadata_broker_list=None, + socket_timeout_ms=30*1000, + auto_offset_reset='largest', + deserializer_class=Event.from_bytes, + auto_commit_enable=False, + auto_commit_interval_ms=60 * 1000, + consumer_timeout_ms=-1, + + + Configuration parameters are described in more detail at + http://kafka.apache.org/documentation.html#highlevelconsumerapi + + Message fetching proceeds in batches, with each topic/partition + queried for a maximum of `fetch_message_max_bytes` data + After consuming all messages in a batch, StopIteration is raised + Iterating again after StopIteration will trigger another batch to be fetched + """ + + DEFAULT_CONSUMER_CONFIG = { + 'client_id': __name__, + 'group_id': None, + 'metadata_broker_list': None, + 'socket_timeout_ms': 30 * 1000, + 'fetch_message_max_bytes': 1024 * 1024, + 'auto_offset_reset': 'largest', + 'fetch_min_bytes': 1, + 'fetch_wait_max_ms': 100, + 'refresh_leader_backoff_ms': 200, + 'deserializer_class': lambda msg: msg, + 'auto_commit_enable': False, + 'auto_commit_interval_ms': 60 * 1000, + 'consumer_timeout_ms': -1, + + # Currently unused + 'socket_receive_buffer_bytes': 64 * 1024, + 'refresh_leader_backoff_ms': 200, + 'num_consumer_fetchers': 1, + 'default_fetcher_backoff_ms': 1000, + 'queued_max_message_chunks': 10, + 'rebalance_max_retries': 4, + 'rebalance_backoff_ms': 2000, + } + + def __init__(self, *topics, **configs): + self.topics = topics + self.config = configs + self.client = KafkaClient(self._get_config('metadata_broker_list'), + client_id=self._get_config('client_id'), + timeout=(self._get_config('socket_timeout_ms') / 1000.0)) + + # Get initial topic metadata + self.client.load_metadata_for_topics() + for topic in self.topics: + if topic not in self.client.topic_partitions: + raise ValueError("Topic %s not found in broker metadata" % topic) + logger.info("Configuring consumer to fetch topic '%s'", topic) + + # Check auto-commit configuration + if self._get_config('auto_commit_enable'): + if not self._get_config('group_id'): + raise RuntimeError('KafkaConsumer configured to auto-commit without required consumer group (group_id)') + + logger.info("Configuring consumer to auto-commit offsets") + self._set_next_auto_commit_time() + + # Setup offsets + self._offsets = OffsetsStruct(fetch=defaultdict(dict), + commit=defaultdict(dict), + highwater= defaultdict(dict), + task_done=defaultdict(dict)) + + # If we have a consumer group, try to fetch stored offsets + if self._get_config('group_id'): + self._fetch_stored_offsets() + else: + self._auto_reset_offsets() + + # highwater marks (received from server on fetch response) + # and task_done (set locally by user) + # should always get initialized to None + self._reset_highwater_offsets() + self._reset_task_done_offsets() + + # Start the message fetch generator + self._msg_iter = self.fetch_messages() + + def _fetch_stored_offsets(self): + logger.info("Consumer fetching stored offsets") + for topic in self.topics: + for partition in self.client.topic_partitions[topic]: + + (resp,) = self.client.send_offset_fetch_request( + self._get_config('group_id'), + [OffsetFetchRequest(topic, partition)], + fail_on_error=False) + try: + check_error(resp) + # API spec says server wont set an error here + # but 0.8.1.1 does actually... + except UnknownTopicOrPartitionError: + pass + + # -1 offset signals no commit is currently stored + if resp.offset == -1: + self._offsets.commit[topic][partition] = None + self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + + # Otherwise we committed the stored offset + # and need to fetch the next one + else: + self._offsets.commit[topic][partition] = resp.offset + self._offsets.fetch[topic][partition] = resp.offset + + def _auto_reset_offsets(self): + logger.info("Consumer auto-resetting offsets") + for topic in self.topics: + for partition in self.client.topic_partitions[topic]: + + self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + self._offsets.commit[topic][partition] = None + + def _reset_highwater_offsets(self): + for topic in self.topics: + for partition in self.client.topic_partitions[topic]: + self._offsets.highwater[topic][partition] = None + + def _reset_task_done_offsets(self): + for topic in self.topics: + for partition in self.client.topic_partitions[topic]: + self._offsets.task_done[topic][partition] = None + + def __repr__(self): + return '' % ', '.join(self.topics) + + def __iter__(self): + return self + + def next(self): + consumer_timeout = False + if self._get_config('consumer_timeout_ms') >= 0: + consumer_timeout = time.time() + (self._get_config('consumer_timeout_ms') / 1000.0) + + while True: + + # Check for auto-commit + if self.should_auto_commit(): + self.commit() + + try: + return self._msg_iter.next() + + # If the previous batch finishes, start get new batch + except StopIteration: + self._msg_iter = self.fetch_messages() + + if consumer_timeout and time.time() > consumer_timeout: + raise ConsumerTimeout('Consumer timed out waiting to fetch messages') + + def offsets(self, group): + return dict(deepcopy(getattr(self._offsets, group))) + + def task_done(self, message): + topic = message.topic + partition = message.partition + offset = message.offset + + # Warn on non-contiguous offsets + prev_done = self._offsets.task_done[topic][partition] + if prev_done is not None and offset != (prev_done + 1): + logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', + offset, prev_done) + + # Warn on smaller offsets than previous commit + # "commit" offsets are actually the offset of the next # message to fetch. + # so task_done should be compared with (commit - 1) + prev_done = (self._offsets.commit[topic][partition] - 1) + if prev_done is not None and (offset <= prev_done): + logger.warning('Marking task_done on a previously committed offset?: %d <= %d', + offset, prev_done) + + self._offsets.task_done[topic][partition] = offset + + def should_auto_commit(self): + if not self._get_config('auto_commit_enable'): + return False + + if not self._next_commit: + return False + + return (time.time() >= self._next_commit) + + def _set_next_auto_commit_time(self): + self._next_commit = time.time() + (self._get_config('auto_commit_interval_ms') / 1000.0) + + def commit(self): + if not self._get_config('group_id'): + logger.warning('Cannot commit without a group_id!') + raise RuntimeError('Attempted to commit offsets without a configured consumer group (group_id)') + + # API supports storing metadata with each commit + # but for now it is unused + metadata = '' + + offsets = self._offsets.task_done + commits = [] + for topic, partitions in offsets.iteritems(): + for partition, task_done in partitions.iteritems(): + + # Skip if None + if task_done is None: + continue + + # Commit offsets as the next offset to fetch + # which is consistent with the Java Client + # task_done is marked by messages consumed, + # so add one to mark the next message for fetching + commit_offset = (task_done + 1) + + # Skip if no change from previous committed + if commit_offset == self._offsets.commit[topic][partition]: + continue + + commits.append(OffsetCommitRequest(topic, partition, commit_offset, metadata)) + + if commits: + logger.info('committing consumer offsets to group %s', self._get_config('group_id')) + resps = self.client.send_offset_commit_request(self._get_config('group_id'), + commits, + fail_on_error=False) + + for r in resps: + check_error(r) + task_done = self._offsets.task_done[r.topic][r.partition] + self._offsets.commit[r.topic][r.partition] = (task_done + 1) + + if self._get_config('auto_commit_enable'): + self._set_next_auto_commit_time() + + return True + + else: + logger.info('No new offsets found to commit in group %s', self._get_config('group_id')) + return False + + def _get_config(self, key): + return self.config.get(key, self.DEFAULT_CONSUMER_CONFIG[key]) + + def fetch_messages(self): + + max_bytes = self._get_config('fetch_message_max_bytes') + max_wait_time = self._get_config('fetch_wait_max_ms') + min_bytes = self._get_config('fetch_min_bytes') + + fetches = [] + offsets = self._offsets.fetch + for topic, partitions in offsets.iteritems(): + for partition, offset in partitions.iteritems(): + fetches.append(FetchRequest(topic, partition, offset, max_bytes)) + + # client.send_fetch_request will collect topic/partition requests by leader + # and send each group as a single FetchRequest to the correct broker + responses = self.client.send_fetch_request(fetches, + max_wait_time=max_wait_time, + min_bytes=min_bytes, + fail_on_error=False) + + for resp in responses: + topic = resp.topic + partition = resp.partition + try: + check_error(resp) + except OffsetOutOfRangeError: + logger.warning('OffsetOutOfRange: topic %s, partition %d, offset %d ' + '(Highwatermark: %d)', + topic, partition, offsets[topic][partition], + resp.highwaterMark) + # Reset offset + self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + continue + + except NotLeaderForPartitionError: + logger.warning("NotLeaderForPartitionError for %s - %d. " + "Metadata may be out of date", + topic, partition) + sleep_ms = self._get_config('refresh_leader_backoff_ms') + logger.warning("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) + time.sleep(sleep_ms / 1000.0) + self.client.load_metadata_for_topics() + continue + + except RequestTimedOutError: + logger.warning("RequestTimedOutError for %s - %d", topic, partition) + continue + + # Track server highwater mark + self._offsets.highwater[topic][partition] = resp.highwaterMark + + # Yield each message + # Kafka-python could raise an exception during iteration + # we are not catching -- user will need to address + for (offset, message) in resp.messages: + # deserializer_class could raise an exception here + msg = KafkaMessage(topic, partition, offset, message.key, + self._get_config('deserializer_class')(message.value)) + + # Only increment fetch offset if we safely got the message and deserialized + self._offsets.fetch[topic][partition] = offset + 1 + + # Then yield to user + yield msg + + def _reset_partition_offset(self, topic, partition): + LATEST = -1 + EARLIEST = -2 + + RequestTime = None + if self._get_config('auto_offset_reset') == 'largest': + RequestTime = LATEST + elif self._get_config('auto_offset_reset') == 'smallest': + RequestTime = EARLIEST + else: + + # Let's raise an reasonable exception type if user calls + # outside of an exception context + if sys.exc_info() == (None, None, None): + raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' + 'valid auto_offset_reset setting ' + '(largest|smallest)') + + # Otherwise we should re-raise the upstream exception + # b/c it typically includes additional data about + # the request that triggered it, and we do not want to drop that + raise + + (offset, ) = self.get_partition_offsets(topic, partition, RequestTime, + num_offsets=1) + return offset + + def get_partition_offsets(self, topic, partition, request_time, num_offsets): + reqs = [OffsetRequest(topic, partition, request_time, num_offsets)] + + (resp,) = self.client.send_offset_request(reqs) + + check_error(resp) + + # Just for sanity.. + # probably unnecessary + assert resp.topic == topic + assert resp.partition == partition + + return resp.offsets From 82b3e011fad44c92188ce7645738dea691fa5849 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Sep 2014 11:41:51 -0700 Subject: [PATCH 02/33] Handle FailedPayloadsError on client.send_fetch_request; permit offsets(); update docstring --- kafka/consumer/new.py | 51 +++++++++++++++++++++++++++++-------------- 1 file changed, 35 insertions(+), 16 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 5790a31d4..5ef592163 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -8,7 +8,8 @@ from kafka.common import ( OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, - OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout + OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, + FailedPayloadsError, KafkaUnavailableError ) logger = logging.getLogger(__name__) @@ -90,11 +91,6 @@ class KafkaConsumer(object): Configuration parameters are described in more detail at http://kafka.apache.org/documentation.html#highlevelconsumerapi - - Message fetching proceeds in batches, with each topic/partition - queried for a maximum of `fetch_message_max_bytes` data - After consuming all messages in a batch, StopIteration is raised - Iterating again after StopIteration will trigger another batch to be fetched """ DEFAULT_CONSUMER_CONFIG = { @@ -237,8 +233,16 @@ def next(self): if consumer_timeout and time.time() > consumer_timeout: raise ConsumerTimeout('Consumer timed out waiting to fetch messages') - def offsets(self, group): - return dict(deepcopy(getattr(self._offsets, group))) + def offsets(self, group=None): + if not group: + return { + 'fetch': self.offsets('fetch'), + 'commit': self.offsets('commit'), + 'task_done': self.offsets('task_done'), + 'highwater': self.offsets('highwater') + } + else: + return dict(deepcopy(getattr(self._offsets, group))) def task_done(self, message): topic = message.topic @@ -340,10 +344,15 @@ def fetch_messages(self): # client.send_fetch_request will collect topic/partition requests by leader # and send each group as a single FetchRequest to the correct broker - responses = self.client.send_fetch_request(fetches, - max_wait_time=max_wait_time, - min_bytes=min_bytes, - fail_on_error=False) + try: + responses = self.client.send_fetch_request(fetches, + max_wait_time=max_wait_time, + min_bytes=min_bytes, + fail_on_error=False) + except FailedPayloadsError: + logger.warning('FailedPayloadsError attempting to fetch data from kafka') + self._refresh_metadata_on_error() + return for resp in responses: topic = resp.topic @@ -363,10 +372,7 @@ def fetch_messages(self): logger.warning("NotLeaderForPartitionError for %s - %d. " "Metadata may be out of date", topic, partition) - sleep_ms = self._get_config('refresh_leader_backoff_ms') - logger.warning("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) - time.sleep(sleep_ms / 1000.0) - self.client.load_metadata_for_topics() + self._refresh_metadata_on_error() continue except RequestTimedOutError: @@ -430,3 +436,16 @@ def get_partition_offsets(self, topic, partition, request_time, num_offsets): assert resp.partition == partition return resp.offsets + + def _refresh_metadata_on_error(self): + sleep_ms = self._get_config('refresh_leader_backoff_ms') + while True: + logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) + time.sleep(sleep_ms / 1000.0) + try: + self.client.load_metadata_for_topics() + except KafkaUnavailableError: + logger.warning("Unable to refresh topic metadata... cluster unavailable") + else: + logger.info("Topic metadata refreshed") + return From b264d8f51751f9fc81cfe8e0fef0606dd877a8db Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Sep 2014 11:44:53 -0700 Subject: [PATCH 03/33] add private methods _set_consumer_timeout_start() and _check_consumer_timeout() --- kafka/consumer/new.py | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 5ef592163..b579bfc6c 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -86,7 +86,7 @@ class KafkaConsumer(object): deserializer_class=Event.from_bytes, auto_commit_enable=False, auto_commit_interval_ms=60 * 1000, - consumer_timeout_ms=-1, + consumer_timeout_ms=-1 Configuration parameters are described in more detail at @@ -110,7 +110,6 @@ class KafkaConsumer(object): # Currently unused 'socket_receive_buffer_bytes': 64 * 1024, - 'refresh_leader_backoff_ms': 200, 'num_consumer_fetchers': 1, 'default_fetcher_backoff_ms': 1000, 'queued_max_message_chunks': 10, @@ -213,10 +212,7 @@ def __iter__(self): return self def next(self): - consumer_timeout = False - if self._get_config('consumer_timeout_ms') >= 0: - consumer_timeout = time.time() + (self._get_config('consumer_timeout_ms') / 1000.0) - + self._set_consumer_timeout_start() while True: # Check for auto-commit @@ -230,8 +226,7 @@ def next(self): except StopIteration: self._msg_iter = self.fetch_messages() - if consumer_timeout and time.time() > consumer_timeout: - raise ConsumerTimeout('Consumer timed out waiting to fetch messages') + self._check_consumer_timeout() def offsets(self, group=None): if not group: @@ -446,6 +441,16 @@ def _refresh_metadata_on_error(self): self.client.load_metadata_for_topics() except KafkaUnavailableError: logger.warning("Unable to refresh topic metadata... cluster unavailable") + self._check_consumer_timeout() else: logger.info("Topic metadata refreshed") return + + def _set_consumer_timeout_start(self): + self._consumer_timeout = False + if self._get_config('consumer_timeout_ms') >= 0: + self._consumer_timeout = time.time() + (self._get_config('consumer_timeout_ms') / 1000.0) + + def _check_consumer_timeout(self): + if self._consumer_timeout and time.time() > self._consumer_timeout: + raise ConsumerTimeout('Consumer timed out after %d ms' % + self._get_config('consumer_timeout_ms')) From 53763510c77b252b103ed2a1b7cdba8e527ba7f6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 13 Sep 2014 19:34:08 -0700 Subject: [PATCH 04/33] Use configure() to check and set configuration keys --- kafka/common.py | 4 ++ kafka/consumer/new.py | 126 ++++++++++++++++++++++-------------------- 2 files changed, 71 insertions(+), 59 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 2e817cb79..e4b3b1bd1 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -197,6 +197,10 @@ class UnsupportedCodecError(KafkaError): pass +class KafkaConfigurationError(KafkaError): + pass + + kafka_errors = { -1 : UnknownError, 0 : NoError, diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index b579bfc6c..a192b5848 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -9,13 +9,37 @@ OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, - FailedPayloadsError, KafkaUnavailableError + FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError ) logger = logging.getLogger(__name__) OffsetsStruct = namedtuple("OffsetsStruct", ["fetch", "highwater", "commit", "task_done"]) +DEFAULT_CONSUMER_CONFIG = { + 'client_id': __name__, + 'group_id': None, + 'metadata_broker_list': None, + 'socket_timeout_ms': 30 * 1000, + 'fetch_message_max_bytes': 1024 * 1024, + 'auto_offset_reset': 'largest', + 'fetch_min_bytes': 1, + 'fetch_wait_max_ms': 100, + 'refresh_leader_backoff_ms': 200, + 'deserializer_class': lambda msg: msg, + 'auto_commit_enable': False, + 'auto_commit_interval_ms': 60 * 1000, + 'consumer_timeout_ms': -1, + + # Currently unused + 'socket_receive_buffer_bytes': 64 * 1024, + 'num_consumer_fetchers': 1, + 'default_fetcher_backoff_ms': 1000, + 'queued_max_message_chunks': 10, + 'rebalance_max_retries': 4, + 'rebalance_backoff_ms': 2000, +} + class KafkaConsumer(object): """ @@ -93,52 +117,17 @@ class KafkaConsumer(object): http://kafka.apache.org/documentation.html#highlevelconsumerapi """ - DEFAULT_CONSUMER_CONFIG = { - 'client_id': __name__, - 'group_id': None, - 'metadata_broker_list': None, - 'socket_timeout_ms': 30 * 1000, - 'fetch_message_max_bytes': 1024 * 1024, - 'auto_offset_reset': 'largest', - 'fetch_min_bytes': 1, - 'fetch_wait_max_ms': 100, - 'refresh_leader_backoff_ms': 200, - 'deserializer_class': lambda msg: msg, - 'auto_commit_enable': False, - 'auto_commit_interval_ms': 60 * 1000, - 'consumer_timeout_ms': -1, - - # Currently unused - 'socket_receive_buffer_bytes': 64 * 1024, - 'num_consumer_fetchers': 1, - 'default_fetcher_backoff_ms': 1000, - 'queued_max_message_chunks': 10, - 'rebalance_max_retries': 4, - 'rebalance_backoff_ms': 2000, - } - def __init__(self, *topics, **configs): self.topics = topics - self.config = configs - self.client = KafkaClient(self._get_config('metadata_broker_list'), - client_id=self._get_config('client_id'), - timeout=(self._get_config('socket_timeout_ms') / 1000.0)) + self.configure(**configs) # Get initial topic metadata self.client.load_metadata_for_topics() for topic in self.topics: if topic not in self.client.topic_partitions: - raise ValueError("Topic %s not found in broker metadata" % topic) + raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) logger.info("Configuring consumer to fetch topic '%s'", topic) - # Check auto-commit configuration - if self._get_config('auto_commit_enable'): - if not self._get_config('group_id'): - raise RuntimeError('KafkaConsumer configured to auto-commit without required consumer group (group_id)') - - logger.info("Configuring consumer to auto-commit offsets") - self._set_next_auto_commit_time() - # Setup offsets self._offsets = OffsetsStruct(fetch=defaultdict(dict), commit=defaultdict(dict), @@ -146,7 +135,7 @@ def __init__(self, *topics, **configs): task_done=defaultdict(dict)) # If we have a consumer group, try to fetch stored offsets - if self._get_config('group_id'): + if self._config['group_id']: self._fetch_stored_offsets() else: self._auto_reset_offsets() @@ -166,7 +155,7 @@ def _fetch_stored_offsets(self): for partition in self.client.topic_partitions[topic]: (resp,) = self.client.send_offset_fetch_request( - self._get_config('group_id'), + self._config['group_id'], [OffsetFetchRequest(topic, partition)], fail_on_error=False) try: @@ -261,7 +250,7 @@ def task_done(self, message): self._offsets.task_done[topic][partition] = offset def should_auto_commit(self): - if not self._get_config('auto_commit_enable'): + if not self._config['auto_commit_enable']: return False if not self._next_commit: @@ -270,10 +259,10 @@ def should_auto_commit(self): return (time.time() >= self._next_commit) def _set_next_auto_commit_time(self): - self._next_commit = time.time() + (self._get_config('auto_commit_interval_ms') / 1000.0) + self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) def commit(self): - if not self._get_config('group_id'): + if not self._config['group_id']: logger.warning('Cannot commit without a group_id!') raise RuntimeError('Attempted to commit offsets without a configured consumer group (group_id)') @@ -303,8 +292,8 @@ def commit(self): commits.append(OffsetCommitRequest(topic, partition, commit_offset, metadata)) if commits: - logger.info('committing consumer offsets to group %s', self._get_config('group_id')) - resps = self.client.send_offset_commit_request(self._get_config('group_id'), + logger.info('committing consumer offsets to group %s', self._config['group_id']) + resps = self.client.send_offset_commit_request(self._config['group_id'], commits, fail_on_error=False) @@ -313,23 +302,42 @@ def commit(self): task_done = self._offsets.task_done[r.topic][r.partition] self._offsets.commit[r.topic][r.partition] = (task_done + 1) - if self._get_config('auto_commit_enable'): + if self._config['auto_commit_enable']: self._set_next_auto_commit_time() return True else: - logger.info('No new offsets found to commit in group %s', self._get_config('group_id')) + logger.info('No new offsets found to commit in group %s', self._config['group_id']) return False - def _get_config(self, key): - return self.config.get(key, self.DEFAULT_CONSUMER_CONFIG[key]) + def configure(self, **configs): + self._config = {} + for key in DEFAULT_CONSUMER_CONFIG: + self._config[key] = configs.pop(key, DEFAULT_CONSUMER_CONFIG[key]) + + if configs: + raise KafkaConfigurationError('Unknown configuration key(s): ' + + str(list(configs.keys()))) + + if self._config['auto_commit_enable']: + if not self._config['group_id']: + raise KafkaConfigurationError('KafkaConsumer configured to auto-commit without required consumer group (group_id)') + + # Check auto-commit configuration + if self._config['auto_commit_enable']: + logger.info("Configuring consumer to auto-commit offsets") + self._set_next_auto_commit_time() + + self.client = KafkaClient(self._config['metadata_broker_list'], + client_id=self._config['client_id'], + timeout=(self._config['socket_timeout_ms'] / 1000.0)) def fetch_messages(self): - max_bytes = self._get_config('fetch_message_max_bytes') - max_wait_time = self._get_config('fetch_wait_max_ms') - min_bytes = self._get_config('fetch_min_bytes') + max_bytes = self._config['fetch_message_max_bytes'] + max_wait_time = self._config['fetch_wait_max_ms'] + min_bytes = self._config['fetch_min_bytes'] fetches = [] offsets = self._offsets.fetch @@ -383,7 +391,7 @@ def fetch_messages(self): for (offset, message) in resp.messages: # deserializer_class could raise an exception here msg = KafkaMessage(topic, partition, offset, message.key, - self._get_config('deserializer_class')(message.value)) + self._config['deserializer_class'](message.value)) # Only increment fetch offset if we safely got the message and deserialized self._offsets.fetch[topic][partition] = offset + 1 @@ -396,9 +404,9 @@ def _reset_partition_offset(self, topic, partition): EARLIEST = -2 RequestTime = None - if self._get_config('auto_offset_reset') == 'largest': + if self._config['auto_offset_reset'] == 'largest': RequestTime = LATEST - elif self._get_config('auto_offset_reset') == 'smallest': + elif self._config['auto_offset_reset'] == 'smallest': RequestTime = EARLIEST else: @@ -433,7 +441,7 @@ def get_partition_offsets(self, topic, partition, request_time, num_offsets): return resp.offsets def _refresh_metadata_on_error(self): - sleep_ms = self._get_config('refresh_leader_backoff_ms') + sleep_ms = self._config['refresh_leader_backoff_ms'] while True: logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) time.sleep(sleep_ms / 1000.0) @@ -448,9 +456,9 @@ def _refresh_metadata_on_error(self): def _set_consumer_timeout_start(self): self._consumer_timeout = False - if self._get_config('consumer_timeout_ms') >= 0: - self._consumer_timeout = time.time() + (self._get_config('consumer_timeout_ms') / 1000.0) + if self._config['consumer_timeout_ms'] >= 0: + self._consumer_timeout = time.time() + (self._config['consumer_timeout_ms'] / 1000.0) def _check_consumer_timeout(self): if self._consumer_timeout and time.time() > self._consumer_timeout: - raise ConsumerTimeout('Consumer timed out after %d ms' % + self._get_config('consumer_timeout_ms')) + raise ConsumerTimeout('Consumer timed out after %d ms' % + self._config['consumer_timeout_ms']) From 4c111d3d7c760d9e67be5251eef1df02a64f33c7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 13 Sep 2014 19:34:44 -0700 Subject: [PATCH 05/33] Use client.get_partition_ids_for_topic --- kafka/consumer/new.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index a192b5848..80fdcecda 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -152,7 +152,7 @@ def __init__(self, *topics, **configs): def _fetch_stored_offsets(self): logger.info("Consumer fetching stored offsets") for topic in self.topics: - for partition in self.client.topic_partitions[topic]: + for partition in self.client.get_partition_ids_for_topic(topic): (resp,) = self.client.send_offset_fetch_request( self._config['group_id'], @@ -179,19 +179,19 @@ def _fetch_stored_offsets(self): def _auto_reset_offsets(self): logger.info("Consumer auto-resetting offsets") for topic in self.topics: - for partition in self.client.topic_partitions[topic]: + for partition in self.client.get_partition_ids_for_topic(topic): self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) self._offsets.commit[topic][partition] = None def _reset_highwater_offsets(self): for topic in self.topics: - for partition in self.client.topic_partitions[topic]: + for partition in self.client.get_partition_ids_for_topic(topic): self._offsets.highwater[topic][partition] = None def _reset_task_done_offsets(self): for topic in self.topics: - for partition in self.client.topic_partitions[topic]: + for partition in self.client.get_partition_ids_for_topic(topic): self._offsets.task_done[topic][partition] = None def __repr__(self): From 62a71892a687d99bf0076ed3a3d9c614f16a112c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 10:07:58 -0700 Subject: [PATCH 06/33] Add set_topic_partitions method to configure topics/partitions to consume --- kafka/consumer/new.py | 67 +++++++++++++++++++++++++++++++++++++------ 1 file changed, 59 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 80fdcecda..54b1922eb 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -4,12 +4,15 @@ import sys import time +import six + from kafka.client import KafkaClient from kafka.common import ( OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, - FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError + FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError, + TopicAndPartition ) logger = logging.getLogger(__name__) @@ -118,15 +121,9 @@ class KafkaConsumer(object): """ def __init__(self, *topics, **configs): - self.topics = topics self.configure(**configs) - # Get initial topic metadata - self.client.load_metadata_for_topics() - for topic in self.topics: - if topic not in self.client.topic_partitions: - raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) - logger.info("Configuring consumer to fetch topic '%s'", topic) + self.set_topic_partitions(*topics) # Setup offsets self._offsets = OffsetsStruct(fetch=defaultdict(dict), @@ -333,6 +330,60 @@ def configure(self, **configs): client_id=self._config['client_id'], timeout=(self._config['socket_timeout_ms'] / 1000.0)) + def set_topic_partitions(self, *topics): + """ + Set the topic/partitions to consume + + Accepts types: + str - topic name, will consume all available partitions + TopicAndPartition namedtuple - will consume topic/partition + tuple - will consume (topic, partition) + dict - will consume { topic: partition } + { topic: [partition list] } + { topic: (partition tuple,) } + + Ex: + kafka = KafkaConsumer() + + # Consume topic1-all; topic2-partition2; topic3-partition0 + kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) + """ + self.topics = [] + self.client.load_metadata_for_topics() + + for arg in topics: + if isinstance(arg, six.string_types): + for partition in self.client.get_partition_ids_for_topic(arg): + self.topics.append(TopicAndPartition(arg, partition)) + + elif isinstance(arg, TopicAndPartition): + self.topics.append(arg) + + elif isinstance(arg, tuple): + self.topics.append(TopicAndPartition(*arg)) + + elif isinstance(arg, dict): + for topic in arg: + if isinstance(arg[topic], int): + self.topics.append(TopicAndPartition(topic, arg[topic])) + elif isinstance(arg[topic], (list, tuple)): + for partition in arg[topic]: + self.topics.append(TopicAndPartition(topic, partition)) + else: + raise KafkaConfigurationError('Unknown topic type (dict key must be ' + 'int or list/tuple of ints)') + else: + raise KafkaConfigurationError('Unknown topic type (topic must be ' + 'string, TopicAndPartition, ' + '(topic,partition) tuple, or {topic: ' + 'partitions} dict)') + + # Get initial topic metadata + for topic_partitions in self.topics: + if topic not in self.client.topic_partitions: + raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) + logger.info("Configuring consumer to fetch topic '%s'", topic) + def fetch_messages(self): max_bytes = self._config['fetch_message_max_bytes'] From e702880bda02f5f8c142afe34ce7924a08516389 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 10:42:04 -0700 Subject: [PATCH 07/33] self._topics is private; fixup topic iterations for new TopicAndPartition list; add more type checks to set_topic_and_partitions --- kafka/consumer/new.py | 111 +++++++++++++++++++++++++----------------- 1 file changed, 67 insertions(+), 44 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 54b1922eb..e7c9c55eb 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -148,51 +148,48 @@ def __init__(self, *topics, **configs): def _fetch_stored_offsets(self): logger.info("Consumer fetching stored offsets") - for topic in self.topics: - for partition in self.client.get_partition_ids_for_topic(topic): - - (resp,) = self.client.send_offset_fetch_request( - self._config['group_id'], - [OffsetFetchRequest(topic, partition)], - fail_on_error=False) - try: - check_error(resp) - # API spec says server wont set an error here - # but 0.8.1.1 does actually... - except UnknownTopicOrPartitionError: - pass - - # -1 offset signals no commit is currently stored - if resp.offset == -1: - self._offsets.commit[topic][partition] = None - self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) - - # Otherwise we committed the stored offset - # and need to fetch the next one - else: - self._offsets.commit[topic][partition] = resp.offset - self._offsets.fetch[topic][partition] = resp.offset + for topic, partition in self._topics: + (resp,) = self.client.send_offset_fetch_request( + self._config['group_id'], + [OffsetFetchRequest(topic, partition)], + fail_on_error=False) + try: + check_error(resp) + # API spec says server wont set an error here + # but 0.8.1.1 does actually... + except UnknownTopicOrPartitionError: + pass + + # -1 offset signals no commit is currently stored + if resp.offset == -1: + self._offsets.commit[topic][partition] = None + self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + + # Otherwise we committed the stored offset + # and need to fetch the next one + else: + self._offsets.commit[topic][partition] = resp.offset + self._offsets.fetch[topic][partition] = resp.offset def _auto_reset_offsets(self): logger.info("Consumer auto-resetting offsets") - for topic in self.topics: - for partition in self.client.get_partition_ids_for_topic(topic): + for topic, partition in self._topics: - self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) - self._offsets.commit[topic][partition] = None + self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + self._offsets.commit[topic][partition] = None def _reset_highwater_offsets(self): - for topic in self.topics: - for partition in self.client.get_partition_ids_for_topic(topic): - self._offsets.highwater[topic][partition] = None + for topic, partition in self._topics: + self._offsets.highwater[topic][partition] = None def _reset_task_done_offsets(self): - for topic in self.topics: - for partition in self.client.get_partition_ids_for_topic(topic): - self._offsets.task_done[topic][partition] = None + for topic, partition in self._topics: + self._offsets.task_done[topic][partition] = None def __repr__(self): - return '' % ', '.join(self.topics) + return '' % ', '.join(["%s-%d" % topic_partition + for topic_partition in + self._topics]) def __iter__(self): return self @@ -348,27 +345,50 @@ def set_topic_partitions(self, *topics): # Consume topic1-all; topic2-partition2; topic3-partition0 kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) """ - self.topics = [] + self._topics = [] self.client.load_metadata_for_topics() + # Handle different topic types for arg in topics: + + # Topic name str -- all partitions if isinstance(arg, six.string_types): for partition in self.client.get_partition_ids_for_topic(arg): - self.topics.append(TopicAndPartition(arg, partition)) + self._topics.append(TopicAndPartition(arg, partition)) + # TopicAndPartition namedtuple elif isinstance(arg, TopicAndPartition): - self.topics.append(arg) + self._topics.append(arg) + # (topic, partition) tuple elif isinstance(arg, tuple): - self.topics.append(TopicAndPartition(*arg)) - + if not isinstance(arg[0], six.string_types): + raise KafkaConfigurationError('Unknown topic type in (topic, ' + 'partition) tuple (expected string)') + if not isinstance(arg[1], int): + raise KafkaConfigurationError('Unknown partition type in (topic, ' + 'partition) tuple (expected int)') + self._topics.append(TopicAndPartition(*arg)) + + # { topic: partitions, ... } dict elif isinstance(arg, dict): for topic in arg: + + if not isinstance(topic, six.string_types): + raise KafkaConfigurationError('Unknown topic type in {topic: ' + 'partitions} dict (expected string)') + # partition can either be a single partition int if isinstance(arg[topic], int): - self.topics.append(TopicAndPartition(topic, arg[topic])) + self._topics.append(TopicAndPartition(topic, arg[topic])) + + # or a list/tuple of partition ints elif isinstance(arg[topic], (list, tuple)): for partition in arg[topic]: - self.topics.append(TopicAndPartition(topic, partition)) + if not isinstance(arg[topic], int): + raise KafkaConfigurationError('Unknown partition type in {topic: ' + 'partitions} dict (expected list ' + 'or tuple of ints)') + self._topics.append(TopicAndPartition(topic, partition)) else: raise KafkaConfigurationError('Unknown topic type (dict key must be ' 'int or list/tuple of ints)') @@ -378,10 +398,13 @@ def set_topic_partitions(self, *topics): '(topic,partition) tuple, or {topic: ' 'partitions} dict)') - # Get initial topic metadata - for topic_partitions in self.topics: + # Verify that all topic/partitions exist in metadata + for topic, partition in self._topics: if topic not in self.client.topic_partitions: raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) + if partition not in self.client.get_partition_ids_for_topic(topic): + raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " + "in broker metadata" % (partition, topic)) logger.info("Configuring consumer to fetch topic '%s'", topic) def fetch_messages(self): From 80dfaeb9a7b86a8de541556fce04438aff144bcd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:26:17 -0700 Subject: [PATCH 08/33] Move kafka._msg_iter initialization from __init__() to next() --- kafka/consumer/new.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index e7c9c55eb..5946c932a 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -122,8 +122,8 @@ class KafkaConsumer(object): def __init__(self, *topics, **configs): self.configure(**configs) - self.set_topic_partitions(*topics) + self._msg_iter = None # Setup offsets self._offsets = OffsetsStruct(fetch=defaultdict(dict), @@ -143,9 +143,6 @@ def __init__(self, *topics, **configs): self._reset_highwater_offsets() self._reset_task_done_offsets() - # Start the message fetch generator - self._msg_iter = self.fetch_messages() - def _fetch_stored_offsets(self): logger.info("Consumer fetching stored offsets") for topic, partition in self._topics: @@ -198,6 +195,10 @@ def next(self): self._set_consumer_timeout_start() while True: + # Fetch a new batch if needed + if self._msg_iter is None: + self._msg_iter = self.fetch_messages() + # Check for auto-commit if self.should_auto_commit(): self.commit() @@ -205,9 +206,9 @@ def next(self): try: return self._msg_iter.next() - # If the previous batch finishes, start get new batch + # Handle batch completion except StopIteration: - self._msg_iter = self.fetch_messages() + self._msg_iter = None self._check_consumer_timeout() From 93dd705ea7ce59e3f0a96b4d716bcbc59b8cb5a1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:33:43 -0700 Subject: [PATCH 09/33] _client is private var --- kafka/consumer/new.py | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 5946c932a..39db60c2d 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -146,7 +146,7 @@ def __init__(self, *topics, **configs): def _fetch_stored_offsets(self): logger.info("Consumer fetching stored offsets") for topic, partition in self._topics: - (resp,) = self.client.send_offset_fetch_request( + (resp,) = self._client.send_offset_fetch_request( self._config['group_id'], [OffsetFetchRequest(topic, partition)], fail_on_error=False) @@ -288,7 +288,7 @@ def commit(self): if commits: logger.info('committing consumer offsets to group %s', self._config['group_id']) - resps = self.client.send_offset_commit_request(self._config['group_id'], + resps = self._client.send_offset_commit_request(self._config['group_id'], commits, fail_on_error=False) @@ -324,7 +324,11 @@ def configure(self, **configs): logger.info("Configuring consumer to auto-commit offsets") self._set_next_auto_commit_time() - self.client = KafkaClient(self._config['metadata_broker_list'], + if self._config['metadata_broker_list'] is None: + raise KafkaConfigurationError('metadata_broker_list required to ' + 'configure KafkaConsumer') + + self._client = KafkaClient(self._config['metadata_broker_list'], client_id=self._config['client_id'], timeout=(self._config['socket_timeout_ms'] / 1000.0)) @@ -347,7 +351,7 @@ def set_topic_partitions(self, *topics): kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) """ self._topics = [] - self.client.load_metadata_for_topics() + self._client.load_metadata_for_topics() # Handle different topic types for arg in topics: @@ -423,7 +427,7 @@ def fetch_messages(self): # client.send_fetch_request will collect topic/partition requests by leader # and send each group as a single FetchRequest to the correct broker try: - responses = self.client.send_fetch_request(fetches, + responses = self._client.send_fetch_request(fetches, max_wait_time=max_wait_time, min_bytes=min_bytes, fail_on_error=False) @@ -504,7 +508,7 @@ def _reset_partition_offset(self, topic, partition): def get_partition_offsets(self, topic, partition, request_time, num_offsets): reqs = [OffsetRequest(topic, partition, request_time, num_offsets)] - (resp,) = self.client.send_offset_request(reqs) + (resp,) = self._client.send_offset_request(reqs) check_error(resp) @@ -521,7 +525,7 @@ def _refresh_metadata_on_error(self): logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) time.sleep(sleep_ms / 1000.0) try: - self.client.load_metadata_for_topics() + self._client.load_metadata_for_topics() except KafkaUnavailableError: logger.warning("Unable to refresh topic metadata... cluster unavailable") self._check_consumer_timeout() From d5ec0a60212fb37f77f48d9ed757135364d6aed5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:37:07 -0700 Subject: [PATCH 10/33] Support setting offsets in set_topic_partitions(); reorganize offsets initialization --- kafka/consumer/new.py | 278 +++++++++++++++++++++++------------------- 1 file changed, 151 insertions(+), 127 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 39db60c2d..edf7843c4 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -1,4 +1,4 @@ -from collections import defaultdict, namedtuple +from collections import namedtuple from copy import deepcopy import logging import sys @@ -11,8 +11,7 @@ OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, - FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError, - TopicAndPartition + FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError ) logger = logging.getLogger(__name__) @@ -125,30 +124,12 @@ def __init__(self, *topics, **configs): self.set_topic_partitions(*topics) self._msg_iter = None - # Setup offsets - self._offsets = OffsetsStruct(fetch=defaultdict(dict), - commit=defaultdict(dict), - highwater= defaultdict(dict), - task_done=defaultdict(dict)) - - # If we have a consumer group, try to fetch stored offsets - if self._config['group_id']: - self._fetch_stored_offsets() - else: - self._auto_reset_offsets() - - # highwater marks (received from server on fetch response) - # and task_done (set locally by user) - # should always get initialized to None - self._reset_highwater_offsets() - self._reset_task_done_offsets() - - def _fetch_stored_offsets(self): + def _get_commit_offsets(self): logger.info("Consumer fetching stored offsets") - for topic, partition in self._topics: + for topic_partition in self._topics: (resp,) = self._client.send_offset_fetch_request( self._config['group_id'], - [OffsetFetchRequest(topic, partition)], + [OffsetFetchRequest(topic_partition[0], topic_partition[1])], fail_on_error=False) try: check_error(resp) @@ -159,29 +140,20 @@ def _fetch_stored_offsets(self): # -1 offset signals no commit is currently stored if resp.offset == -1: - self._offsets.commit[topic][partition] = None - self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + self._offsets.commit[topic_partition] = None # Otherwise we committed the stored offset # and need to fetch the next one else: - self._offsets.commit[topic][partition] = resp.offset - self._offsets.fetch[topic][partition] = resp.offset - - def _auto_reset_offsets(self): - logger.info("Consumer auto-resetting offsets") - for topic, partition in self._topics: - - self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) - self._offsets.commit[topic][partition] = None + self._offsets.commit[topic_partition] = resp.offset def _reset_highwater_offsets(self): - for topic, partition in self._topics: - self._offsets.highwater[topic][partition] = None + for topic_partition in self._topics: + self._offsets.highwater[topic_partition] = None def _reset_task_done_offsets(self): - for topic, partition in self._topics: - self._offsets.task_done[topic][partition] = None + for topic_partition in self._topics: + self._offsets.task_done[topic_partition] = None def __repr__(self): return '' % ', '.join(["%s-%d" % topic_partition @@ -224,12 +196,16 @@ def offsets(self, group=None): return dict(deepcopy(getattr(self._offsets, group))) def task_done(self, message): - topic = message.topic - partition = message.partition + """ + Mark a fetched message as consumed. + Offsets for messages marked as "task_done" will be stored back + to the kafka cluster for this consumer group on commit() + """ + topic_partition = (message.topic, message.partition) offset = message.offset # Warn on non-contiguous offsets - prev_done = self._offsets.task_done[topic][partition] + prev_done = self._offsets.task_done[topic_partition] if prev_done is not None and offset != (prev_done + 1): logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', offset, prev_done) @@ -237,12 +213,12 @@ def task_done(self, message): # Warn on smaller offsets than previous commit # "commit" offsets are actually the offset of the next # message to fetch. # so task_done should be compared with (commit - 1) - prev_done = (self._offsets.commit[topic][partition] - 1) + prev_done = (self._offsets.commit[topic_partition] - 1) if prev_done is not None and (offset <= prev_done): logger.warning('Marking task_done on a previously committed offset?: %d <= %d', offset, prev_done) - self._offsets.task_done[topic][partition] = offset + self._offsets.task_done[topic_partition] = offset def should_auto_commit(self): if not self._config['auto_commit_enable']: @@ -267,24 +243,23 @@ def commit(self): offsets = self._offsets.task_done commits = [] - for topic, partitions in offsets.iteritems(): - for partition, task_done in partitions.iteritems(): + for topic_partition, task_done_offset in offsets.iteritems(): - # Skip if None - if task_done is None: - continue + # Skip if None + if task_done_offset is None: + continue - # Commit offsets as the next offset to fetch - # which is consistent with the Java Client - # task_done is marked by messages consumed, - # so add one to mark the next message for fetching - commit_offset = (task_done + 1) + # Commit offsets as the next offset to fetch + # which is consistent with the Java Client + # task_done is marked by messages consumed, + # so add one to mark the next message for fetching + commit_offset = (task_done_offset + 1) - # Skip if no change from previous committed - if commit_offset == self._offsets.commit[topic][partition]: - continue + # Skip if no change from previous committed + if commit_offset == self._offsets.commit[topic_partition]: + continue - commits.append(OffsetCommitRequest(topic, partition, commit_offset, metadata)) + commits.append(OffsetCommitRequest(topic_partition[0], topic_partition[1], commit_offset, metadata)) if commits: logger.info('committing consumer offsets to group %s', self._config['group_id']) @@ -294,8 +269,9 @@ def commit(self): for r in resps: check_error(r) - task_done = self._offsets.task_done[r.topic][r.partition] - self._offsets.commit[r.topic][r.partition] = (task_done + 1) + topic_partition = (r.topic, r.partition) + task_done = self._offsets.task_done[topic_partition] + self._offsets.commit[topic_partition] = (task_done + 1) if self._config['auto_commit_enable']: self._set_next_auto_commit_time() @@ -335,82 +311,128 @@ def configure(self, **configs): def set_topic_partitions(self, *topics): """ Set the topic/partitions to consume + Optionally specify offsets to start from Accepts types: - str - topic name, will consume all available partitions - TopicAndPartition namedtuple - will consume topic/partition - tuple - will consume (topic, partition) - dict - will consume { topic: partition } - { topic: [partition list] } - { topic: (partition tuple,) } + str: topic name (will consume all available partitions) + tuple: (topic, partition) + dict: { topic: partition } + { topic: [partition list] } + { topic: (partition tuple,) } + + Optionally, offsets can be specified directly: + tuple: (topic, partition, offset) + dict: { (topic, partition): offset, ... } Ex: kafka = KafkaConsumer() # Consume topic1-all; topic2-partition2; topic3-partition0 kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) + + # Consume topic1-0 starting at offset 123, and topic2-1 at offset 456 + # using tuples -- + kafka.set_topic_partitions(("topic1", 0, 123), ("topic2", 1, 456)) + + # using dict -- + kafka.set_topic_partitions({ ("topic1", 0): 123, ("topic2", 1): 456 }) """ self._topics = [] self._client.load_metadata_for_topics() + # Setup offsets + self._offsets = OffsetsStruct(fetch=dict(), + commit=dict(), + highwater=dict(), + task_done=dict()) + # Handle different topic types for arg in topics: # Topic name str -- all partitions if isinstance(arg, six.string_types): - for partition in self.client.get_partition_ids_for_topic(arg): - self._topics.append(TopicAndPartition(arg, partition)) - - # TopicAndPartition namedtuple - elif isinstance(arg, TopicAndPartition): - self._topics.append(arg) + topic = arg + for partition in self._client.get_partition_ids_for_topic(arg): + self._consume_topic_partition(topic, partition) - # (topic, partition) tuple + # (topic, partition [, offset]) tuple elif isinstance(arg, tuple): - if not isinstance(arg[0], six.string_types): - raise KafkaConfigurationError('Unknown topic type in (topic, ' - 'partition) tuple (expected string)') - if not isinstance(arg[1], int): - raise KafkaConfigurationError('Unknown partition type in (topic, ' - 'partition) tuple (expected int)') - self._topics.append(TopicAndPartition(*arg)) + (topic, partition) = arg[0:2] + if len(arg) == 3: + offset = arg[2] + self._offsets.fetch[(topic, partition)] = offset + self._consume_topic_partition(topic, partition) # { topic: partitions, ... } dict elif isinstance(arg, dict): - for topic in arg: - - if not isinstance(topic, six.string_types): - raise KafkaConfigurationError('Unknown topic type in {topic: ' - 'partitions} dict (expected string)') - # partition can either be a single partition int - if isinstance(arg[topic], int): - self._topics.append(TopicAndPartition(topic, arg[topic])) - - # or a list/tuple of partition ints - elif isinstance(arg[topic], (list, tuple)): - for partition in arg[topic]: - if not isinstance(arg[topic], int): - raise KafkaConfigurationError('Unknown partition type in {topic: ' - 'partitions} dict (expected list ' - 'or tuple of ints)') - self._topics.append(TopicAndPartition(topic, partition)) - else: - raise KafkaConfigurationError('Unknown topic type (dict key must be ' - 'int or list/tuple of ints)') + for key, value in arg.iteritems(): + + # key can be string (a topic) + if isinstance(key, six.string_types): + + # topic: partition + if isinstance(value, int): + self._consume_topic_partition(key, value) + + # topic: [ partition1, partition2, ... ] + elif isinstance(value, (list, tuple)): + for partition in value: + self._consume_topic_partition(key, partition) + else: + raise KafkaConfigurationError('Unknown topic type (dict key must be ' + 'int or list/tuple of ints)') + + # (topic, partition): offset + elif isinstance(key, tuple): + self._consume_topic_partition(*key) + self._offsets.fetch[key] = value + else: - raise KafkaConfigurationError('Unknown topic type (topic must be ' - 'string, TopicAndPartition, ' - '(topic,partition) tuple, or {topic: ' - 'partitions} dict)') - - # Verify that all topic/partitions exist in metadata - for topic, partition in self._topics: - if topic not in self.client.topic_partitions: - raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) - if partition not in self.client.get_partition_ids_for_topic(topic): - raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " - "in broker metadata" % (partition, topic)) - logger.info("Configuring consumer to fetch topic '%s'", topic) + raise KafkaConfigurationError('Unknown topic type (%s)' % type(arg)) + + # If we have a consumer group, try to fetch stored offsets + if self._config['group_id']: + self._get_commit_offsets() + + # Update missing fetch/commit offsets + for topic_partition in self._topics: + + # Commit offsets default is None + if topic_partition not in self._offsets.commit: + self._offsets.commit[topic_partition] = None + + # Skip if we already have a fetch offset from user args + if topic_partition not in self._offsets.fetch: + + # Fetch offsets default is (1) commit + if self._offsets.commit[topic_partition] is not None: + self._offsets.fetch[topic_partition] = self._offsets.commit[topic_partition] + + # or (2) auto reset + else: + self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) + + # highwater marks (received from server on fetch response) + # and task_done (set locally by user) + # should always get initialized to None + self._reset_highwater_offsets() + self._reset_task_done_offsets() + + def _consume_topic_partition(self, topic, partition): + if not isinstance(topic, six.string_types): + raise KafkaConfigurationError('Unknown topic type (%s) ' + '-- expected string' % type(topic)) + if not isinstance(partition, int): + raise KafkaConfigurationError('Unknown partition type (%s) ' + '-- expected int' % type(partition)) + + if topic not in self._client.topic_partitions: + raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) + if partition not in self._client.get_partition_ids_for_topic(topic): + raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " + "in broker metadata" % (partition, topic)) + logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) + self._topics.append((topic, partition)) def fetch_messages(self): @@ -420,9 +442,8 @@ def fetch_messages(self): fetches = [] offsets = self._offsets.fetch - for topic, partitions in offsets.iteritems(): - for partition, offset in partitions.iteritems(): - fetches.append(FetchRequest(topic, partition, offset, max_bytes)) + for topic_partition, offset in offsets.iteritems(): + fetches.append(FetchRequest(topic_partition[0], topic_partition[1], offset, max_bytes)) # client.send_fetch_request will collect topic/partition requests by leader # and send each group as a single FetchRequest to the correct broker @@ -437,48 +458,51 @@ def fetch_messages(self): return for resp in responses: - topic = resp.topic - partition = resp.partition + topic_partition = (resp.topic, resp.partition) try: check_error(resp) except OffsetOutOfRangeError: logger.warning('OffsetOutOfRange: topic %s, partition %d, offset %d ' '(Highwatermark: %d)', - topic, partition, offsets[topic][partition], - resp.highwaterMark) + resp.topic, resp.partition, + offsets[topic_partition], resp.highwaterMark) # Reset offset - self._offsets.fetch[topic][partition] = self._reset_partition_offset(topic, partition) + self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) continue except NotLeaderForPartitionError: logger.warning("NotLeaderForPartitionError for %s - %d. " "Metadata may be out of date", - topic, partition) + resp.topic, resp.partition) self._refresh_metadata_on_error() continue except RequestTimedOutError: - logger.warning("RequestTimedOutError for %s - %d", topic, partition) + logger.warning("RequestTimedOutError for %s - %d", + resp.topic, resp.partition) continue # Track server highwater mark - self._offsets.highwater[topic][partition] = resp.highwaterMark + self._offsets.highwater[topic_partition] = resp.highwaterMark # Yield each message # Kafka-python could raise an exception during iteration # we are not catching -- user will need to address for (offset, message) in resp.messages: # deserializer_class could raise an exception here - msg = KafkaMessage(topic, partition, offset, message.key, + msg = KafkaMessage(resp.topic, + resp.partition, + offset, message.key, self._config['deserializer_class'](message.value)) # Only increment fetch offset if we safely got the message and deserialized - self._offsets.fetch[topic][partition] = offset + 1 + self._offsets.fetch[topic_partition] = offset + 1 # Then yield to user yield msg - def _reset_partition_offset(self, topic, partition): + def _reset_partition_offset(self, topic_partition): + (topic, partition) = topic_partition LATEST = -1 EARLIEST = -2 From 67aa1a1c041d03a8139aac7081500ddd55665c46 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:37:32 -0700 Subject: [PATCH 11/33] _should_auto_commit is private --- kafka/consumer/new.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index edf7843c4..e1273ea77 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -172,7 +172,7 @@ def next(self): self._msg_iter = self.fetch_messages() # Check for auto-commit - if self.should_auto_commit(): + if self._should_auto_commit(): self.commit() try: @@ -220,7 +220,7 @@ def task_done(self, message): self._offsets.task_done[topic_partition] = offset - def should_auto_commit(self): + def _should_auto_commit(self): if not self._config['auto_commit_enable']: return False From 8dc362320851e7e5b751a0196d91dfd25bb4e07f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:38:24 -0700 Subject: [PATCH 12/33] raise KafkaConfigurationError in commit() if there is no configured 'group_id'; add docstring --- kafka/consumer/new.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index e1273ea77..72c471fa5 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -233,9 +233,16 @@ def _set_next_auto_commit_time(self): self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) def commit(self): + """ + Store consumed message offsets (marked via task_done()) + to kafka cluster for this consumer_group. + + Note -- this functionality requires server version >=0.8.1.1 + see https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + """ if not self._config['group_id']: logger.warning('Cannot commit without a group_id!') - raise RuntimeError('Attempted to commit offsets without a configured consumer group (group_id)') + raise KafkaConfigurationError('Attempted to commit offsets without a configured consumer group (group_id)') # API supports storing metadata with each commit # but for now it is unused From 9dd7d7e07ca4dd2c47aed9371844c5006b56e4a9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:38:48 -0700 Subject: [PATCH 13/33] Add docstring to configure() --- kafka/consumer/new.py | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 72c471fa5..e7d38de87 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -290,6 +290,27 @@ def commit(self): return False def configure(self, **configs): + """ + Configuration settings can be passed to constructor, + otherwise defaults will be used: + client_id='kafka.consumer.XXX', + group_id=None, + fetch_message_max_bytes=1024*1024, + fetch_min_bytes=1, + fetch_wait_max_ms=100, + refresh_leader_backoff_ms=200, + metadata_broker_list=None, + socket_timeout_ms=30*1000, + auto_offset_reset='largest', + deserializer_class=Event.from_bytes, + auto_commit_enable=False, + auto_commit_interval_ms=60 * 1000, + consumer_timeout_ms=-1 + + + Configuration parameters are described in more detail at + http://kafka.apache.org/documentation.html#highlevelconsumerapi + """ self._config = {} for key in DEFAULT_CONSUMER_CONFIG: self._config[key] = configs.pop(key, DEFAULT_CONSUMER_CONFIG[key]) From 26e18ce0c032e2801cbbe5d9f444107b8ab4919a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Sep 2014 23:40:40 -0700 Subject: [PATCH 14/33] Add docstring to get_partition_offsets; use request_time_ms and max_num_offsets var names --- kafka/consumer/new.py | 32 +++++++++++++++++++++++++------- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index e7d38de87..04696af64 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -534,11 +534,11 @@ def _reset_partition_offset(self, topic_partition): LATEST = -1 EARLIEST = -2 - RequestTime = None + request_time_ms = None if self._config['auto_offset_reset'] == 'largest': - RequestTime = LATEST + request_time_ms = LATEST elif self._config['auto_offset_reset'] == 'smallest': - RequestTime = EARLIEST + request_time_ms = EARLIEST else: # Let's raise an reasonable exception type if user calls @@ -553,12 +553,30 @@ def _reset_partition_offset(self, topic_partition): # the request that triggered it, and we do not want to drop that raise - (offset, ) = self.get_partition_offsets(topic, partition, RequestTime, - num_offsets=1) + (offset, ) = self.get_partition_offsets(topic, partition, + request_time_ms, max_num_offsets=1) return offset - def get_partition_offsets(self, topic, partition, request_time, num_offsets): - reqs = [OffsetRequest(topic, partition, request_time, num_offsets)] + def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): + """ + Request available fetch offsets for a single topic/partition + + @param topic (str) + @param partition (int) + @param request_time_ms (int) -- Used to ask for all messages before a + certain time (ms). There are two special + values. Specify -1 to receive the latest + offset (i.e. the offset of the next coming + message) and -2 to receive the earliest + available offset. Note that because offsets + are pulled in descending order, asking for + the earliest offset will always return you + a single element. + @param max_num_offsets (int) + + @return offsets (list) + """ + reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] (resp,) = self._client.send_offset_request(reqs) From 54ecfeaec05d4a4c85a37310885430b771c8bc57 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 18:08:00 -0700 Subject: [PATCH 15/33] Add a few basic KafkaConsumer tests --- test/test_consumer.py | 6 +++ test/test_consumer_integration.py | 90 ++++++++++++++++++++++++++----- 2 files changed, 83 insertions(+), 13 deletions(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index 906091919..b33e53719 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -3,8 +3,14 @@ from . import unittest from kafka.consumer import SimpleConsumer +from kafka.consumer.new import KafkaConsumer +from kafka.common import KafkaConfigurationError class TestKafkaConsumer(unittest.TestCase): def test_non_integer_partitions(self): with self.assertRaises(AssertionError): SimpleConsumer(MagicMock(), 'group', 'topic', partitions = [ '0' ]) + + def test_broker_list_required(self): + with self.assertRaises(KafkaConfigurationError): + KafkaConsumer() diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 2762008d6..b4af70b78 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -1,10 +1,14 @@ +import logging import os from six.moves import xrange from kafka import SimpleConsumer, MultiProcessConsumer, create_message -from kafka.common import ProduceRequest, ConsumerFetchSizeTooSmall +from kafka.common import ( + ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout +) from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES +from kafka.consumer.new import KafkaConsumer from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( @@ -47,6 +51,29 @@ def assert_message_count(self, messages, num_messages): # Make sure there are no duplicates self.assertEquals(len(set(messages)), num_messages) + def consumer(self, **kwargs): + if os.environ['KAFKA_VERSION'] == "0.8.0": + # Kafka 0.8.0 simply doesn't support offset requests, so hard code it being off + kwargs['auto_commit'] = False + else: + kwargs.setdefault('auto_commit', True) + + consumer_class = kwargs.pop('consumer', SimpleConsumer) + group = kwargs.pop('group', self.id().encode('utf-8')) + topic = kwargs.pop('topic', self.topic) + + if consumer_class == SimpleConsumer: + kwargs.setdefault('iter_timeout', 0) + + return consumer_class(self.client, group, topic, **kwargs) + + def kafka_consumer(self, **configs): + brokers = '%s:%d' % (self.server.host, self.server.port) + consumer = KafkaConsumer(self.topic, + metadata_broker_list=brokers, + **configs) + return consumer + @kafka_versions("all") def test_simple_consumer(self): self.send_messages(0, range(0, 100)) @@ -275,18 +302,55 @@ def test_fetch_buffer_size(self): messages = [ message for message in consumer ] self.assertEquals(len(messages), 2) - def consumer(self, **kwargs): - if os.environ['KAFKA_VERSION'] == "0.8.0": - # Kafka 0.8.0 simply doesn't support offset requests, so hard code it being off - kwargs['auto_commit'] = False - else: - kwargs.setdefault('auto_commit', True) + @kafka_versions("all") + def test_kafka_consumer(self): + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) - consumer_class = kwargs.pop('consumer', SimpleConsumer) - group = kwargs.pop('group', self.id().encode('utf-8')) - topic = kwargs.pop('topic', self.topic) + # Start a consumer + consumer = self.kafka_consumer(auto_offset_reset='smallest', + consumer_timeout_ms=5000) + n = 0 + messages = {0: set(), 1: set()} + logging.debug("kafka consumer offsets: %s" % consumer.offsets()) + for m in consumer: + logging.debug("Consumed message %s" % repr(m)) + n += 1 + messages[m.partition].add(m.offset) + if n >= 200: + break + + self.assertEquals(len(messages[0]), 100) + self.assertEquals(len(messages[1]), 100) - if consumer_class == SimpleConsumer: - kwargs.setdefault('iter_timeout', 0) + @kafka_versions("all") + def test_kafka_consumer__blocking(self): + consumer = self.kafka_consumer(auto_offset_reset='smallest', + consumer_timeout_ms=1000) - return consumer_class(self.client, group, topic, **kwargs) + # Ask for 5 messages, nothing in queue, block 5 seconds + with Timer() as t: + with self.assertRaises(ConsumerTimeout): + msg = consumer.next() + self.assertGreaterEqual(t.interval, 1) + + self.send_messages(0, range(0, 10)) + + # Ask for 5 messages, 10 in queue. Get 5 back, no blocking + messages = set() + with Timer() as t: + for i in range(5): + msg = consumer.next() + messages.add((msg.partition, msg.offset)) + self.assertEqual(len(messages), 5) + self.assertLess(t.interval, 1) + + # Ask for 10 messages, get 5 back, block 5 seconds + messages = set() + with Timer() as t: + with self.assertRaises(ConsumerTimeout): + for i in range(10): + msg = consumer.next() + messages.add((msg.partition, msg.offset)) + self.assertEqual(len(messages), 5) + self.assertGreaterEqual(t.interval, 1) From 5b882981d17cfec06cf2f7b44ff34313e7f0180a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 21:16:49 -0700 Subject: [PATCH 16/33] Use 4-space indents --- kafka/consumer/new.py | 1035 ++++++++++++++++++++--------------------- 1 file changed, 517 insertions(+), 518 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 04696af64..f0a442465 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -44,253 +44,60 @@ class KafkaConsumer(object): - """ - A simpler kafka consumer - - ``` - # A very basic 'tail' consumer, with no stored offset management - kafka = KafkaConsumer('topic1') - for m in kafka: - print m - - # Alternate interface: next() - while True: - print kafka.next() - - # Alternate interface: batch iteration - while True: - for m in kafka.fetch_messages(): - print m - print "Done with batch - let's do another!" - ``` - - ``` - # more advanced consumer -- multiple topics w/ auto commit offset management - kafka = KafkaConsumer('topic1', 'topic2', - group_id='my_consumer_group', - auto_commit_enable=True, - auto_commit_interval_ms=30 * 1000, - auto_offset_reset='smallest') - - # Infinite iteration - for m in kafka: - process_message(m) - kafka.task_done(m) - - # Alternate interface: next() - while True: - m = kafka.next() - process_message(m) - kafka.task_done(m) - - # Batch process interface does not auto_commit! - while True: - for m in kafka.fetch_messages(): - process_message(m) - kafka.task_done(m) - kafka.commit() - ``` - - messages (m) are namedtuples with attributes: - m.topic: topic name (str) - m.partition: partition number (int) - m.offset: message offset on topic-partition log (int) - m.key: key (bytes - can be None) - m.value: message (output of deserializer_class - default is event object) - - Configuration settings can be passed to constructor, - otherwise defaults will be used: - client_id='kafka.consumer.XXX', - group_id=None, - fetch_message_max_bytes=1024*1024, - fetch_min_bytes=1, - fetch_wait_max_ms=100, - refresh_leader_backoff_ms=200, - metadata_broker_list=None, - socket_timeout_ms=30*1000, - auto_offset_reset='largest', - deserializer_class=Event.from_bytes, - auto_commit_enable=False, - auto_commit_interval_ms=60 * 1000, - consumer_timeout_ms=-1 - - - Configuration parameters are described in more detail at - http://kafka.apache.org/documentation.html#highlevelconsumerapi - """ - - def __init__(self, *topics, **configs): - self.configure(**configs) - self.set_topic_partitions(*topics) - self._msg_iter = None - - def _get_commit_offsets(self): - logger.info("Consumer fetching stored offsets") - for topic_partition in self._topics: - (resp,) = self._client.send_offset_fetch_request( - self._config['group_id'], - [OffsetFetchRequest(topic_partition[0], topic_partition[1])], - fail_on_error=False) - try: - check_error(resp) - # API spec says server wont set an error here - # but 0.8.1.1 does actually... - except UnknownTopicOrPartitionError: - pass - - # -1 offset signals no commit is currently stored - if resp.offset == -1: - self._offsets.commit[topic_partition] = None - - # Otherwise we committed the stored offset - # and need to fetch the next one - else: - self._offsets.commit[topic_partition] = resp.offset - - def _reset_highwater_offsets(self): - for topic_partition in self._topics: - self._offsets.highwater[topic_partition] = None - - def _reset_task_done_offsets(self): - for topic_partition in self._topics: - self._offsets.task_done[topic_partition] = None - - def __repr__(self): - return '' % ', '.join(["%s-%d" % topic_partition - for topic_partition in - self._topics]) - - def __iter__(self): - return self - - def next(self): - self._set_consumer_timeout_start() - while True: - - # Fetch a new batch if needed - if self._msg_iter is None: - self._msg_iter = self.fetch_messages() - - # Check for auto-commit - if self._should_auto_commit(): - self.commit() - - try: - return self._msg_iter.next() - - # Handle batch completion - except StopIteration: - self._msg_iter = None - - self._check_consumer_timeout() - - def offsets(self, group=None): - if not group: - return { - 'fetch': self.offsets('fetch'), - 'commit': self.offsets('commit'), - 'task_done': self.offsets('task_done'), - 'highwater': self.offsets('highwater') - } - else: - return dict(deepcopy(getattr(self._offsets, group))) - - def task_done(self, message): - """ - Mark a fetched message as consumed. - Offsets for messages marked as "task_done" will be stored back - to the kafka cluster for this consumer group on commit() """ - topic_partition = (message.topic, message.partition) - offset = message.offset - - # Warn on non-contiguous offsets - prev_done = self._offsets.task_done[topic_partition] - if prev_done is not None and offset != (prev_done + 1): - logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', - offset, prev_done) - - # Warn on smaller offsets than previous commit - # "commit" offsets are actually the offset of the next # message to fetch. - # so task_done should be compared with (commit - 1) - prev_done = (self._offsets.commit[topic_partition] - 1) - if prev_done is not None and (offset <= prev_done): - logger.warning('Marking task_done on a previously committed offset?: %d <= %d', - offset, prev_done) - - self._offsets.task_done[topic_partition] = offset - - def _should_auto_commit(self): - if not self._config['auto_commit_enable']: - return False - - if not self._next_commit: - return False - - return (time.time() >= self._next_commit) - - def _set_next_auto_commit_time(self): - self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) - - def commit(self): - """ - Store consumed message offsets (marked via task_done()) - to kafka cluster for this consumer_group. - - Note -- this functionality requires server version >=0.8.1.1 - see https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI - """ - if not self._config['group_id']: - logger.warning('Cannot commit without a group_id!') - raise KafkaConfigurationError('Attempted to commit offsets without a configured consumer group (group_id)') - - # API supports storing metadata with each commit - # but for now it is unused - metadata = '' - - offsets = self._offsets.task_done - commits = [] - for topic_partition, task_done_offset in offsets.iteritems(): - - # Skip if None - if task_done_offset is None: - continue - - # Commit offsets as the next offset to fetch - # which is consistent with the Java Client - # task_done is marked by messages consumed, - # so add one to mark the next message for fetching - commit_offset = (task_done_offset + 1) - - # Skip if no change from previous committed - if commit_offset == self._offsets.commit[topic_partition]: - continue - - commits.append(OffsetCommitRequest(topic_partition[0], topic_partition[1], commit_offset, metadata)) + A simpler kafka consumer - if commits: - logger.info('committing consumer offsets to group %s', self._config['group_id']) - resps = self._client.send_offset_commit_request(self._config['group_id'], - commits, - fail_on_error=False) + ``` + # A very basic 'tail' consumer, with no stored offset management + kafka = KafkaConsumer('topic1') + for m in kafka: + print m - for r in resps: - check_error(r) - topic_partition = (r.topic, r.partition) - task_done = self._offsets.task_done[topic_partition] - self._offsets.commit[topic_partition] = (task_done + 1) + # Alternate interface: next() + while True: + print kafka.next() - if self._config['auto_commit_enable']: - self._set_next_auto_commit_time() + # Alternate interface: batch iteration + while True: + for m in kafka.fetch_messages(): + print m + print "Done with batch - let's do another!" + ``` + + ``` + # more advanced consumer -- multiple topics w/ auto commit offset management + kafka = KafkaConsumer('topic1', 'topic2', + group_id='my_consumer_group', + auto_commit_enable=True, + auto_commit_interval_ms=30 * 1000, + auto_offset_reset='smallest') + + # Infinite iteration + for m in kafka: + process_message(m) + kafka.task_done(m) - return True + # Alternate interface: next() + while True: + m = kafka.next() + process_message(m) + kafka.task_done(m) - else: - logger.info('No new offsets found to commit in group %s', self._config['group_id']) - return False + # Batch process interface does not auto_commit! + while True: + for m in kafka.fetch_messages(): + process_message(m) + kafka.task_done(m) + kafka.commit() + ``` + + messages (m) are namedtuples with attributes: + m.topic: topic name (str) + m.partition: partition number (int) + m.offset: message offset on topic-partition log (int) + m.key: key (bytes - can be None) + m.value: message (output of deserializer_class - default is event object) - def configure(self, **configs): - """ Configuration settings can be passed to constructor, otherwise defaults will be used: client_id='kafka.consumer.XXX', @@ -311,303 +118,495 @@ def configure(self, **configs): Configuration parameters are described in more detail at http://kafka.apache.org/documentation.html#highlevelconsumerapi """ - self._config = {} - for key in DEFAULT_CONSUMER_CONFIG: - self._config[key] = configs.pop(key, DEFAULT_CONSUMER_CONFIG[key]) - - if configs: - raise KafkaConfigurationError('Unknown configuration key(s): ' + - str(list(configs.keys()))) - - if self._config['auto_commit_enable']: - if not self._config['group_id']: - raise KafkaConfigurationError('KafkaConsumer configured to auto-commit without required consumer group (group_id)') - # Check auto-commit configuration - if self._config['auto_commit_enable']: - logger.info("Configuring consumer to auto-commit offsets") - self._set_next_auto_commit_time() - - if self._config['metadata_broker_list'] is None: - raise KafkaConfigurationError('metadata_broker_list required to ' - 'configure KafkaConsumer') - - self._client = KafkaClient(self._config['metadata_broker_list'], - client_id=self._config['client_id'], - timeout=(self._config['socket_timeout_ms'] / 1000.0)) + def __init__(self, *topics, **configs): + self.configure(**configs) + self.set_topic_partitions(*topics) + self._msg_iter = None - def set_topic_partitions(self, *topics): - """ - Set the topic/partitions to consume - Optionally specify offsets to start from + def _get_commit_offsets(self): + logger.info("Consumer fetching stored offsets") + for topic_partition in self._topics: + (resp,) = self._client.send_offset_fetch_request( + self._config['group_id'], + [OffsetFetchRequest(topic_partition[0], topic_partition[1])], + fail_on_error=False) + try: + check_error(resp) + # API spec says server wont set an error here + # but 0.8.1.1 does actually... + except UnknownTopicOrPartitionError: + pass + + # -1 offset signals no commit is currently stored + if resp.offset == -1: + self._offsets.commit[topic_partition] = None + + # Otherwise we committed the stored offset + # and need to fetch the next one + else: + self._offsets.commit[topic_partition] = resp.offset - Accepts types: - str: topic name (will consume all available partitions) - tuple: (topic, partition) - dict: { topic: partition } - { topic: [partition list] } - { topic: (partition tuple,) } + def _reset_highwater_offsets(self): + for topic_partition in self._topics: + self._offsets.highwater[topic_partition] = None - Optionally, offsets can be specified directly: - tuple: (topic, partition, offset) - dict: { (topic, partition): offset, ... } + def _reset_task_done_offsets(self): + for topic_partition in self._topics: + self._offsets.task_done[topic_partition] = None - Ex: - kafka = KafkaConsumer() + def __repr__(self): + return '' % ', '.join(["%s-%d" % topic_partition + for topic_partition in + self._topics]) - # Consume topic1-all; topic2-partition2; topic3-partition0 - kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) + def __iter__(self): + return self - # Consume topic1-0 starting at offset 123, and topic2-1 at offset 456 - # using tuples -- - kafka.set_topic_partitions(("topic1", 0, 123), ("topic2", 1, 456)) + def next(self): + self._set_consumer_timeout_start() + while True: - # using dict -- - kafka.set_topic_partitions({ ("topic1", 0): 123, ("topic2", 1): 456 }) - """ - self._topics = [] - self._client.load_metadata_for_topics() - - # Setup offsets - self._offsets = OffsetsStruct(fetch=dict(), - commit=dict(), - highwater=dict(), - task_done=dict()) - - # Handle different topic types - for arg in topics: - - # Topic name str -- all partitions - if isinstance(arg, six.string_types): - topic = arg - for partition in self._client.get_partition_ids_for_topic(arg): - self._consume_topic_partition(topic, partition) - - # (topic, partition [, offset]) tuple - elif isinstance(arg, tuple): - (topic, partition) = arg[0:2] - if len(arg) == 3: - offset = arg[2] - self._offsets.fetch[(topic, partition)] = offset - self._consume_topic_partition(topic, partition) - - # { topic: partitions, ... } dict - elif isinstance(arg, dict): - for key, value in arg.iteritems(): - - # key can be string (a topic) - if isinstance(key, six.string_types): - - # topic: partition - if isinstance(value, int): - self._consume_topic_partition(key, value) - - # topic: [ partition1, partition2, ... ] - elif isinstance(value, (list, tuple)): - for partition in value: - self._consume_topic_partition(key, partition) - else: - raise KafkaConfigurationError('Unknown topic type (dict key must be ' - 'int or list/tuple of ints)') + # Fetch a new batch if needed + if self._msg_iter is None: + self._msg_iter = self.fetch_messages() - # (topic, partition): offset - elif isinstance(key, tuple): - self._consume_topic_partition(*key) - self._offsets.fetch[key] = value + # Check for auto-commit + if self._should_auto_commit(): + self.commit() - else: - raise KafkaConfigurationError('Unknown topic type (%s)' % type(arg)) + try: + return self._msg_iter.next() - # If we have a consumer group, try to fetch stored offsets - if self._config['group_id']: - self._get_commit_offsets() + # Handle batch completion + except StopIteration: + self._msg_iter = None - # Update missing fetch/commit offsets - for topic_partition in self._topics: + self._check_consumer_timeout() - # Commit offsets default is None - if topic_partition not in self._offsets.commit: - self._offsets.commit[topic_partition] = None + def offsets(self, group=None): + if not group: + return { + 'fetch': self.offsets('fetch'), + 'commit': self.offsets('commit'), + 'task_done': self.offsets('task_done'), + 'highwater': self.offsets('highwater') + } + else: + return dict(deepcopy(getattr(self._offsets, group))) + + def task_done(self, message): + """ + Mark a fetched message as consumed. + Offsets for messages marked as "task_done" will be stored back + to the kafka cluster for this consumer group on commit() + """ + topic_partition = (message.topic, message.partition) + offset = message.offset + + # Warn on non-contiguous offsets + prev_done = self._offsets.task_done[topic_partition] + if prev_done is not None and offset != (prev_done + 1): + logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', + offset, prev_done) + + # Warn on smaller offsets than previous commit + # "commit" offsets are actually the offset of the next # message to fetch. + # so task_done should be compared with (commit - 1) + prev_done = (self._offsets.commit[topic_partition] - 1) + if prev_done is not None and (offset <= prev_done): + logger.warning('Marking task_done on a previously committed offset?: %d <= %d', + offset, prev_done) + + self._offsets.task_done[topic_partition] = offset + + def _should_auto_commit(self): + if not self._config['auto_commit_enable']: + return False + + if not self._next_commit: + return False + + return (time.time() >= self._next_commit) + + def _set_next_auto_commit_time(self): + self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) + + def commit(self): + """ + Store consumed message offsets (marked via task_done()) + to kafka cluster for this consumer_group. + + Note -- this functionality requires server version >=0.8.1.1 + see https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + """ + if not self._config['group_id']: + logger.warning('Cannot commit without a group_id!') + raise KafkaConfigurationError('Attempted to commit offsets without a configured consumer group (group_id)') + + # API supports storing metadata with each commit + # but for now it is unused + metadata = '' + + offsets = self._offsets.task_done + commits = [] + for topic_partition, task_done_offset in offsets.iteritems(): + + # Skip if None + if task_done_offset is None: + continue + + # Commit offsets as the next offset to fetch + # which is consistent with the Java Client + # task_done is marked by messages consumed, + # so add one to mark the next message for fetching + commit_offset = (task_done_offset + 1) + + # Skip if no change from previous committed + if commit_offset == self._offsets.commit[topic_partition]: + continue + + commits.append(OffsetCommitRequest(topic_partition[0], topic_partition[1], commit_offset, metadata)) + + if commits: + logger.info('committing consumer offsets to group %s', self._config['group_id']) + resps = self._client.send_offset_commit_request(self._config['group_id'], + commits, + fail_on_error=False) + + for r in resps: + check_error(r) + topic_partition = (r.topic, r.partition) + task_done = self._offsets.task_done[topic_partition] + self._offsets.commit[topic_partition] = (task_done + 1) + + if self._config['auto_commit_enable']: + self._set_next_auto_commit_time() + + return True - # Skip if we already have a fetch offset from user args - if topic_partition not in self._offsets.fetch: + else: + logger.info('No new offsets found to commit in group %s', self._config['group_id']) + return False + + def configure(self, **configs): + """ + Configuration settings can be passed to constructor, + otherwise defaults will be used: + client_id='kafka.consumer.XXX', + group_id=None, + fetch_message_max_bytes=1024*1024, + fetch_min_bytes=1, + fetch_wait_max_ms=100, + refresh_leader_backoff_ms=200, + metadata_broker_list=None, + socket_timeout_ms=30*1000, + auto_offset_reset='largest', + deserializer_class=Event.from_bytes, + auto_commit_enable=False, + auto_commit_interval_ms=60 * 1000, + consumer_timeout_ms=-1 + + Configuration parameters are described in more detail at + http://kafka.apache.org/documentation.html#highlevelconsumerapi + """ + self._config = {} + for key in DEFAULT_CONSUMER_CONFIG: + self._config[key] = configs.pop(key, DEFAULT_CONSUMER_CONFIG[key]) + + if configs: + raise KafkaConfigurationError('Unknown configuration key(s): ' + + str(list(configs.keys()))) + + if self._config['auto_commit_enable']: + if not self._config['group_id']: + raise KafkaConfigurationError('KafkaConsumer configured to auto-commit without required consumer group (group_id)') + + # Check auto-commit configuration + if self._config['auto_commit_enable']: + logger.info("Configuring consumer to auto-commit offsets") + self._set_next_auto_commit_time() + + if self._config['metadata_broker_list'] is None: + raise KafkaConfigurationError('metadata_broker_list required to ' + 'configure KafkaConsumer') + + self._client = KafkaClient(self._config['metadata_broker_list'], + client_id=self._config['client_id'], + timeout=(self._config['socket_timeout_ms'] / 1000.0)) + + def set_topic_partitions(self, *topics): + """ + Set the topic/partitions to consume + Optionally specify offsets to start from + + Accepts types: + str: topic name (will consume all available partitions) + tuple: (topic, partition) + dict: { topic: partition } + { topic: [partition list] } + { topic: (partition tuple,) } + + Optionally, offsets can be specified directly: + tuple: (topic, partition, offset) + dict: { (topic, partition): offset, ... } + + Ex: + kafka = KafkaConsumer() + + # Consume topic1-all; topic2-partition2; topic3-partition0 + kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) + + # Consume topic1-0 starting at offset 123, and topic2-1 at offset 456 + # using tuples -- + kafka.set_topic_partitions(("topic1", 0, 123), ("topic2", 1, 456)) + + # using dict -- + kafka.set_topic_partitions({ ("topic1", 0): 123, ("topic2", 1): 456 }) + """ + self._topics = [] + self._client.load_metadata_for_topics() - # Fetch offsets default is (1) commit - if self._offsets.commit[topic_partition] is not None: - self._offsets.fetch[topic_partition] = self._offsets.commit[topic_partition] + # Setup offsets + self._offsets = OffsetsStruct(fetch=dict(), + commit=dict(), + highwater=dict(), + task_done=dict()) + + # Handle different topic types + for arg in topics: + + # Topic name str -- all partitions + if isinstance(arg, six.string_types): + topic = arg + for partition in self._client.get_partition_ids_for_topic(arg): + self._consume_topic_partition(topic, partition) + + # (topic, partition [, offset]) tuple + elif isinstance(arg, tuple): + (topic, partition) = arg[0:2] + if len(arg) == 3: + offset = arg[2] + self._offsets.fetch[(topic, partition)] = offset + self._consume_topic_partition(topic, partition) + + # { topic: partitions, ... } dict + elif isinstance(arg, dict): + for key, value in arg.iteritems(): + + # key can be string (a topic) + if isinstance(key, six.string_types): + + # topic: partition + if isinstance(value, int): + self._consume_topic_partition(key, value) + + # topic: [ partition1, partition2, ... ] + elif isinstance(value, (list, tuple)): + for partition in value: + self._consume_topic_partition(key, partition) + else: + raise KafkaConfigurationError('Unknown topic type (dict key must be ' + 'int or list/tuple of ints)') + + # (topic, partition): offset + elif isinstance(key, tuple): + self._consume_topic_partition(*key) + self._offsets.fetch[key] = value - # or (2) auto reset + else: + raise KafkaConfigurationError('Unknown topic type (%s)' % type(arg)) + + # If we have a consumer group, try to fetch stored offsets + if self._config['group_id']: + self._get_commit_offsets() + + # Update missing fetch/commit offsets + for topic_partition in self._topics: + + # Commit offsets default is None + if topic_partition not in self._offsets.commit: + self._offsets.commit[topic_partition] = None + + # Skip if we already have a fetch offset from user args + if topic_partition not in self._offsets.fetch: + + # Fetch offsets default is (1) commit + if self._offsets.commit[topic_partition] is not None: + self._offsets.fetch[topic_partition] = self._offsets.commit[topic_partition] + + # or (2) auto reset + else: + self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) + + # highwater marks (received from server on fetch response) + # and task_done (set locally by user) + # should always get initialized to None + self._reset_highwater_offsets() + self._reset_task_done_offsets() + + def _consume_topic_partition(self, topic, partition): + if not isinstance(topic, six.string_types): + raise KafkaConfigurationError('Unknown topic type (%s) ' + '-- expected string' % type(topic)) + if not isinstance(partition, int): + raise KafkaConfigurationError('Unknown partition type (%s) ' + '-- expected int' % type(partition)) + + if topic not in self._client.topic_partitions: + raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) + if partition not in self._client.get_partition_ids_for_topic(topic): + raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " + "in broker metadata" % (partition, topic)) + logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) + self._topics.append((topic, partition)) + + def fetch_messages(self): + + max_bytes = self._config['fetch_message_max_bytes'] + max_wait_time = self._config['fetch_wait_max_ms'] + min_bytes = self._config['fetch_min_bytes'] + + fetches = [] + offsets = self._offsets.fetch + for topic_partition, offset in offsets.iteritems(): + fetches.append(FetchRequest(topic_partition[0], topic_partition[1], offset, max_bytes)) + + # client.send_fetch_request will collect topic/partition requests by leader + # and send each group as a single FetchRequest to the correct broker + try: + responses = self._client.send_fetch_request(fetches, + max_wait_time=max_wait_time, + min_bytes=min_bytes, + fail_on_error=False) + except FailedPayloadsError: + logger.warning('FailedPayloadsError attempting to fetch data from kafka') + self._refresh_metadata_on_error() + return + + for resp in responses: + topic_partition = (resp.topic, resp.partition) + try: + check_error(resp) + except OffsetOutOfRangeError: + logger.warning('OffsetOutOfRange: topic %s, partition %d, offset %d ' + '(Highwatermark: %d)', + resp.topic, resp.partition, + offsets[topic_partition], resp.highwaterMark) + # Reset offset + self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) + continue + + except NotLeaderForPartitionError: + logger.warning("NotLeaderForPartitionError for %s - %d. " + "Metadata may be out of date", + resp.topic, resp.partition) + self._refresh_metadata_on_error() + continue + + except RequestTimedOutError: + logger.warning("RequestTimedOutError for %s - %d", + resp.topic, resp.partition) + continue + + # Track server highwater mark + self._offsets.highwater[topic_partition] = resp.highwaterMark + + # Yield each message + # Kafka-python could raise an exception during iteration + # we are not catching -- user will need to address + for (offset, message) in resp.messages: + # deserializer_class could raise an exception here + msg = KafkaMessage(resp.topic, + resp.partition, + offset, message.key, + self._config['deserializer_class'](message.value)) + + # Only increment fetch offset if we safely got the message and deserialized + self._offsets.fetch[topic_partition] = offset + 1 + + # Then yield to user + yield msg + + def _reset_partition_offset(self, topic_partition): + (topic, partition) = topic_partition + LATEST = -1 + EARLIEST = -2 + + request_time_ms = None + if self._config['auto_offset_reset'] == 'largest': + request_time_ms = LATEST + elif self._config['auto_offset_reset'] == 'smallest': + request_time_ms = EARLIEST else: - self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) - - # highwater marks (received from server on fetch response) - # and task_done (set locally by user) - # should always get initialized to None - self._reset_highwater_offsets() - self._reset_task_done_offsets() - - def _consume_topic_partition(self, topic, partition): - if not isinstance(topic, six.string_types): - raise KafkaConfigurationError('Unknown topic type (%s) ' - '-- expected string' % type(topic)) - if not isinstance(partition, int): - raise KafkaConfigurationError('Unknown partition type (%s) ' - '-- expected int' % type(partition)) - - if topic not in self._client.topic_partitions: - raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) - if partition not in self._client.get_partition_ids_for_topic(topic): - raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " - "in broker metadata" % (partition, topic)) - logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) - self._topics.append((topic, partition)) - - def fetch_messages(self): - - max_bytes = self._config['fetch_message_max_bytes'] - max_wait_time = self._config['fetch_wait_max_ms'] - min_bytes = self._config['fetch_min_bytes'] - - fetches = [] - offsets = self._offsets.fetch - for topic_partition, offset in offsets.iteritems(): - fetches.append(FetchRequest(topic_partition[0], topic_partition[1], offset, max_bytes)) - - # client.send_fetch_request will collect topic/partition requests by leader - # and send each group as a single FetchRequest to the correct broker - try: - responses = self._client.send_fetch_request(fetches, - max_wait_time=max_wait_time, - min_bytes=min_bytes, - fail_on_error=False) - except FailedPayloadsError: - logger.warning('FailedPayloadsError attempting to fetch data from kafka') - self._refresh_metadata_on_error() - return - - for resp in responses: - topic_partition = (resp.topic, resp.partition) - try: - check_error(resp) - except OffsetOutOfRangeError: - logger.warning('OffsetOutOfRange: topic %s, partition %d, offset %d ' - '(Highwatermark: %d)', - resp.topic, resp.partition, - offsets[topic_partition], resp.highwaterMark) - # Reset offset - self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) - continue - - except NotLeaderForPartitionError: - logger.warning("NotLeaderForPartitionError for %s - %d. " - "Metadata may be out of date", - resp.topic, resp.partition) - self._refresh_metadata_on_error() - continue - - except RequestTimedOutError: - logger.warning("RequestTimedOutError for %s - %d", - resp.topic, resp.partition) - continue - - # Track server highwater mark - self._offsets.highwater[topic_partition] = resp.highwaterMark - - # Yield each message - # Kafka-python could raise an exception during iteration - # we are not catching -- user will need to address - for (offset, message) in resp.messages: - # deserializer_class could raise an exception here - msg = KafkaMessage(resp.topic, - resp.partition, - offset, message.key, - self._config['deserializer_class'](message.value)) - - # Only increment fetch offset if we safely got the message and deserialized - self._offsets.fetch[topic_partition] = offset + 1 - - # Then yield to user - yield msg - - def _reset_partition_offset(self, topic_partition): - (topic, partition) = topic_partition - LATEST = -1 - EARLIEST = -2 - - request_time_ms = None - if self._config['auto_offset_reset'] == 'largest': - request_time_ms = LATEST - elif self._config['auto_offset_reset'] == 'smallest': - request_time_ms = EARLIEST - else: - - # Let's raise an reasonable exception type if user calls - # outside of an exception context - if sys.exc_info() == (None, None, None): - raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' - 'valid auto_offset_reset setting ' - '(largest|smallest)') - - # Otherwise we should re-raise the upstream exception - # b/c it typically includes additional data about - # the request that triggered it, and we do not want to drop that - raise - - (offset, ) = self.get_partition_offsets(topic, partition, - request_time_ms, max_num_offsets=1) - return offset - - def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): - """ - Request available fetch offsets for a single topic/partition - - @param topic (str) - @param partition (int) - @param request_time_ms (int) -- Used to ask for all messages before a - certain time (ms). There are two special - values. Specify -1 to receive the latest - offset (i.e. the offset of the next coming - message) and -2 to receive the earliest - available offset. Note that because offsets - are pulled in descending order, asking for - the earliest offset will always return you - a single element. - @param max_num_offsets (int) - - @return offsets (list) - """ - reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] - (resp,) = self._client.send_offset_request(reqs) + # Let's raise an reasonable exception type if user calls + # outside of an exception context + if sys.exc_info() == (None, None, None): + raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' + 'valid auto_offset_reset setting ' + '(largest|smallest)') + + # Otherwise we should re-raise the upstream exception + # b/c it typically includes additional data about + # the request that triggered it, and we do not want to drop that + raise + + (offset, ) = self.get_partition_offsets(topic, partition, + request_time_ms, max_num_offsets=1) + return offset + + def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): + """ + Request available fetch offsets for a single topic/partition + + @param topic (str) + @param partition (int) + @param request_time_ms (int) -- Used to ask for all messages before a + certain time (ms). There are two special + values. Specify -1 to receive the latest + offset (i.e. the offset of the next coming + message) and -2 to receive the earliest + available offset. Note that because offsets + are pulled in descending order, asking for + the earliest offset will always return you + a single element. + @param max_num_offsets (int) + + @return offsets (list) + """ + reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] + + (resp,) = self._client.send_offset_request(reqs) - check_error(resp) + check_error(resp) - # Just for sanity.. - # probably unnecessary - assert resp.topic == topic - assert resp.partition == partition + # Just for sanity.. + # probably unnecessary + assert resp.topic == topic + assert resp.partition == partition + + return resp.offsets + + def _refresh_metadata_on_error(self): + sleep_ms = self._config['refresh_leader_backoff_ms'] + while True: + logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) + time.sleep(sleep_ms / 1000.0) + try: + self._client.load_metadata_for_topics() + except KafkaUnavailableError: + logger.warning("Unable to refresh topic metadata... cluster unavailable") + self._check_consumer_timeout() + else: + logger.info("Topic metadata refreshed") + return - return resp.offsets + def _set_consumer_timeout_start(self): + self._consumer_timeout = False + if self._config['consumer_timeout_ms'] >= 0: + self._consumer_timeout = time.time() + (self._config['consumer_timeout_ms'] / 1000.0) - def _refresh_metadata_on_error(self): - sleep_ms = self._config['refresh_leader_backoff_ms'] - while True: - logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) - time.sleep(sleep_ms / 1000.0) - try: - self._client.load_metadata_for_topics() - except KafkaUnavailableError: - logger.warning("Unable to refresh topic metadata... cluster unavailable") - self._check_consumer_timeout() - else: - logger.info("Topic metadata refreshed") - return - - def _set_consumer_timeout_start(self): - self._consumer_timeout = False - if self._config['consumer_timeout_ms'] >= 0: - self._consumer_timeout = time.time() + (self._config['consumer_timeout_ms'] / 1000.0) - - def _check_consumer_timeout(self): - if self._consumer_timeout and time.time() > self._consumer_timeout: - raise ConsumerTimeout('Consumer timed out after %d ms' % + self._config['consumer_timeout_ms']) + def _check_consumer_timeout(self): + if self._consumer_timeout and time.time() > self._consumer_timeout: + raise ConsumerTimeout('Consumer timed out after %d ms' % + self._config['consumer_timeout_ms']) From 08f6ad94556256d710a5d4b517986111de32ffa1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 21:48:04 -0700 Subject: [PATCH 17/33] Reorder methods, add docstrings to public methds, section comments for private methods --- kafka/consumer/new.py | 462 +++++++++++++++++++++++------------------- 1 file changed, 255 insertions(+), 207 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index f0a442465..abafae819 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -124,171 +124,6 @@ def __init__(self, *topics, **configs): self.set_topic_partitions(*topics) self._msg_iter = None - def _get_commit_offsets(self): - logger.info("Consumer fetching stored offsets") - for topic_partition in self._topics: - (resp,) = self._client.send_offset_fetch_request( - self._config['group_id'], - [OffsetFetchRequest(topic_partition[0], topic_partition[1])], - fail_on_error=False) - try: - check_error(resp) - # API spec says server wont set an error here - # but 0.8.1.1 does actually... - except UnknownTopicOrPartitionError: - pass - - # -1 offset signals no commit is currently stored - if resp.offset == -1: - self._offsets.commit[topic_partition] = None - - # Otherwise we committed the stored offset - # and need to fetch the next one - else: - self._offsets.commit[topic_partition] = resp.offset - - def _reset_highwater_offsets(self): - for topic_partition in self._topics: - self._offsets.highwater[topic_partition] = None - - def _reset_task_done_offsets(self): - for topic_partition in self._topics: - self._offsets.task_done[topic_partition] = None - - def __repr__(self): - return '' % ', '.join(["%s-%d" % topic_partition - for topic_partition in - self._topics]) - - def __iter__(self): - return self - - def next(self): - self._set_consumer_timeout_start() - while True: - - # Fetch a new batch if needed - if self._msg_iter is None: - self._msg_iter = self.fetch_messages() - - # Check for auto-commit - if self._should_auto_commit(): - self.commit() - - try: - return self._msg_iter.next() - - # Handle batch completion - except StopIteration: - self._msg_iter = None - - self._check_consumer_timeout() - - def offsets(self, group=None): - if not group: - return { - 'fetch': self.offsets('fetch'), - 'commit': self.offsets('commit'), - 'task_done': self.offsets('task_done'), - 'highwater': self.offsets('highwater') - } - else: - return dict(deepcopy(getattr(self._offsets, group))) - - def task_done(self, message): - """ - Mark a fetched message as consumed. - Offsets for messages marked as "task_done" will be stored back - to the kafka cluster for this consumer group on commit() - """ - topic_partition = (message.topic, message.partition) - offset = message.offset - - # Warn on non-contiguous offsets - prev_done = self._offsets.task_done[topic_partition] - if prev_done is not None and offset != (prev_done + 1): - logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', - offset, prev_done) - - # Warn on smaller offsets than previous commit - # "commit" offsets are actually the offset of the next # message to fetch. - # so task_done should be compared with (commit - 1) - prev_done = (self._offsets.commit[topic_partition] - 1) - if prev_done is not None and (offset <= prev_done): - logger.warning('Marking task_done on a previously committed offset?: %d <= %d', - offset, prev_done) - - self._offsets.task_done[topic_partition] = offset - - def _should_auto_commit(self): - if not self._config['auto_commit_enable']: - return False - - if not self._next_commit: - return False - - return (time.time() >= self._next_commit) - - def _set_next_auto_commit_time(self): - self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) - - def commit(self): - """ - Store consumed message offsets (marked via task_done()) - to kafka cluster for this consumer_group. - - Note -- this functionality requires server version >=0.8.1.1 - see https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI - """ - if not self._config['group_id']: - logger.warning('Cannot commit without a group_id!') - raise KafkaConfigurationError('Attempted to commit offsets without a configured consumer group (group_id)') - - # API supports storing metadata with each commit - # but for now it is unused - metadata = '' - - offsets = self._offsets.task_done - commits = [] - for topic_partition, task_done_offset in offsets.iteritems(): - - # Skip if None - if task_done_offset is None: - continue - - # Commit offsets as the next offset to fetch - # which is consistent with the Java Client - # task_done is marked by messages consumed, - # so add one to mark the next message for fetching - commit_offset = (task_done_offset + 1) - - # Skip if no change from previous committed - if commit_offset == self._offsets.commit[topic_partition]: - continue - - commits.append(OffsetCommitRequest(topic_partition[0], topic_partition[1], commit_offset, metadata)) - - if commits: - logger.info('committing consumer offsets to group %s', self._config['group_id']) - resps = self._client.send_offset_commit_request(self._config['group_id'], - commits, - fail_on_error=False) - - for r in resps: - check_error(r) - topic_partition = (r.topic, r.partition) - task_done = self._offsets.task_done[topic_partition] - self._offsets.commit[topic_partition] = (task_done + 1) - - if self._config['auto_commit_enable']: - self._set_next_auto_commit_time() - - return True - - else: - logger.info('No new offsets found to commit in group %s', self._config['group_id']) - return False - def configure(self, **configs): """ Configuration settings can be passed to constructor, @@ -445,23 +280,56 @@ def set_topic_partitions(self, *topics): self._reset_highwater_offsets() self._reset_task_done_offsets() - def _consume_topic_partition(self, topic, partition): - if not isinstance(topic, six.string_types): - raise KafkaConfigurationError('Unknown topic type (%s) ' - '-- expected string' % type(topic)) - if not isinstance(partition, int): - raise KafkaConfigurationError('Unknown partition type (%s) ' - '-- expected int' % type(partition)) - if topic not in self._client.topic_partitions: - raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) - if partition not in self._client.get_partition_ids_for_topic(topic): - raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " - "in broker metadata" % (partition, topic)) - logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) - self._topics.append((topic, partition)) + def next(self): + """ + Return a single message from the message iterator + If consumer_timeout_ms is set, will raise ConsumerTimeout + if no message is available + Otherwise blocks indefinitely + + Note that this is also the method called internally during iteration: + ``` + for m in consumer: + pass + ``` + """ + self._set_consumer_timeout_start() + while True: + + # Fetch a new batch if needed + if self._msg_iter is None: + self._msg_iter = self.fetch_messages() + + # Check for auto-commit + if self._should_auto_commit(): + self.commit() + + try: + return self._msg_iter.next() + + # Handle batch completion + except StopIteration: + self._msg_iter = None + + self._check_consumer_timeout() def fetch_messages(self): + """ + Sends FetchRequests for all topic/partitions set for consumption + Returns a generator that yields KafkaMessage structs + after deserializing with the configured `deserializer_class` + + Refreshes metadata on errors, and resets fetch offset on + OffsetOutOfRange, per the configured `auto_offset_reset` policy + + Key configuration parameters: + `fetch_message_max_bytes` + `fetch_max_wait_ms` + `fetch_min_bytes` + `deserializer_class` + `auto_offset_reset` + """ max_bytes = self._config['fetch_message_max_bytes'] max_wait_time = self._config['fetch_wait_max_ms'] @@ -528,34 +396,6 @@ def fetch_messages(self): # Then yield to user yield msg - def _reset_partition_offset(self, topic_partition): - (topic, partition) = topic_partition - LATEST = -1 - EARLIEST = -2 - - request_time_ms = None - if self._config['auto_offset_reset'] == 'largest': - request_time_ms = LATEST - elif self._config['auto_offset_reset'] == 'smallest': - request_time_ms = EARLIEST - else: - - # Let's raise an reasonable exception type if user calls - # outside of an exception context - if sys.exc_info() == (None, None, None): - raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' - 'valid auto_offset_reset setting ' - '(largest|smallest)') - - # Otherwise we should re-raise the upstream exception - # b/c it typically includes additional data about - # the request that triggered it, and we do not want to drop that - raise - - (offset, ) = self.get_partition_offsets(topic, partition, - request_time_ms, max_num_offsets=1) - return offset - def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): """ Request available fetch offsets for a single topic/partition @@ -588,6 +428,123 @@ def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offse return resp.offsets + def offsets(self, group=None): + """ + Returns a copy of internal offsets struct + optional param: group [fetch|commit|task_done|highwater] + if no group specified, returns all groups + """ + if not group: + return { + 'fetch': self.offsets('fetch'), + 'commit': self.offsets('commit'), + 'task_done': self.offsets('task_done'), + 'highwater': self.offsets('highwater') + } + else: + return dict(deepcopy(getattr(self._offsets, group))) + + def task_done(self, message): + """ + Mark a fetched message as consumed. + Offsets for messages marked as "task_done" will be stored back + to the kafka cluster for this consumer group on commit() + """ + topic_partition = (message.topic, message.partition) + offset = message.offset + + # Warn on non-contiguous offsets + prev_done = self._offsets.task_done[topic_partition] + if prev_done is not None and offset != (prev_done + 1): + logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', + offset, prev_done) + + # Warn on smaller offsets than previous commit + # "commit" offsets are actually the offset of the next # message to fetch. + # so task_done should be compared with (commit - 1) + prev_done = (self._offsets.commit[topic_partition] - 1) + if prev_done is not None and (offset <= prev_done): + logger.warning('Marking task_done on a previously committed offset?: %d <= %d', + offset, prev_done) + + self._offsets.task_done[topic_partition] = offset + + def commit(self): + """ + Store consumed message offsets (marked via task_done()) + to kafka cluster for this consumer_group. + + Note -- this functionality requires server version >=0.8.1.1 + see https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + """ + if not self._config['group_id']: + logger.warning('Cannot commit without a group_id!') + raise KafkaConfigurationError('Attempted to commit offsets without a configured consumer group (group_id)') + + # API supports storing metadata with each commit + # but for now it is unused + metadata = '' + + offsets = self._offsets.task_done + commits = [] + for topic_partition, task_done_offset in offsets.iteritems(): + + # Skip if None + if task_done_offset is None: + continue + + # Commit offsets as the next offset to fetch + # which is consistent with the Java Client + # task_done is marked by messages consumed, + # so add one to mark the next message for fetching + commit_offset = (task_done_offset + 1) + + # Skip if no change from previous committed + if commit_offset == self._offsets.commit[topic_partition]: + continue + + commits.append(OffsetCommitRequest(topic_partition[0], topic_partition[1], commit_offset, metadata)) + + if commits: + logger.info('committing consumer offsets to group %s', self._config['group_id']) + resps = self._client.send_offset_commit_request(self._config['group_id'], + commits, + fail_on_error=False) + + for r in resps: + check_error(r) + topic_partition = (r.topic, r.partition) + task_done = self._offsets.task_done[topic_partition] + self._offsets.commit[topic_partition] = (task_done + 1) + + if self._config['auto_commit_enable']: + self._set_next_auto_commit_time() + + return True + + else: + logger.info('No new offsets found to commit in group %s', self._config['group_id']) + return False + + # + # Topic/partition management private methods + # + def _consume_topic_partition(self, topic, partition): + if not isinstance(topic, six.string_types): + raise KafkaConfigurationError('Unknown topic type (%s) ' + '-- expected string' % type(topic)) + if not isinstance(partition, int): + raise KafkaConfigurationError('Unknown partition type (%s) ' + '-- expected int' % type(partition)) + + if topic not in self._client.topic_partitions: + raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) + if partition not in self._client.get_partition_ids_for_topic(topic): + raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " + "in broker metadata" % (partition, topic)) + logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) + self._topics.append((topic, partition)) + def _refresh_metadata_on_error(self): sleep_ms = self._config['refresh_leader_backoff_ms'] while True: @@ -602,6 +559,71 @@ def _refresh_metadata_on_error(self): logger.info("Topic metadata refreshed") return + # + # Offset-managment private methods + # + def _get_commit_offsets(self): + logger.info("Consumer fetching stored offsets") + for topic_partition in self._topics: + (resp,) = self._client.send_offset_fetch_request( + self._config['group_id'], + [OffsetFetchRequest(topic_partition[0], topic_partition[1])], + fail_on_error=False) + try: + check_error(resp) + # API spec says server wont set an error here + # but 0.8.1.1 does actually... + except UnknownTopicOrPartitionError: + pass + + # -1 offset signals no commit is currently stored + if resp.offset == -1: + self._offsets.commit[topic_partition] = None + + # Otherwise we committed the stored offset + # and need to fetch the next one + else: + self._offsets.commit[topic_partition] = resp.offset + + def _reset_highwater_offsets(self): + for topic_partition in self._topics: + self._offsets.highwater[topic_partition] = None + + def _reset_task_done_offsets(self): + for topic_partition in self._topics: + self._offsets.task_done[topic_partition] = None + + def _reset_partition_offset(self, topic_partition): + (topic, partition) = topic_partition + LATEST = -1 + EARLIEST = -2 + + request_time_ms = None + if self._config['auto_offset_reset'] == 'largest': + request_time_ms = LATEST + elif self._config['auto_offset_reset'] == 'smallest': + request_time_ms = EARLIEST + else: + + # Let's raise an reasonable exception type if user calls + # outside of an exception context + if sys.exc_info() == (None, None, None): + raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' + 'valid auto_offset_reset setting ' + '(largest|smallest)') + + # Otherwise we should re-raise the upstream exception + # b/c it typically includes additional data about + # the request that triggered it, and we do not want to drop that + raise + + (offset, ) = self.get_partition_offsets(topic, partition, + request_time_ms, max_num_offsets=1) + return offset + + # + # Consumer Timeout private methods + # def _set_consumer_timeout_start(self): self._consumer_timeout = False if self._config['consumer_timeout_ms'] >= 0: @@ -610,3 +632,29 @@ def _set_consumer_timeout_start(self): def _check_consumer_timeout(self): if self._consumer_timeout and time.time() > self._consumer_timeout: raise ConsumerTimeout('Consumer timed out after %d ms' % + self._config['consumer_timeout_ms']) + + # + # Autocommit private methods + # + def _should_auto_commit(self): + if not self._config['auto_commit_enable']: + return False + + if not self._next_commit: + return False + + return (time.time() >= self._next_commit) + + def _set_next_auto_commit_time(self): + self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) + + # + # python private methods + # + def __repr__(self): + return '' % ', '.join(["%s-%d" % topic_partition + for topic_partition in + self._topics]) + + def __iter__(self): + return self From 7caf9bef491b368fd1dec4430c38332fec3dc1b6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 21:49:46 -0700 Subject: [PATCH 18/33] Add private methods to manage internal _msg_iter --- kafka/consumer/new.py | 30 ++++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index abafae819..8f243cd92 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -122,7 +122,6 @@ class KafkaConsumer(object): def __init__(self, *topics, **configs): self.configure(**configs) self.set_topic_partitions(*topics) - self._msg_iter = None def configure(self, **configs): """ @@ -280,6 +279,8 @@ def set_topic_partitions(self, *topics): self._reset_highwater_offsets() self._reset_task_done_offsets() + # Reset message iterator in case we were in the middle of one + self._reset_message_iterator() def next(self): """ @@ -297,20 +298,16 @@ def next(self): self._set_consumer_timeout_start() while True: - # Fetch a new batch if needed - if self._msg_iter is None: - self._msg_iter = self.fetch_messages() - # Check for auto-commit if self._should_auto_commit(): self.commit() try: - return self._msg_iter.next() + return self._get_message_iterator().next() # Handle batch completion except StopIteration: - self._msg_iter = None + self._reset_message_iterator() self._check_consumer_timeout() @@ -648,6 +645,22 @@ def _should_auto_commit(self): def _set_next_auto_commit_time(self): self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) + # + # Message iterator private methods + # + def __iter__(self): + return self + + def _get_message_iterator(self): + # Fetch a new batch if needed + if self._msg_iter is None: + self._msg_iter = self.fetch_messages() + + return self._msg_iter + + def _reset_message_iterator(self): + self._msg_iter = None + # # python private methods # @@ -655,6 +668,3 @@ def __repr__(self): return '' % ', '.join(["%s-%d" % topic_partition for topic_partition in self._topics]) - - def __iter__(self): - return self From 07ff623392d1398f801c95d9af3e0a388b049068 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 22:36:12 -0700 Subject: [PATCH 19/33] Move auto-commit checks to task_done; add support for auto_commit_interval_messages --- kafka/consumer/new.py | 38 +++++++++++++++++++++++++++----------- 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index 8f243cd92..ad453874a 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -31,6 +31,7 @@ 'deserializer_class': lambda msg: msg, 'auto_commit_enable': False, 'auto_commit_interval_ms': 60 * 1000, + 'auto_commit_interval_messages': None, 'consumer_timeout_ms': -1, # Currently unused @@ -139,6 +140,7 @@ def configure(self, **configs): deserializer_class=Event.from_bytes, auto_commit_enable=False, auto_commit_interval_ms=60 * 1000, + auto_commit_interval_messages=None, consumer_timeout_ms=-1 Configuration parameters are described in more detail at @@ -159,7 +161,7 @@ def configure(self, **configs): # Check auto-commit configuration if self._config['auto_commit_enable']: logger.info("Configuring consumer to auto-commit offsets") - self._set_next_auto_commit_time() + self._reset_auto_commit() if self._config['metadata_broker_list'] is None: raise KafkaConfigurationError('metadata_broker_list required to ' @@ -298,10 +300,6 @@ def next(self): self._set_consumer_timeout_start() while True: - # Check for auto-commit - if self._should_auto_commit(): - self.commit() - try: return self._get_message_iterator().next() @@ -466,6 +464,13 @@ def task_done(self, message): self._offsets.task_done[topic_partition] = offset + # Check for auto-commit + if self._config['auto_commit_enable']: + self._incr_auto_commit_message_count() + + if self._should_auto_commit(): + self.commit() + def commit(self): """ Store consumed message offsets (marked via task_done()) @@ -515,7 +520,7 @@ def commit(self): self._offsets.commit[topic_partition] = (task_done + 1) if self._config['auto_commit_enable']: - self._set_next_auto_commit_time() + self._reset_auto_commit() return True @@ -637,13 +642,24 @@ def _should_auto_commit(self): if not self._config['auto_commit_enable']: return False - if not self._next_commit: - return False + if self._config['auto_commit_interval_ms'] > 0: + if time.time() >= self._next_commit_time: + return True + + if self._config['auto_commit_interval_messages'] > 0: + if self._uncommitted_message_count >= self._config['auto_commit_interval_messages']: + return True + + return False - return (time.time() >= self._next_commit) + def _reset_auto_commit(self): + self._uncommitted_message_count = 0 + self._next_commit_time = None + if self._config['auto_commit_interval_ms'] > 0: + self._next_commit_time = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) - def _set_next_auto_commit_time(self): - self._next_commit = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) + def _incr_auto_commit_message_count(self, n=1): + self._uncommitted_message_count += n # # Message iterator private methods From 742af4f7e0bad6159e63ed4b369e34426ab9f670 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 22:36:53 -0700 Subject: [PATCH 20/33] Raise KafkaConfigurationError during fetch_messages if not topics/partitions configured --- kafka/consumer/new.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index ad453874a..bad1f3de2 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -330,8 +330,14 @@ def fetch_messages(self): max_wait_time = self._config['fetch_wait_max_ms'] min_bytes = self._config['fetch_min_bytes'] - fetches = [] + # Get current fetch offsets offsets = self._offsets.fetch + if not offsets: + if not self._topics: + raise KafkaConfigurationError('No topics or partitions configured') + raise KafkaConfigurationError('No fetch offsets found when calling fetch_messages') + + fetches = [] for topic_partition, offset in offsets.iteritems(): fetches.append(FetchRequest(topic_partition[0], topic_partition[1], offset, max_bytes)) From 391b53201f200ab246b78e76c6e7945c8af6846e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 15 Sep 2014 22:49:38 -0700 Subject: [PATCH 21/33] Update docstrings w/ current interface / config defaults --- kafka/consumer/new.py | 37 +++++++++++++++++++++---------------- 1 file changed, 21 insertions(+), 16 deletions(-) diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index bad1f3de2..e0884d3fc 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -54,9 +54,8 @@ class KafkaConsumer(object): for m in kafka: print m - # Alternate interface: next() - while True: - print kafka.next() + # Alternate interface: next() + print kafka.next() # Alternate interface: batch iteration while True: @@ -79,17 +78,18 @@ class KafkaConsumer(object): kafka.task_done(m) # Alternate interface: next() - while True: - m = kafka.next() - process_message(m) - kafka.task_done(m) + m = kafka.next() + process_message(m) + kafka.task_done(m) + + # If auto_commit_enable is False, remember to commit() periodically + kafka.commit() - # Batch process interface does not auto_commit! + # Batch process interface while True: for m in kafka.fetch_messages(): process_message(m) kafka.task_done(m) - kafka.commit() ``` messages (m) are namedtuples with attributes: @@ -97,7 +97,7 @@ class KafkaConsumer(object): m.partition: partition number (int) m.offset: message offset on topic-partition log (int) m.key: key (bytes - can be None) - m.value: message (output of deserializer_class - default is event object) + m.value: message (output of deserializer_class - default is raw bytes) Configuration settings can be passed to constructor, otherwise defaults will be used: @@ -110,12 +110,11 @@ class KafkaConsumer(object): metadata_broker_list=None, socket_timeout_ms=30*1000, auto_offset_reset='largest', - deserializer_class=Event.from_bytes, + deserializer_class=lambda msg: msg, auto_commit_enable=False, auto_commit_interval_ms=60 * 1000, consumer_timeout_ms=-1 - Configuration parameters are described in more detail at http://kafka.apache.org/documentation.html#highlevelconsumerapi """ @@ -137,7 +136,7 @@ def configure(self, **configs): metadata_broker_list=None, socket_timeout_ms=30*1000, auto_offset_reset='largest', - deserializer_class=Event.from_bytes, + deserializer_class=lambda msg: msg, auto_commit_enable=False, auto_commit_interval_ms=60 * 1000, auto_commit_interval_messages=None, @@ -345,9 +344,9 @@ def fetch_messages(self): # and send each group as a single FetchRequest to the correct broker try: responses = self._client.send_fetch_request(fetches, - max_wait_time=max_wait_time, - min_bytes=min_bytes, - fail_on_error=False) + max_wait_time=max_wait_time, + min_bytes=min_bytes, + fail_on_error=False) except FailedPayloadsError: logger.warning('FailedPayloadsError attempting to fetch data from kafka') self._refresh_metadata_on_error() @@ -537,6 +536,7 @@ def commit(self): # # Topic/partition management private methods # + def _consume_topic_partition(self, topic, partition): if not isinstance(topic, six.string_types): raise KafkaConfigurationError('Unknown topic type (%s) ' @@ -570,6 +570,7 @@ def _refresh_metadata_on_error(self): # # Offset-managment private methods # + def _get_commit_offsets(self): logger.info("Consumer fetching stored offsets") for topic_partition in self._topics: @@ -632,6 +633,7 @@ def _reset_partition_offset(self, topic_partition): # # Consumer Timeout private methods # + def _set_consumer_timeout_start(self): self._consumer_timeout = False if self._config['consumer_timeout_ms'] >= 0: @@ -644,6 +646,7 @@ def _check_consumer_timeout(self): # # Autocommit private methods # + def _should_auto_commit(self): if not self._config['auto_commit_enable']: return False @@ -670,6 +673,7 @@ def _incr_auto_commit_message_count(self, n=1): # # Message iterator private methods # + def __iter__(self): return self @@ -686,6 +690,7 @@ def _reset_message_iterator(self): # # python private methods # + def __repr__(self): return '' % ', '.join(["%s-%d" % topic_partition for topic_partition in From 6e6ae533272ed32d150221534a16d588e42f9c51 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 16 Sep 2014 16:00:14 -0700 Subject: [PATCH 22/33] Use six for py3 support in KafkaConsumer Log connection failures w/ traceback in kafka/client.py --- kafka/client.py | 6 ++--- kafka/consumer/new.py | 52 ++++++++++++++++++++++++++++++++----------- 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 8c786944e..bc3d85326 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -119,9 +119,9 @@ def _send_broker_unaware_request(self, payloads, encoder_fn, decoder_fn): response = conn.recv(requestId) return decoder_fn(response) - except Exception as e: - log.warning("Could not send request [%r] to server %s:%i, " - "trying next server: %s" % (requestId, host, port, e)) + except Exception: + log.exception("Could not send request [%r] to server %s:%i, " + "trying next server" % (requestId, host, port)) raise KafkaUnavailableError("All servers failed to process request") diff --git a/kafka/consumer/new.py b/kafka/consumer/new.py index e0884d3fc..90770b562 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/new.py @@ -43,6 +43,8 @@ 'rebalance_backoff_ms': 2000, } +BYTES_CONFIGURATION_KEYS = ('client_id', 'group_id') + class KafkaConsumer(object): """ @@ -153,6 +155,14 @@ def configure(self, **configs): raise KafkaConfigurationError('Unknown configuration key(s): ' + str(list(configs.keys()))) + # Handle str/bytes conversions + for config_key in BYTES_CONFIGURATION_KEYS: + if not (self._config[config_key] is None or + isinstance(self._config[config_key], six.binary_type)): + logger.warning("Converting configuration key '%s' to bytes" % + config_key) + self._config[config_key] = self._config[config_key].encode('utf-8') + if self._config['auto_commit_enable']: if not self._config['group_id']: raise KafkaConfigurationError('KafkaConsumer configured to auto-commit without required consumer group (group_id)') @@ -176,7 +186,7 @@ def set_topic_partitions(self, *topics): Optionally specify offsets to start from Accepts types: - str: topic name (will consume all available partitions) + str (utf-8): topic name (will consume all available partitions) tuple: (topic, partition) dict: { topic: partition } { topic: [partition list] } @@ -212,14 +222,20 @@ def set_topic_partitions(self, *topics): for arg in topics: # Topic name str -- all partitions - if isinstance(arg, six.string_types): + if isinstance(arg, (six.string_types, six.binary_type)): topic = arg + if isinstance(topic, six.string_types): + topic = topic.encode('utf-8') + for partition in self._client.get_partition_ids_for_topic(arg): self._consume_topic_partition(topic, partition) # (topic, partition [, offset]) tuple elif isinstance(arg, tuple): - (topic, partition) = arg[0:2] + topic = arg[0] + if isinstance(topic, six.string_types): + topic = topic.encode('utf-8') + partition = arg[1] if len(arg) == 3: offset = arg[2] self._offsets.fetch[(topic, partition)] = offset @@ -227,26 +243,33 @@ def set_topic_partitions(self, *topics): # { topic: partitions, ... } dict elif isinstance(arg, dict): - for key, value in arg.iteritems(): + for key, value in six.iteritems(arg): # key can be string (a topic) - if isinstance(key, six.string_types): + if isinstance(key, (six.string_types, six.binary_type)): + topic = key + if isinstance(topic, six.string_types): + topic = topic.encode('utf-8') # topic: partition if isinstance(value, int): - self._consume_topic_partition(key, value) + self._consume_topic_partition(topic, value) # topic: [ partition1, partition2, ... ] elif isinstance(value, (list, tuple)): for partition in value: - self._consume_topic_partition(key, partition) + self._consume_topic_partition(topic, partition) else: raise KafkaConfigurationError('Unknown topic type (dict key must be ' 'int or list/tuple of ints)') # (topic, partition): offset elif isinstance(key, tuple): - self._consume_topic_partition(*key) + topic = key[0] + if isinstance(topic, six.string_types): + topic = topic.encode('utf-8') + partition = key[1] + self._consume_topic_partition(topic, partition) self._offsets.fetch[key] = value else: @@ -300,7 +323,7 @@ def next(self): while True: try: - return self._get_message_iterator().next() + return six.next(self._get_message_iterator()) # Handle batch completion except StopIteration: @@ -337,7 +360,7 @@ def fetch_messages(self): raise KafkaConfigurationError('No fetch offsets found when calling fetch_messages') fetches = [] - for topic_partition, offset in offsets.iteritems(): + for topic_partition, offset in six.iteritems(offsets): fetches.append(FetchRequest(topic_partition[0], topic_partition[1], offset, max_bytes)) # client.send_fetch_request will collect topic/partition requests by leader @@ -494,7 +517,7 @@ def commit(self): offsets = self._offsets.task_done commits = [] - for topic_partition, task_done_offset in offsets.iteritems(): + for topic_partition, task_done_offset in six.iteritems(offsets): # Skip if None if task_done_offset is None: @@ -538,9 +561,9 @@ def commit(self): # def _consume_topic_partition(self, topic, partition): - if not isinstance(topic, six.string_types): + if not isinstance(topic, six.binary_type): raise KafkaConfigurationError('Unknown topic type (%s) ' - '-- expected string' % type(topic)) + '-- expected bytes' % type(topic)) if not isinstance(partition, int): raise KafkaConfigurationError('Unknown partition type (%s) ' '-- expected int' % type(partition)) @@ -677,6 +700,9 @@ def _incr_auto_commit_message_count(self, n=1): def __iter__(self): return self + def __next__(self): + return self.next() + def _get_message_iterator(self): # Fetch a new batch if needed if self._msg_iter is None: From 6b44828685e75117d9def8a80b165e6154f78e11 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 16 Sep 2014 22:31:19 -0700 Subject: [PATCH 23/33] Move KafkaConsumer to kafka.consumer.kafka module; make available for import from kafka at top-level --- kafka/__init__.py | 2 +- kafka/consumer/__init__.py | 3 ++- kafka/consumer/{new.py => kafka.py} | 4 ++-- test/test_consumer.py | 3 +-- test/test_consumer_integration.py | 3 +-- 5 files changed, 7 insertions(+), 8 deletions(-) rename kafka/consumer/{new.py => kafka.py} (99%) diff --git a/kafka/__init__.py b/kafka/__init__.py index 58ca61970..16b90948b 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -13,7 +13,7 @@ ) from kafka.producer import SimpleProducer, KeyedProducer from kafka.partitioner import RoundRobinPartitioner, HashedPartitioner -from kafka.consumer import SimpleConsumer, MultiProcessConsumer +from kafka.consumer import SimpleConsumer, MultiProcessConsumer, KafkaConsumer __all__ = [ 'KafkaClient', 'KafkaConnection', 'SimpleProducer', 'KeyedProducer', diff --git a/kafka/consumer/__init__.py b/kafka/consumer/__init__.py index d2fa30632..935f56e1e 100644 --- a/kafka/consumer/__init__.py +++ b/kafka/consumer/__init__.py @@ -1,6 +1,7 @@ from .simple import SimpleConsumer from .multiprocess import MultiProcessConsumer +from .kafka import KafkaConsumer __all__ = [ - 'SimpleConsumer', 'MultiProcessConsumer' + 'SimpleConsumer', 'MultiProcessConsumer', 'KafkaConsumer' ] diff --git a/kafka/consumer/new.py b/kafka/consumer/kafka.py similarity index 99% rename from kafka/consumer/new.py rename to kafka/consumer/kafka.py index 90770b562..af1a9cfb3 100644 --- a/kafka/consumer/new.py +++ b/kafka/consumer/kafka.py @@ -103,7 +103,7 @@ class KafkaConsumer(object): Configuration settings can be passed to constructor, otherwise defaults will be used: - client_id='kafka.consumer.XXX', + client_id='kafka.consumer.kafka', group_id=None, fetch_message_max_bytes=1024*1024, fetch_min_bytes=1, @@ -129,7 +129,7 @@ def configure(self, **configs): """ Configuration settings can be passed to constructor, otherwise defaults will be used: - client_id='kafka.consumer.XXX', + client_id='kafka.consumer.kafka', group_id=None, fetch_message_max_bytes=1024*1024, fetch_min_bytes=1, diff --git a/test/test_consumer.py b/test/test_consumer.py index b33e53719..7b8f3700b 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -2,8 +2,7 @@ from mock import MagicMock from . import unittest -from kafka.consumer import SimpleConsumer -from kafka.consumer.new import KafkaConsumer +from kafka import SimpleConsumer, KafkaConsumer from kafka.common import KafkaConfigurationError class TestKafkaConsumer(unittest.TestCase): diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index b4af70b78..c97081bdc 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -3,12 +3,11 @@ from six.moves import xrange -from kafka import SimpleConsumer, MultiProcessConsumer, create_message +from kafka import SimpleConsumer, MultiProcessConsumer, KafkaConsumer, create_message from kafka.common import ( ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout ) from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES -from kafka.consumer.new import KafkaConsumer from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( From 206560a74b56e7a2dcc7f358f24b5769a22769b5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 16 Sep 2014 22:52:13 -0700 Subject: [PATCH 24/33] Force absolue_imports in kafka/consumer/kafka.py --- kafka/consumer/kafka.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index af1a9cfb3..43e8c5560 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import + from collections import namedtuple from copy import deepcopy import logging From b80e83b7335a92fcbfcf25e38d51f24fc00c20ea Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 17 Sep 2014 13:49:23 -0700 Subject: [PATCH 25/33] Fix task_done checks when no previous commit exists; add test --- kafka/consumer/kafka.py | 11 ++++---- test/test_consumer_integration.py | 43 +++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 6 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 43e8c5560..705c70d6f 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -485,12 +485,11 @@ def task_done(self, message): offset, prev_done) # Warn on smaller offsets than previous commit - # "commit" offsets are actually the offset of the next # message to fetch. - # so task_done should be compared with (commit - 1) - prev_done = (self._offsets.commit[topic_partition] - 1) - if prev_done is not None and (offset <= prev_done): - logger.warning('Marking task_done on a previously committed offset?: %d <= %d', - offset, prev_done) + # "commit" offsets are actually the offset of the next message to fetch. + prev_commit = self._offsets.commit[topic_partition] + if prev_commit is not None and ((offset + 1) <= prev_commit): + logger.warning('Marking task_done on a previously committed offset?: %d (+1) <= %d', + offset, prev_commit) self._offsets.task_done[topic_partition] = offset diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index c97081bdc..1d28f8e6e 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -353,3 +353,46 @@ def test_kafka_consumer__blocking(self): messages.add((msg.partition, msg.offset)) self.assertEqual(len(messages), 5) self.assertGreaterEqual(t.interval, 1) + + @kafka_versions("0.8.1", "0.8.1.1") + def test_kafka_consumer__offset_commit_resume(self): + GROUP_ID = random_string(10) + + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) + + # Start a consumer + consumer1 = self.kafka_consumer( + group_id = GROUP_ID, + auto_commit_enable = True, + auto_commit_interval_ms = None, + auto_commit_interval_messages = 20, + auto_offset_reset='smallest', + ) + + # Grab the first 195 messages + output_msgs1 = [] + for _ in xrange(195): + m = consumer1.next() + output_msgs1.append(m) + consumer1.task_done(m) + self.assert_message_count(output_msgs1, 195) + + # The total offset across both partitions should be at 180 + consumer2 = self.kafka_consumer( + group_id = GROUP_ID, + auto_commit_enable = True, + auto_commit_interval_ms = None, + auto_commit_interval_messages = 20, + consumer_timeout_ms = 100, + auto_offset_reset='smallest', + ) + + # 181-200 + output_msgs2 = [] + with self.assertRaises(ConsumerTimeout): + while True: + m = consumer2.next() + output_msgs2.append(m) + self.assert_message_count(output_msgs2, 20) + self.assertEqual(len(set(output_msgs1) & set(output_msgs2)), 15) From 9e1b20306919eaa14774befead190ff52df19ba4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 17 Sep 2014 13:49:55 -0700 Subject: [PATCH 26/33] Merge conflict w/ assertEqual (assertEquals deprecated) --- test/test_client_integration.py | 16 ++++++++-------- test/test_codec.py | 8 ++++---- test/test_conn.py | 6 +++--- test/test_consumer_integration.py | 28 +++++++++++++++------------- test/test_failover_integration.py | 2 +- test/test_package.py | 22 +++++++++++----------- test/test_producer_integration.py | 20 ++++++++++---------- 7 files changed, 52 insertions(+), 50 deletions(-) diff --git a/test/test_client_integration.py b/test/test_client_integration.py index 0cd2c9eca..cc60778bb 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -32,12 +32,12 @@ def test_consume_none(self): fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp, = self.client.send_fetch_request([fetch]) - self.assertEquals(fetch_resp.error, 0) - self.assertEquals(fetch_resp.topic, self.topic) - self.assertEquals(fetch_resp.partition, 0) + self.assertEqual(fetch_resp.error, 0) + self.assertEqual(fetch_resp.topic, self.topic) + self.assertEqual(fetch_resp.partition, 0) messages = list(fetch_resp.messages) - self.assertEquals(len(messages), 0) + self.assertEqual(len(messages), 0) @kafka_versions("all") def test_ensure_topic_exists(self): @@ -58,10 +58,10 @@ def test_ensure_topic_exists(self): def test_commit_fetch_offsets(self): req = OffsetCommitRequest(self.topic, 0, 42, b"metadata") (resp,) = self.client.send_offset_commit_request(b"group", [req]) - self.assertEquals(resp.error, 0) + self.assertEqual(resp.error, 0) req = OffsetFetchRequest(self.topic, 0) (resp,) = self.client.send_offset_fetch_request(b"group", [req]) - self.assertEquals(resp.error, 0) - self.assertEquals(resp.offset, 42) - self.assertEquals(resp.metadata, b"") # Metadata isn't stored for now + self.assertEqual(resp.error, 0) + self.assertEqual(resp.offset, 42) + self.assertEqual(resp.metadata, b"") # Metadata isn't stored for now diff --git a/test/test_codec.py b/test/test_codec.py index 0ea1074d6..2d7670a36 100644 --- a/test/test_codec.py +++ b/test/test_codec.py @@ -15,14 +15,14 @@ def test_gzip(self): for i in xrange(1000): s1 = random_string(100) s2 = gzip_decode(gzip_encode(s1)) - self.assertEquals(s1, s2) + self.assertEqual(s1, s2) @unittest.skipUnless(has_snappy(), "Snappy not available") def test_snappy(self): for i in xrange(1000): s1 = random_string(100) s2 = snappy_decode(snappy_encode(s1)) - self.assertEquals(s1, s2) + self.assertEqual(s1, s2) @unittest.skipUnless(has_snappy(), "Snappy not available") def test_snappy_detect_xerial(self): @@ -53,7 +53,7 @@ def test_snappy_decode_xerial(self): + struct.pack('!i', block_len) + random_snappy \ + struct.pack('!i', block_len2) + random_snappy2 \ - self.assertEquals(snappy_decode(to_test), (b'SNAPPY' * 50) + (b'XERIAL' * 50)) + self.assertEqual(snappy_decode(to_test), (b'SNAPPY' * 50) + (b'XERIAL' * 50)) @unittest.skipUnless(has_snappy(), "Snappy not available") def test_snappy_encode_xerial(self): @@ -68,5 +68,5 @@ def test_snappy_encode_xerial(self): to_test = (b'SNAPPY' * 50) + (b'XERIAL' * 50) compressed = snappy_encode(to_test, xerial_compatible=True, xerial_blocksize=300) - self.assertEquals(compressed, to_ensure) + self.assertEqual(compressed, to_ensure) diff --git a/test/test_conn.py b/test/test_conn.py index 7b3beb717..2c8f3b290 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -120,7 +120,7 @@ def raise_error(*args): def test_recv(self): - self.assertEquals(self.conn.recv(self.config['request_id']), self.config['payload']) + self.assertEqual(self.conn.recv(self.config['request_id']), self.config['payload']) def test_recv__reconnects_on_dirty_conn(self): @@ -151,8 +151,8 @@ def raise_error(*args): def test_recv__doesnt_consume_extra_data_in_stream(self): # Here just test that each call to recv will return a single payload - self.assertEquals(self.conn.recv(self.config['request_id']), self.config['payload']) - self.assertEquals(self.conn.recv(self.config['request_id']), self.config['payload2']) + self.assertEqual(self.conn.recv(self.config['request_id']), self.config['payload']) + self.assertEqual(self.conn.recv(self.config['request_id']), self.config['payload2']) def test_close__object_is_reusable(self): diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 1d28f8e6e..007f78868 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -39,16 +39,16 @@ def send_messages(self, partition, messages): messages = [ create_message(self.msg(str(msg))) for msg in messages ] produce = ProduceRequest(self.topic, partition, messages = messages) resp, = self.client.send_produce_request([produce]) - self.assertEquals(resp.error, 0) + self.assertEqual(resp.error, 0) return [ x.value for x in messages ] def assert_message_count(self, messages, num_messages): # Make sure we got them all - self.assertEquals(len(messages), num_messages) + self.assertEqual(len(messages), num_messages) # Make sure there are no duplicates - self.assertEquals(len(set(messages)), num_messages) + self.assertEqual(len(set(messages)), num_messages) def consumer(self, **kwargs): if os.environ['KAFKA_VERSION'] == "0.8.0": @@ -140,9 +140,11 @@ def test_simple_consumer_pending(self): self.send_messages(0, range(0, 10)) self.send_messages(1, range(10, 20)) - self.assertEquals(consumer.pending(), 20) - self.assertEquals(consumer.pending(partitions=[0]), 10) - self.assertEquals(consumer.pending(partitions=[1]), 10) + consumer = self.consumer() + + self.assertEqual(consumer.pending(), 20) + self.assertEqual(consumer.pending(partitions=[0]), 10) + self.assertEqual(consumer.pending(partitions=[1]), 10) # move to last message, so one partition should have 1 pending # message and other 0 @@ -201,9 +203,9 @@ def test_multi_proc_pending(self): consumer = MultiProcessConsumer(self.client, "group1", self.topic, auto_commit=False) - self.assertEquals(consumer.pending(), 20) - self.assertEquals(consumer.pending(partitions=[0]), 10) - self.assertEquals(consumer.pending(partitions=[1]), 10) + self.assertEqual(consumer.pending(), 20) + self.assertEqual(consumer.pending(partitions=[0]), 10) + self.assertEqual(consumer.pending(partitions=[1]), 10) consumer.stop() @@ -251,7 +253,7 @@ def test_huge_messages(self): # Consume giant message successfully message = big_consumer.get_message(block=False, timeout=10) self.assertIsNotNone(message) - self.assertEquals(message.message.value, huge_message) + self.assertEqual(message.message.value, huge_message) big_consumer.stop() @@ -299,7 +301,7 @@ def test_fetch_buffer_size(self): consumer = self.consumer(buffer_size=1024, max_buffer_size=2048) messages = [ message for message in consumer ] - self.assertEquals(len(messages), 2) + self.assertEqual(len(messages), 2) @kafka_versions("all") def test_kafka_consumer(self): @@ -319,8 +321,8 @@ def test_kafka_consumer(self): if n >= 200: break - self.assertEquals(len(messages[0]), 100) - self.assertEquals(len(messages[1]), 100) + self.assertEqual(len(messages[0]), 100) + self.assertEqual(len(messages[1]), 100) @kafka_versions("all") def test_kafka_consumer__blocking(self): diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index d307d4165..ca71f2ddb 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -121,7 +121,7 @@ def _send_random_messages(self, producer, topic, partition, n): logging.debug('_send_random_message to %s:%d -- try %d', topic, partition, j) resp = producer.send_messages(topic, partition, random_string(10)) if len(resp) > 0: - self.assertEquals(resp[0].error, 0) + self.assertEqual(resp[0].error, 0) logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j) def _kill_leader(self, topic, partition): diff --git a/test/test_package.py b/test/test_package.py index 9b69a7cd8..e91753c0c 100644 --- a/test/test_package.py +++ b/test/test_package.py @@ -3,27 +3,27 @@ class TestPackage(unittest.TestCase): def test_top_level_namespace(self): import kafka as kafka1 - self.assertEquals(kafka1.KafkaClient.__name__, "KafkaClient") - self.assertEquals(kafka1.client.__name__, "kafka.client") - self.assertEquals(kafka1.codec.__name__, "kafka.codec") + self.assertEqual(kafka1.KafkaClient.__name__, "KafkaClient") + self.assertEqual(kafka1.client.__name__, "kafka.client") + self.assertEqual(kafka1.codec.__name__, "kafka.codec") def test_submodule_namespace(self): import kafka.client as client1 - self.assertEquals(client1.__name__, "kafka.client") - self.assertEquals(client1.KafkaClient.__name__, "KafkaClient") + self.assertEqual(client1.__name__, "kafka.client") + self.assertEqual(client1.KafkaClient.__name__, "KafkaClient") from kafka import client as client2 - self.assertEquals(client2.__name__, "kafka.client") - self.assertEquals(client2.KafkaClient.__name__, "KafkaClient") + self.assertEqual(client2.__name__, "kafka.client") + self.assertEqual(client2.KafkaClient.__name__, "KafkaClient") from kafka.client import KafkaClient as KafkaClient1 - self.assertEquals(KafkaClient1.__name__, "KafkaClient") + self.assertEqual(KafkaClient1.__name__, "KafkaClient") from kafka.codec import gzip_encode as gzip_encode1 - self.assertEquals(gzip_encode1.__name__, "gzip_encode") + self.assertEqual(gzip_encode1.__name__, "gzip_encode") from kafka import KafkaClient as KafkaClient2 - self.assertEquals(KafkaClient2.__name__, "KafkaClient") + self.assertEqual(KafkaClient2.__name__, "KafkaClient") from kafka.codec import snappy_encode - self.assertEquals(snappy_encode.__name__, "snappy_encode") + self.assertEqual(snappy_encode.__name__, "snappy_encode") diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index d68af7212..4331d23f6 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -251,7 +251,7 @@ def test_acks_none(self): producer = SimpleProducer(self.client, req_acks=SimpleProducer.ACK_NOT_REQUIRED) resp = producer.send_messages(self.topic, self.msg("one")) - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) producer.stop() @@ -301,7 +301,7 @@ def test_batched_simple_producer__triggers_by_message(self): ) # Batch mode is async. No ack - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) # It hasn't sent yet self.assert_fetch_offset(0, start_offset0, []) @@ -314,7 +314,7 @@ def test_batched_simple_producer__triggers_by_message(self): ) # Batch mode is async. No ack - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) self.assert_fetch_offset(0, start_offset0, [ self.msg("one"), @@ -350,7 +350,7 @@ def test_batched_simple_producer__triggers_by_time(self): ) # Batch mode is async. No ack - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) # It hasn't sent yet self.assert_fetch_offset(0, start_offset0, []) @@ -363,7 +363,7 @@ def test_batched_simple_producer__triggers_by_time(self): ) # Batch mode is async. No ack - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) # Wait the timeout out time.sleep(5) @@ -389,7 +389,7 @@ def test_async_simple_producer(self): producer = SimpleProducer(self.client, async=True) resp = producer.send_messages(self.topic, self.msg("one")) - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) @@ -402,7 +402,7 @@ def test_async_keyed_producer(self): producer = KeyedProducer(self.client, partitioner = RoundRobinPartitioner, async=True) resp = producer.send(self.topic, self.key("key1"), self.msg("one")) - self.assertEquals(len(resp), 0) + self.assertEqual(len(resp), 0) self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) @@ -429,9 +429,9 @@ def assert_fetch_offset(self, partition, start_offset, expected_messages): resp, = self.client.send_fetch_request([ FetchRequest(self.topic, partition, start_offset, 1024) ]) - self.assertEquals(resp.error, 0) - self.assertEquals(resp.partition, partition) + self.assertEqual(resp.error, 0) + self.assertEqual(resp.partition, partition) messages = [ x.message.value for x in resp.messages ] self.assertEqual(messages, expected_messages) - self.assertEquals(resp.highwaterMark, start_offset+len(expected_messages)) + self.assertEqual(resp.highwaterMark, start_offset+len(expected_messages)) From 1a06f79cb8ca0d68f06e517e2aad7f0b30c0278e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 17 Sep 2014 13:53:04 -0700 Subject: [PATCH 27/33] Make TIMEOUT_MS configurable in test_kafka_consumer__blocking --- test/test_consumer_integration.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 007f78868..ea32318af 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -326,14 +326,15 @@ def test_kafka_consumer(self): @kafka_versions("all") def test_kafka_consumer__blocking(self): + TIMEOUT_MS = 500 consumer = self.kafka_consumer(auto_offset_reset='smallest', - consumer_timeout_ms=1000) + consumer_timeout_ms=TIMEOUT_MS) # Ask for 5 messages, nothing in queue, block 5 seconds with Timer() as t: with self.assertRaises(ConsumerTimeout): msg = consumer.next() - self.assertGreaterEqual(t.interval, 1) + self.assertGreaterEqual(t.interval, TIMEOUT_MS / 1000.0 ) self.send_messages(0, range(0, 10)) @@ -344,7 +345,7 @@ def test_kafka_consumer__blocking(self): msg = consumer.next() messages.add((msg.partition, msg.offset)) self.assertEqual(len(messages), 5) - self.assertLess(t.interval, 1) + self.assertLess(t.interval, TIMEOUT_MS / 1000.0 ) # Ask for 10 messages, get 5 back, block 5 seconds messages = set() @@ -354,7 +355,7 @@ def test_kafka_consumer__blocking(self): msg = consumer.next() messages.add((msg.partition, msg.offset)) self.assertEqual(len(messages), 5) - self.assertGreaterEqual(t.interval, 1) + self.assertGreaterEqual(t.interval, TIMEOUT_MS / 1000.0 ) @kafka_versions("0.8.1", "0.8.1.1") def test_kafka_consumer__offset_commit_resume(self): From 08df93b3f971a3b75db2270e4d31530dbb60b5b0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 21 Sep 2014 19:34:48 -0700 Subject: [PATCH 28/33] Add private methods _does_auto_commit_ms and _does_auto_commit_messages --- kafka/consumer/kafka.py | 29 ++++++++++++++++++++++------- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 705c70d6f..af59e7ebb 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -494,7 +494,7 @@ def task_done(self, message): self._offsets.task_done[topic_partition] = offset # Check for auto-commit - if self._config['auto_commit_enable']: + if self._does_auto_commit_messages(): self._incr_auto_commit_message_count() if self._should_auto_commit(): @@ -672,14 +672,11 @@ def _check_consumer_timeout(self): # def _should_auto_commit(self): - if not self._config['auto_commit_enable']: - return False - - if self._config['auto_commit_interval_ms'] > 0: + if self._does_auto_commit_ms(): if time.time() >= self._next_commit_time: return True - if self._config['auto_commit_interval_messages'] > 0: + if self._does_auto_commit_messages(): if self._uncommitted_message_count >= self._config['auto_commit_interval_messages']: return True @@ -688,12 +685,30 @@ def _should_auto_commit(self): def _reset_auto_commit(self): self._uncommitted_message_count = 0 self._next_commit_time = None - if self._config['auto_commit_interval_ms'] > 0: + if self._does_auto_commit_ms(): self._next_commit_time = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) def _incr_auto_commit_message_count(self, n=1): self._uncommitted_message_count += n + def _does_auto_commit_ms(self): + if not self._config['auto_commit_enable']: + return False + + conf = self._config['auto_commit_interval_ms'] + if conf is not None and conf > 0: + return True + return False + + def _does_auto_commit_messages(self): + if not self._config['auto_commit_enable']: + return False + + conf = self._config['auto_commit_interval_messages'] + if conf is not None and conf > 0: + return True + return False + # # Message iterator private methods # From 209a8f28e6df2c5be8117b3bbb1b3188f1e29adc Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 21 Sep 2014 20:55:41 -0700 Subject: [PATCH 29/33] OffsetCommit metadata must be bytes --- kafka/consumer/kafka.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index af59e7ebb..069ad069c 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -514,7 +514,7 @@ def commit(self): # API supports storing metadata with each commit # but for now it is unused - metadata = '' + metadata = b'' offsets = self._offsets.task_done commits = [] From ed893c3bcfdf54f440b98b958c86fc0c13573b6f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Dec 2014 16:35:15 -0800 Subject: [PATCH 30/33] Use kafka.util.kafka_bytestring to encode utf-8 when necessary --- kafka/consumer/kafka.py | 21 ++++++--------------- kafka/util.py | 12 ++++++++++++ 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 069ad069c..f4eb6cb94 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -15,6 +15,7 @@ OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError ) +from kafka.util import kafka_bytestring logger = logging.getLogger(__name__) @@ -225,18 +226,14 @@ def set_topic_partitions(self, *topics): # Topic name str -- all partitions if isinstance(arg, (six.string_types, six.binary_type)): - topic = arg - if isinstance(topic, six.string_types): - topic = topic.encode('utf-8') + topic = kafka_bytestring(arg) for partition in self._client.get_partition_ids_for_topic(arg): self._consume_topic_partition(topic, partition) # (topic, partition [, offset]) tuple elif isinstance(arg, tuple): - topic = arg[0] - if isinstance(topic, six.string_types): - topic = topic.encode('utf-8') + topic = kafka_bytestring(arg[0]) partition = arg[1] if len(arg) == 3: offset = arg[2] @@ -249,9 +246,7 @@ def set_topic_partitions(self, *topics): # key can be string (a topic) if isinstance(key, (six.string_types, six.binary_type)): - topic = key - if isinstance(topic, six.string_types): - topic = topic.encode('utf-8') + topic = kafka_bytestring(key) # topic: partition if isinstance(value, int): @@ -267,9 +262,7 @@ def set_topic_partitions(self, *topics): # (topic, partition): offset elif isinstance(key, tuple): - topic = key[0] - if isinstance(topic, six.string_types): - topic = topic.encode('utf-8') + topic = kafka_bytestring(key[0]) partition = key[1] self._consume_topic_partition(topic, partition) self._offsets.fetch[key] = value @@ -562,9 +555,7 @@ def commit(self): # def _consume_topic_partition(self, topic, partition): - if not isinstance(topic, six.binary_type): - raise KafkaConfigurationError('Unknown topic type (%s) ' - '-- expected bytes' % type(topic)) + topic = kafka_bytestring(topic) if not isinstance(partition, int): raise KafkaConfigurationError('Unknown partition type (%s) ' '-- expected int' % type(partition)) diff --git a/kafka/util.py b/kafka/util.py index 1e03cf128..72ac5216f 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -86,6 +86,18 @@ def group_by_topic_and_partition(tuples): return out +def kafka_bytestring(s): + """ + Takes a string or bytes instance + Returns bytes, encoding strings in utf-8 as necessary + """ + if isinstance(s, six.binary_type): + return s + if isinstance(s, six.string_types): + return s.encode('utf-8') + raise TypeError(s) + + class ReentrantTimer(object): """ A timer that can be restarted, unlike threading.Timer From d27d49fd6b1c02dc764035cb06c3b47bf2a4b7a5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Dec 2014 16:36:12 -0800 Subject: [PATCH 31/33] Fixup call to self._client.get_partition_ids_for_topic -- use encoded topic bytes --- kafka/consumer/kafka.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index f4eb6cb94..f07bedfa4 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -228,7 +228,7 @@ def set_topic_partitions(self, *topics): if isinstance(arg, (six.string_types, six.binary_type)): topic = kafka_bytestring(arg) - for partition in self._client.get_partition_ids_for_topic(arg): + for partition in self._client.get_partition_ids_for_topic(topic): self._consume_topic_partition(topic, partition) # (topic, partition [, offset]) tuple From e3fd29cb37a9d661afcf913dfa2c4552638bb4fd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Dec 2014 16:36:41 -0800 Subject: [PATCH 32/33] Simplify BYTES_CONFIGURATION_KEYS logic, per wizzat review --- kafka/consumer/kafka.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index f07bedfa4..db86bab8e 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -160,8 +160,7 @@ def configure(self, **configs): # Handle str/bytes conversions for config_key in BYTES_CONFIGURATION_KEYS: - if not (self._config[config_key] is None or - isinstance(self._config[config_key], six.binary_type)): + if isinstance(self._config[config_key], six.string_types): logger.warning("Converting configuration key '%s' to bytes" % config_key) self._config[config_key] = self._config[config_key].encode('utf-8') From 29cae3e40b1d89d1a21525864794de7de3700461 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Dec 2014 16:37:25 -0800 Subject: [PATCH 33/33] Add some jitter to refresh_leader_backoff_ms, per wizzat review --- kafka/consumer/kafka.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index db86bab8e..f16b526e9 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -3,6 +3,7 @@ from collections import namedtuple from copy import deepcopy import logging +import random import sys import time @@ -568,7 +569,12 @@ def _consume_topic_partition(self, topic, partition): self._topics.append((topic, partition)) def _refresh_metadata_on_error(self): - sleep_ms = self._config['refresh_leader_backoff_ms'] + refresh_ms = self._config['refresh_leader_backoff_ms'] + jitter_pct = 0.20 + sleep_ms = random.randint( + int((1.0 - 0.5 * jitter_pct) * refresh_ms), + int((1.0 + 0.5 * jitter_pct) * refresh_ms) + ) while True: logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) time.sleep(sleep_ms / 1000.0)