From 878e4cefb696547d76f2eb0ad5b7afd2b1cfe044 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Tue, 6 Jan 2015 17:33:21 +0100 Subject: [PATCH 001/250] Implement offsets reset when OffsetOutOfRangeError This slightly changes the SimpleConsumer interface adding the default option use_latest_offsets. The fetch behaviour is also changed since it does not raise OffsetOutOfRangeError anymore. Resetting the offsets automatically is especially useful in MultiprocessConsumer, where an explicit seek call is not possible. --- kafka/consumer/simple.py | 38 +++++++++++++++++++++++++++++++++++--- 1 file changed, 35 insertions(+), 3 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index df975f4..5cd15b5 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -16,7 +16,9 @@ except ImportError: # python 2 from kafka.common import ( FetchRequest, OffsetRequest, - ConsumerFetchSizeTooSmall, ConsumerNoMoreData + ConsumerFetchSizeTooSmall, ConsumerNoMoreData, + UnknownTopicOrPartitionError, NotLeaderForPartitionError, + OffsetOutOfRangeError, check_error ) from .base import ( Consumer, @@ -98,7 +100,8 @@ class SimpleConsumer(Consumer): fetch_size_bytes=FETCH_MIN_BYTES, buffer_size=FETCH_BUFFER_SIZE_BYTES, max_buffer_size=MAX_FETCH_BUFFER_SIZE_BYTES, - iter_timeout=None): + iter_timeout=None, + use_latest_offsets=True): super(SimpleConsumer, self).__init__( client, group, topic, partitions=partitions, @@ -117,12 +120,26 @@ class SimpleConsumer(Consumer): self.fetch_min_bytes = fetch_size_bytes self.fetch_offsets = self.offsets.copy() self.iter_timeout = iter_timeout + self.use_latest_offsets = use_latest_offsets self.queue = Queue() def __repr__(self): return '' % \ (self.group, self.topic, str(self.offsets.keys())) + def reset_partition_offset(self, partition): + LATEST = -1 + EARLIEST = -2 + if self.use_latest_offsets: + req = OffsetRequest(self.topic, partition, LATEST, 1) + else: + req = OffsetRequest(self.topic, partition, EARLIEST, 1) + + resp = self.client.send_offset_request(req) + check_error(resp) + self.offsets[partition] = resp.offsets[0] + self.fetch_offsets[partition] = resp.offsets[0] + def provide_partition_info(self): """ Indicates that partition info must be returned by the consumer @@ -286,10 +303,25 @@ class SimpleConsumer(Consumer): responses = self.client.send_fetch_request( requests, max_wait_time=int(self.fetch_max_wait_time), - min_bytes=self.fetch_min_bytes) + min_bytes=self.fetch_min_bytes, + fail_on_error=False + ) retry_partitions = {} for resp in responses: + + try: + check_error(resp) + except (UnknownTopicOrPartitionError, NotLeaderForPartitionError): + self.client.reset_topic_metadata(resp.topic) + raise + except OffsetOutOfRangeError: + log.warning("OffsetOutOfRangeError for %s - %d. " + "Resetting partition offset...", + resp.topic, resp.partition) + self.reset_partition_offset(resp.partition) + continue + partition = resp.partition buffer_size = partitions[partition] try: From 6bc2c7aadac37c6e38c8a3c7be66013e9080aed7 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Wed, 14 Jan 2015 15:21:27 -0800 Subject: [PATCH 002/250] use a list in send_offset_request --- kafka/consumer/simple.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 5cd15b5..39103bd 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -131,11 +131,12 @@ class SimpleConsumer(Consumer): LATEST = -1 EARLIEST = -2 if self.use_latest_offsets: - req = OffsetRequest(self.topic, partition, LATEST, 1) + reqs = [OffsetRequest(self.topic, partition, LATEST, 1)] else: - req = OffsetRequest(self.topic, partition, EARLIEST, 1) + reqs = [OffsetRequest(self.topic, partition, EARLIEST, 1)] - resp = self.client.send_offset_request(req) + # send_offset_request + (resp, ) = self.client.send_offset_request(reqs) check_error(resp) self.offsets[partition] = resp.offsets[0] self.fetch_offsets[partition] = resp.offsets[0] From f517ddf283a86947a15f95e5ec562e81f4c477e5 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Mon, 26 Jan 2015 14:40:49 -0800 Subject: [PATCH 003/250] Make SimpleConsumer auto_offset_reset more like KafkaConsumer --- kafka/consumer/simple.py | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 39103bd..0593b5b 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -8,6 +8,7 @@ import logging import time import six +import sys try: from Queue import Empty, Queue @@ -87,6 +88,9 @@ class SimpleConsumer(Consumer): iter_timeout: default None. How much time (in seconds) to wait for a message in the iterator before exiting. None means no timeout, so it will wait forever. + auto_offset_reset: default largest. Reset partition offsets upon + OffsetOutOfRangeError. Valid values are largest and smallest. + If None do not reset the offsets and raise OffsetOutOfRangeError. Auto commit details: If both auto_commit_every_n and auto_commit_every_t are set, they will @@ -101,7 +105,7 @@ class SimpleConsumer(Consumer): buffer_size=FETCH_BUFFER_SIZE_BYTES, max_buffer_size=MAX_FETCH_BUFFER_SIZE_BYTES, iter_timeout=None, - use_latest_offsets=True): + auto_offset_reset='largest'): super(SimpleConsumer, self).__init__( client, group, topic, partitions=partitions, @@ -120,7 +124,7 @@ class SimpleConsumer(Consumer): self.fetch_min_bytes = fetch_size_bytes self.fetch_offsets = self.offsets.copy() self.iter_timeout = iter_timeout - self.use_latest_offsets = use_latest_offsets + self.auto_offset_reset = auto_offset_reset self.queue = Queue() def __repr__(self): @@ -130,10 +134,21 @@ class SimpleConsumer(Consumer): def reset_partition_offset(self, partition): LATEST = -1 EARLIEST = -2 - if self.use_latest_offsets: + if self.auto_offset_reset == 'largest': reqs = [OffsetRequest(self.topic, partition, LATEST, 1)] - else: + elif self.auto_offset_reset == 'smallest': reqs = [OffsetRequest(self.topic, partition, EARLIEST, 1)] + 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 # send_offset_request (resp, ) = self.client.send_offset_request(reqs) From 37d0b7f9dbac6c5165e6ea171a97be19c53c27f5 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Mon, 26 Jan 2015 16:24:20 -0800 Subject: [PATCH 004/250] Retry failed partitions and add integration tests --- kafka/consumer/simple.py | 2 ++ test/test_consumer_integration.py | 44 ++++++++++++++++++++++++++++++- 2 files changed, 45 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 2ec99f2..4c835fe 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -348,6 +348,8 @@ class SimpleConsumer(Consumer): "Resetting partition offset...", resp.topic, resp.partition) self.reset_partition_offset(resp.partition) + # Retry this partition + retry_partitions[resp.partition] = partitions[resp.partition] continue partition = resp.partition diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 4723220..9c89190 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -5,7 +5,7 @@ from six.moves import xrange from kafka import SimpleConsumer, MultiProcessConsumer, KafkaConsumer, create_message from kafka.common import ( - ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout + ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout, OffsetOutOfRangeError ) from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES @@ -85,6 +85,48 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer.stop() + @kafka_versions('all') + def test_simple_consumer_smallest_offset_reset(self): + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) + + consumer = self.consumer(auto_offset_reset='smallest') + # Move fetch offset ahead of 300 message (out of range) + consumer.seek(300, 2) + # Since auto_offset_reset is set to smallest we should read all 200 + # messages from beginning. + self.assert_message_count([message for message in consumer], 200) + + @kafka_versions('all') + def test_simple_consumer_largest_offset_reset(self): + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) + + # Default largest + consumer = self.consumer() + # Move fetch offset ahead of 300 message (out of range) + consumer.seek(300, 2) + # Since auto_offset_reset is set to largest we should not read any + # messages. + self.assert_message_count([message for message in consumer], 0) + # Send 200 new messages to the queue + self.send_messages(0, range(200, 300)) + self.send_messages(1, range(300, 400)) + # Since the offset is set to largest we should read all the new messages. + self.assert_message_count([message for message in consumer], 200) + + @kafka_versions('all') + def test_simple_consumer_no_reset(self): + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) + + # Default largest + consumer = self.consumer(auto_offset_reset=None) + # Move fetch offset ahead of 300 message (out of range) + consumer.seek(300, 2) + with self.assertRaises(OffsetOutOfRangeError): + consumer.get_message() + @kafka_versions("all") def test_simple_consumer__seek(self): self.send_messages(0, range(0, 100)) From 13b1dbe56f208cbb0bac4969d27d2d7e9ca7f27c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 3 Feb 2015 16:26:12 -0800 Subject: [PATCH 005/250] Bump VERSION for development of next release --- VERSION | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/VERSION b/VERSION index 965065d..8caff32 100644 --- a/VERSION +++ b/VERSION @@ -1 +1 @@ -0.9.3 +0.9.4-dev From 93e625ddd4af6e5a1551990defb6a1b809bafed1 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Feb 2015 19:34:04 -0500 Subject: [PATCH 006/250] Trying to get ReadTheDocs working --- docs/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/requirements.txt b/docs/requirements.txt index 94c50d8..86b4f05 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -4,4 +4,4 @@ sphinxcontrib-napoleon # Install kafka-python in editable mode # This allows the sphinx autodoc module # to load the Python modules and extract docstrings. --e .. +# -e .. From a4219999926b11d059382d17bd7f495b362c517c Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Feb 2015 19:44:03 -0500 Subject: [PATCH 007/250] Removing queue.py It's just collecting dust and throwing off the coverage report. I pushed a branch queue-fixup in case someone wants to take a stab at refactoring it. --- kafka/queue.py | 215 ------------------------------------------------- 1 file changed, 215 deletions(-) delete mode 100644 kafka/queue.py diff --git a/kafka/queue.py b/kafka/queue.py deleted file mode 100644 index 26cafad..0000000 --- a/kafka/queue.py +++ /dev/null @@ -1,215 +0,0 @@ -from __future__ import absolute_import - -from copy import copy -import logging -from multiprocessing import Process, Queue, Event -from Queue import Empty -import time - -from kafka.client import KafkaClient, FetchRequest, ProduceRequest - -log = logging.getLogger("kafka") - -raise NotImplementedError("Still need to refactor this class") - - -class KafkaConsumerProcess(Process): - def __init__(self, client, topic, partition, out_queue, barrier, - consumer_fetch_size=1024, consumer_sleep=200): - self.client = copy(client) - self.topic = topic - self.partition = partition - self.out_queue = out_queue - self.barrier = barrier - self.consumer_fetch_size = consumer_fetch_size - self.consumer_sleep = consumer_sleep / 1000. - log.info("Initializing %s" % self) - Process.__init__(self) - - def __str__(self): - return "[KafkaConsumerProcess: topic=%s, \ - partition=%s, sleep=%s]" % \ - (self.topic, self.partition, self.consumer_sleep) - - def run(self): - self.barrier.wait() - log.info("Starting %s" % self) - fetchRequest = FetchRequest(self.topic, self.partition, - offset=0, size=self.consumer_fetch_size) - - while True: - if self.barrier.is_set() is False: - log.info("Shutdown %s" % self) - self.client.close() - break - - lastOffset = fetchRequest.offset - (messages, fetchRequest) = self.client.get_message_set(fetchRequest) - - if fetchRequest.offset == lastOffset: - log.debug("No more data for this partition, " - "sleeping a bit (200ms)") - time.sleep(self.consumer_sleep) - continue - - for message in messages: - self.out_queue.put(message) - - -class KafkaProducerProcess(Process): - def __init__(self, client, topic, in_queue, barrier, - producer_flush_buffer=500, - producer_flush_timeout=2000, - producer_timeout=100): - - self.client = copy(client) - self.topic = topic - self.in_queue = in_queue - self.barrier = barrier - self.producer_flush_buffer = producer_flush_buffer - self.producer_flush_timeout = producer_flush_timeout / 1000. - self.producer_timeout = producer_timeout / 1000. - log.info("Initializing %s" % self) - Process.__init__(self) - - def __str__(self): - return "[KafkaProducerProcess: topic=%s, \ - flush_buffer=%s, flush_timeout=%s, timeout=%s]" % \ - (self.topic, - self.producer_flush_buffer, - self.producer_flush_timeout, - self.producer_timeout) - - def run(self): - self.barrier.wait() - log.info("Starting %s" % self) - messages = [] - last_produce = time.time() - - def flush(messages): - self.client.send_message_set(ProduceRequest(self.topic, -1, - messages)) - del messages[:] - - while True: - if self.barrier.is_set() is False: - log.info("Shutdown %s, flushing messages" % self) - flush(messages) - self.client.close() - break - - if len(messages) > self.producer_flush_buffer: - log.debug("Message count threshold reached. Flushing messages") - flush(messages) - last_produce = time.time() - - elif (time.time() - last_produce) > self.producer_flush_timeout: - log.debug("Producer timeout reached. Flushing messages") - flush(messages) - last_produce = time.time() - - try: - msg = KafkaClient.create_message( - self.in_queue.get(True, self.producer_timeout)) - messages.append(msg) - - except Empty: - continue - - -class KafkaQueue(object): - def __init__(self, client, topic, partitions, - producer_config=None, consumer_config=None): - """ - KafkaQueue a Queue-like object backed by a Kafka producer and some - number of consumers - - Messages are eagerly loaded by the consumer in batches of size - consumer_fetch_size. - Messages are buffered in the producer thread until - producer_flush_timeout or producer_flush_buffer is reached. - - Arguments: - client: KafkaClient object - topic: str, the topic name - partitions: list of ints, the partions to consume from - producer_config: dict, see below - consumer_config: dict, see below - - Consumer Config - =============== - consumer_fetch_size: int, number of bytes to fetch in one call - to Kafka. Default is 1024 - consumer_sleep: int, time in milliseconds a consumer should sleep - when it reaches the end of a partition. Default is 200 - - Producer Config - =============== - producer_timeout: int, time in milliseconds a producer should - wait for messages to enqueue for producing. - Default is 100 - producer_flush_timeout: int, time in milliseconds a producer - should allow messages to accumulate before - sending to Kafka. Default is 2000 - producer_flush_buffer: int, number of messages a producer should - allow to accumulate. Default is 500 - - """ - producer_config = {} if producer_config is None else producer_config - consumer_config = {} if consumer_config is None else consumer_config - - self.in_queue = Queue() - self.out_queue = Queue() - self.consumers = [] - self.barrier = Event() - - # Initialize and start consumer threads - for partition in partitions: - consumer = KafkaConsumerProcess(client, topic, partition, - self.in_queue, self.barrier, - **consumer_config) - consumer.start() - self.consumers.append(consumer) - - # Initialize and start producer thread - self.producer = KafkaProducerProcess(client, topic, self.out_queue, - self.barrier, **producer_config) - self.producer.start() - - # Trigger everything to start - self.barrier.set() - - def get(self, block=True, timeout=None): - """ - Consume a message from Kafka - - Arguments: - block: boolean, default True - timeout: int, number of seconds to wait when blocking, default None - - Returns: - msg: str, the payload from Kafka - """ - return self.in_queue.get(block, timeout).payload - - def put(self, msg, block=True, timeout=None): - """ - Send a message to Kafka - - Arguments: - msg: std, the message to send - block: boolean, default True - timeout: int, number of seconds to wait when blocking, default None - """ - self.out_queue.put(msg, block, timeout) - - def close(self): - """ - Close the internal queues and Kafka consumers/producer - """ - self.in_queue.close() - self.out_queue.close() - self.barrier.clear() - self.producer.join() - for consumer in self.consumers: - consumer.join() From c00dc274f9e35948221b2a4f6eb26c2b293aa93e Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Feb 2015 19:52:04 -0500 Subject: [PATCH 008/250] Removing traces of queue.py --- docs/api_reference.rst | 3 --- tox.ini | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/docs/api_reference.rst b/docs/api_reference.rst index 5e178f1..8804664 100644 --- a/docs/api_reference.rst +++ b/docs/api_reference.rst @@ -21,9 +21,6 @@ kafka .. automodule:: kafka.protocol :members: -.. automodule:: kafka.queue - :members: - .. automodule:: kafka.util :members: diff --git a/tox.ini b/tox.ini index 547470b..71c8993 100644 --- a/tox.ini +++ b/tox.ini @@ -36,4 +36,4 @@ deps = unittest2 mock pylint -commands = pylint {posargs: -E --ignore=queue.py kafka test} +commands = pylint {posargs: -E kafka test} From 91156f6ae4c65326fa6f20ea31949f90fe4751f9 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Feb 2015 19:54:39 -0500 Subject: [PATCH 009/250] Adding ReadTheDocs badge Because you can never have enough badges --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 7376362..cd12c2e 100644 --- a/README.md +++ b/README.md @@ -2,8 +2,9 @@ [![Build Status](https://api.travis-ci.org/mumrah/kafka-python.png?branch=master)](https://travis-ci.org/mumrah/kafka-python) [![Coverage Status](https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master)](https://coveralls.io/r/mumrah/kafka-python?branch=master) +[![Full documentation available on ReadTheDocs](https://readthedocs.org/projects/kafka-python/badge/?version=latest)](https://readthedocs.org/projects/kafka-python/?badge=latest) -[Full documentation available on ReadTheDocs](http://kafka-python.readthedocs.org/en/latest/) +[Full documentation available on ReadTheDocs](http://kafka-python.readthedocs.org/en/latest/) This module provides low-level protocol support for Apache Kafka as well as high-level consumer and producer classes. Request batching is supported by the From c29cfdc15606333036a2b1e5d501162a005473e1 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Feb 2015 20:03:07 -0500 Subject: [PATCH 010/250] 2014 -> 2015 --- README.md | 2 +- docs/index.rst | 2 +- kafka/__init__.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index cd12c2e..031382c 100644 --- a/README.md +++ b/README.md @@ -20,7 +20,7 @@ see https://groups.google.com/forum/m/#!forum/kafka-clients # License -Copyright 2014, David Arthur under Apache License, v2.0. See `LICENSE` +Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE` # Status diff --git a/docs/index.rst b/docs/index.rst index 82c3f57..96f0d3b 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -35,7 +35,7 @@ Python versions License ------- -Copyright 2014, David Arthur under Apache License, v2.0. See `LICENSE `_. +Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE `_. Contents diff --git a/kafka/__init__.py b/kafka/__init__.py index 8ccdb4c..3536084 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -4,7 +4,7 @@ import pkg_resources __version__ = pkg_resources.require('kafka-python')[0].version __author__ = 'David Arthur' __license__ = 'Apache License 2.0' -__copyright__ = 'Copyright 2014, David Arthur under Apache License, v2.0' +__copyright__ = 'Copyright 2015, David Arthur under Apache License, v2.0' from kafka.client import KafkaClient from kafka.conn import KafkaConnection From 6cceb762345973cede84d63e890e6e2648acc410 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 5 Feb 2015 10:54:11 -0800 Subject: [PATCH 011/250] Update docs/index.rst for sphinx/RTD --- docs/index.rst | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/index.rst b/docs/index.rst index 96f0d3b..7d9a3b6 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -17,20 +17,21 @@ see https://groups.google.com/forum/m/#!forum/kafka-clients Status ------ -The current stable version of this package is `0.9.2 `_ and is compatible with: +The current stable version of this package is `0.9.3 `_ and is compatible with: Kafka broker versions -* 0.8.0 -* 0.8.1 +* 0.8.2.0 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] * 0.8.1.1 +* 0.8.1 +* 0.8.0 Python versions - * 2.6 (tested on 2.6.9) -* 2.7 (tested on 2.7.8) -* pypy (tested on pypy 2.3.1 / python 2.7.6) -* (Python 3.3 and 3.4 support has been added to trunk and will be available the next release) +* 2.7 (tested on 2.7.9) +* 3.3 (tested on 3.3.5) +* 3.4 (tested on 3.4.2) +* pypy (tested on pypy 2.4.0 / python 2.7.8) License ------- From f206a4bbf0b0e39cf13c7c5918a99abc085d004d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 5 Feb 2015 10:58:01 -0800 Subject: [PATCH 012/250] Fixup index.rst --- docs/index.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/index.rst b/docs/index.rst index 7d9a3b6..79167b0 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -27,6 +27,7 @@ Kafka broker versions * 0.8.0 Python versions + * 2.6 (tested on 2.6.9) * 2.7 (tested on 2.7.9) * 3.3 (tested on 3.3.5) From cc6e8bbb22dc592224757d96579867c58b975803 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Feb 2015 21:35:16 -0800 Subject: [PATCH 013/250] Always return sorted partition ids in KafkaClient.get_partition_ids_for_topic() --- kafka/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index 7b04e71..f8fe555 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -261,7 +261,7 @@ class KafkaClient(object): if topic not in self.topic_partitions: return None - return list(self.topic_partitions[topic]) + return sorted(list(self.topic_partitions[topic])) def ensure_topic_exists(self, topic, timeout = 30): start_time = time.time() From be9f4099e0ff6f8bd299ed9979e42780b1b6a301 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 7 Feb 2015 21:53:33 -0800 Subject: [PATCH 014/250] use setUp / tearDown to rebuild fixtures for each test in test_failover_integration --- test/test_failover_integration.py | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index ca71f2d..3c74a44 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -17,8 +17,7 @@ from test.testutil import ( class TestFailover(KafkaIntegrationTestCase): create_client = False - @classmethod - def setUpClass(cls): # noqa + def setUp(self): if not os.environ.get('KAFKA_VERSION'): return @@ -27,22 +26,23 @@ class TestFailover(KafkaIntegrationTestCase): partitions = 2 # mini zookeeper, 2 kafka brokers - cls.zk = ZookeeperFixture.instance() - kk_args = [cls.zk.host, cls.zk.port, zk_chroot, replicas, partitions] - cls.brokers = [KafkaFixture.instance(i, *kk_args) for i in range(replicas)] + self.zk = ZookeeperFixture.instance() + kk_args = [self.zk.host, self.zk.port, zk_chroot, replicas, partitions] + self.brokers = [KafkaFixture.instance(i, *kk_args) for i in range(replicas)] - hosts = ['%s:%d' % (b.host, b.port) for b in cls.brokers] - cls.client = KafkaClient(hosts) + hosts = ['%s:%d' % (b.host, b.port) for b in self.brokers] + self.client = KafkaClient(hosts) + super(TestFailover, self).setUp() - @classmethod - def tearDownClass(cls): + def tearDown(self): + super(TestFailover, self).tearDown() if not os.environ.get('KAFKA_VERSION'): return - cls.client.close() - for broker in cls.brokers: + self.client.close() + for broker in self.brokers: broker.close() - cls.zk.close() + self.zk.close() @kafka_versions("all") def test_switch_leader(self): From 01f7c0d176e8430d3a117f7ca90b6853f937c5c3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 6 Feb 2015 11:50:34 -0800 Subject: [PATCH 015/250] Write a test for issue 313 - keyed producer failover --- test/test_failover_integration.py | 40 +++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 3c74a44..167a231 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -7,6 +7,7 @@ from . import unittest from kafka import KafkaClient, SimpleConsumer from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer +from kafka.producer import KeyedProducer from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( @@ -116,6 +117,45 @@ class TestFailover(KafkaIntegrationTestCase): # Should be equal to 10 before + 1 recovery + 10 after self.assert_message_count(topic, 21, partitions=(partition,)) + @kafka_versions("all") + def test_switch_leader_keyed_producer(self): + topic = self.topic + + producer = KeyedProducer(self.client, async=False) + + # Send 10 random messages + for _ in range(10): + key = random_string(3) + msg = random_string(10) + producer.send_messages(topic, key, msg) + + # kill leader for partition 0 + self._kill_leader(topic, 0) + + recovered = False + started = time.time() + timeout = 60 + while not recovered and (time.time() - started) < timeout: + try: + key = random_string(3) + msg = random_string(10) + producer.send_messages(topic, key, msg) + if producer.partitioners[topic].partition(key) == 0: + recovered = True + except (FailedPayloadsError, ConnectionError): + logging.debug("caught exception sending message -- will retry") + continue + + # Verify we successfully sent the message + self.assertTrue(recovered) + + # send some more messages just to make sure no more exceptions + for _ in range(10): + key = random_string(3) + msg = random_string(10) + producer.send_messages(topic, key, msg) + + def _send_random_messages(self, producer, topic, partition, n): for j in range(n): logging.debug('_send_random_message to %s:%d -- try %d', topic, partition, j) From bb731eb89cc58c2e6799ff2223c47011594986f1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Feb 2015 10:49:23 -0800 Subject: [PATCH 016/250] Add more comments to test_switch_leader regarding req_acks policy --- test/test_failover_integration.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 167a231..7d27526 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -50,11 +50,18 @@ class TestFailover(KafkaIntegrationTestCase): topic = self.topic partition = 0 - # Test the base class Producer -- send_messages to a specific partition + # Testing the base Producer class here so that we can easily send + # messages to a specific partition, kill the leader for that partition + # and check that after another broker takes leadership the producer + # is able to resume sending messages + + # require that the server commit messages to all in-sync replicas + # so that failover doesn't lose any messages on server-side + # and we can assert that server-side message count equals client-side producer = Producer(self.client, async=False, req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT) - # Send 10 random messages + # Send 100 random messages to a specific partition self._send_random_messages(producer, topic, partition, 100) # kill leader for partition @@ -81,7 +88,7 @@ class TestFailover(KafkaIntegrationTestCase): self._send_random_messages(producer, topic, partition, 100) # count number of messages - # Should be equal to 10 before + 1 recovery + 10 after + # Should be equal to 100 before + 1 recovery + 100 after self.assert_message_count(topic, 201, partitions=(partition,)) From fb03c69544221a25e925b8c6af45e44ac6944b2a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 6 Feb 2015 11:57:49 -0800 Subject: [PATCH 017/250] make partitions arg optional in *Partitioner.partition, defaulting to self.partitions from init --- kafka/partitioner/base.py | 7 +++---- kafka/partitioner/hashed.py | 4 +++- kafka/partitioner/roundrobin.py | 4 ++-- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/kafka/partitioner/base.py b/kafka/partitioner/base.py index 0b1bb59..857f634 100644 --- a/kafka/partitioner/base.py +++ b/kafka/partitioner/base.py @@ -12,14 +12,13 @@ class Partitioner(object): """ self.partitions = partitions - def partition(self, key, partitions): + def partition(self, key, partitions=None): """ Takes a string key and num_partitions as argument and returns a partition to be used for the message Arguments: - partitions: The list of partitions is passed in every call. This - may look like an overhead, but it will be useful - (in future) when we handle cases like rebalancing + key: the key to use for partitioning + partitions: (optional) a list of partitions. """ raise NotImplementedError('partition function has to be implemented') diff --git a/kafka/partitioner/hashed.py b/kafka/partitioner/hashed.py index 587a3de..fb5e598 100644 --- a/kafka/partitioner/hashed.py +++ b/kafka/partitioner/hashed.py @@ -5,7 +5,9 @@ class HashedPartitioner(Partitioner): Implements a partitioner which selects the target partition based on the hash of the key """ - def partition(self, key, partitions): + def partition(self, key, partitions=None): + if not partitions: + partitions = self.partitions size = len(partitions) idx = hash(key) % size diff --git a/kafka/partitioner/roundrobin.py b/kafka/partitioner/roundrobin.py index 54d00da..6439e53 100644 --- a/kafka/partitioner/roundrobin.py +++ b/kafka/partitioner/roundrobin.py @@ -15,9 +15,9 @@ class RoundRobinPartitioner(Partitioner): self.partitions = partitions self.iterpart = cycle(partitions) - def partition(self, key, partitions): + def partition(self, key, partitions=None): # Refresh the partition list if necessary - if self.partitions != partitions: + if partitions and self.partitions != partitions: self._set_partitions(partitions) return next(self.iterpart) From 36316c12759f8e17131cbf33e3b3d1c7899a3385 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 6 Feb 2015 11:58:54 -0800 Subject: [PATCH 018/250] Always use the initial partition list when hashing keys in KeyedProducer --- kafka/producer/keyed.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index fe5b056..36328ed 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -54,7 +54,7 @@ class KeyedProducer(Producer): self.partitioners[topic] = self.partitioner_class(self.client.get_partition_ids_for_topic(topic)) partitioner = self.partitioners[topic] - return partitioner.partition(key, self.client.get_partition_ids_for_topic(topic)) + return partitioner.partition(key) def send_messages(self,topic,key,*msg): partition = self._next_partition(topic, key) From c6bd6e1fa559b98561d7397e70791e4f594e4661 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Feb 2015 13:28:06 -0800 Subject: [PATCH 019/250] Use a different topic for each producer integration test for isolation --- test/test_producer_integration.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 19d28bd..d5831d3 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -19,7 +19,6 @@ from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import KafkaIntegrationTestCase, kafka_versions class TestKafkaProducerIntegration(KafkaIntegrationTestCase): - topic = b'produce_topic' @classmethod def setUpClass(cls): # noqa From 6ed6ad53f23fae8dd7ccadc15a7c84f67334d0c0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Feb 2015 20:07:25 -0800 Subject: [PATCH 020/250] Reorganize producer integration tests; protect against non-sorted partition metadata --- test/test_producer_integration.py | 281 +++++++++++++++++------------- 1 file changed, 160 insertions(+), 121 deletions(-) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index d5831d3..38df69f 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -14,6 +14,7 @@ from kafka.common import ( FetchRequest, ProduceRequest, UnknownTopicOrPartitionError, LeaderNotAvailableError ) +from kafka.producer.base import Producer from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import KafkaIntegrationTestCase, kafka_versions @@ -139,25 +140,26 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_simple_producer(self): - start_offset0 = self.current_offset(self.topic, 0) - start_offset1 = self.current_offset(self.topic, 1) + partitions = self.client.get_partition_ids_for_topic(self.topic) + start_offsets = [self.current_offset(self.topic, p) for p in partitions] + producer = SimpleProducer(self.client, random_start=False) # Goes to first partition, randomly. resp = producer.send_messages(self.topic, self.msg("one"), self.msg("two")) - self.assert_produce_response(resp, start_offset0) + self.assert_produce_response(resp, start_offsets[0]) # Goes to the next partition, randomly. resp = producer.send_messages(self.topic, self.msg("three")) - self.assert_produce_response(resp, start_offset1) + self.assert_produce_response(resp, start_offsets[1]) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one"), self.msg("two") ]) - self.assert_fetch_offset(1, start_offset1, [ self.msg("three") ]) + self.assert_fetch_offset(partitions[0], start_offsets[0], [ self.msg("one"), self.msg("two") ]) + self.assert_fetch_offset(partitions[1], start_offsets[1], [ self.msg("three") ]) # Goes back to the first partition because there's only two partitions resp = producer.send_messages(self.topic, self.msg("four"), self.msg("five")) - self.assert_produce_response(resp, start_offset0+2) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one"), self.msg("two"), self.msg("four"), self.msg("five") ]) + self.assert_produce_response(resp, start_offsets[0]+2) + self.assert_fetch_offset(partitions[0], start_offsets[0], [ self.msg("one"), self.msg("two"), self.msg("four"), self.msg("five") ]) producer.stop() @@ -193,110 +195,38 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assertEqual(resp3[0].partition, 0) @kafka_versions("all") - def test_round_robin_partitioner(self): - start_offset0 = self.current_offset(self.topic, 0) - start_offset1 = self.current_offset(self.topic, 1) + def test_async_simple_producer(self): + partition = self.client.get_partition_ids_for_topic(self.topic)[0] + start_offset = self.current_offset(self.topic, partition) - producer = KeyedProducer(self.client, partitioner=RoundRobinPartitioner) - resp1 = producer.send(self.topic, self.key("key1"), self.msg("one")) - resp2 = producer.send(self.topic, self.key("key2"), self.msg("two")) - resp3 = producer.send(self.topic, self.key("key3"), self.msg("three")) - resp4 = producer.send(self.topic, self.key("key4"), self.msg("four")) - - self.assert_produce_response(resp1, start_offset0+0) - self.assert_produce_response(resp2, start_offset1+0) - self.assert_produce_response(resp3, start_offset0+1) - self.assert_produce_response(resp4, start_offset1+1) - - self.assert_fetch_offset(0, start_offset0, [ self.msg("one"), self.msg("three") ]) - self.assert_fetch_offset(1, start_offset1, [ self.msg("two"), self.msg("four") ]) - - producer.stop() - - @kafka_versions("all") - def test_hashed_partitioner(self): - start_offset0 = self.current_offset(self.topic, 0) - start_offset1 = self.current_offset(self.topic, 1) - - producer = KeyedProducer(self.client, partitioner=HashedPartitioner) - resp1 = producer.send(self.topic, self.key("1"), self.msg("one")) - resp2 = producer.send(self.topic, self.key("2"), self.msg("two")) - resp3 = producer.send(self.topic, self.key("3"), self.msg("three")) - resp4 = producer.send(self.topic, self.key("3"), self.msg("four")) - resp5 = producer.send(self.topic, self.key("4"), self.msg("five")) - - offsets = {0: start_offset0, 1: start_offset1} - messages = {0: [], 1: []} - - keys = [self.key(k) for k in ["1", "2", "3", "3", "4"]] - resps = [resp1, resp2, resp3, resp4, resp5] - msgs = [self.msg(m) for m in ["one", "two", "three", "four", "five"]] - - for key, resp, msg in zip(keys, resps, msgs): - k = hash(key) % 2 - offset = offsets[k] - self.assert_produce_response(resp, offset) - offsets[k] += 1 - messages[k].append(msg) - - self.assert_fetch_offset(0, start_offset0, messages[0]) - self.assert_fetch_offset(1, start_offset1, messages[1]) - - producer.stop() - - @kafka_versions("all") - def test_acks_none(self): - start_offset0 = self.current_offset(self.topic, 0) - - producer = SimpleProducer(self.client, req_acks=SimpleProducer.ACK_NOT_REQUIRED, - random_start=False) + producer = SimpleProducer(self.client, async=True, random_start=False) resp = producer.send_messages(self.topic, self.msg("one")) self.assertEqual(len(resp), 0) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) - producer.stop() + # wait for the server to report a new highwatermark + while self.current_offset(self.topic, partition) == start_offset: + time.sleep(0.1) - @kafka_versions("all") - def test_acks_local_write(self): - start_offset0 = self.current_offset(self.topic, 0) - - producer = SimpleProducer(self.client, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE, - random_start=False) - resp = producer.send_messages(self.topic, self.msg("one")) - - self.assert_produce_response(resp, start_offset0) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) - - producer.stop() - - @kafka_versions("all") - def test_acks_cluster_commit(self): - start_offset0 = self.current_offset(self.topic, 0) - - producer = SimpleProducer( - self.client, - req_acks=SimpleProducer.ACK_AFTER_CLUSTER_COMMIT, - random_start=False) - - resp = producer.send_messages(self.topic, self.msg("one")) - self.assert_produce_response(resp, start_offset0) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) + self.assert_fetch_offset(partition, start_offset, [ self.msg("one") ]) producer.stop() @kafka_versions("all") def test_batched_simple_producer__triggers_by_message(self): - start_offset0 = self.current_offset(self.topic, 0) - start_offset1 = self.current_offset(self.topic, 1) + partitions = self.client.get_partition_ids_for_topic(self.topic) + start_offsets = [self.current_offset(self.topic, p) for p in partitions] + # Configure batch producer + batch_messages = 5 + batch_interval = 5 producer = SimpleProducer( self.client, batch_send=True, - batch_send_every_n=5, - batch_send_every_t=20, + batch_send_every_n=batch_messages, + batch_send_every_t=batch_interval, random_start=False) - # Send 5 messages and do a fetch + # Send 4 messages -- should not trigger a batch resp = producer.send_messages(self.topic, self.msg("one"), self.msg("two"), @@ -308,9 +238,10 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assertEqual(len(resp), 0) # It hasn't sent yet - self.assert_fetch_offset(0, start_offset0, []) - self.assert_fetch_offset(1, start_offset1, []) + self.assert_fetch_offset(partitions[0], start_offsets[0], []) + self.assert_fetch_offset(partitions[1], start_offsets[1], []) + # send 3 more messages -- should trigger batch on first 5 resp = producer.send_messages(self.topic, self.msg("five"), self.msg("six"), @@ -320,30 +251,32 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # Batch mode is async. No ack self.assertEqual(len(resp), 0) - self.assert_fetch_offset(0, start_offset0, [ + # send messages groups all *msgs in a single call to the same partition + # so we should see all messages from the first call in one partition + self.assert_fetch_offset(partitions[0], start_offsets[0], [ self.msg("one"), self.msg("two"), self.msg("three"), self.msg("four"), ]) - self.assert_fetch_offset(1, start_offset1, [ + # Because we are batching every 5 messages, we should only see one + self.assert_fetch_offset(partitions[1], start_offsets[1], [ self.msg("five"), - # self.msg("six"), - # self.msg("seven"), ]) producer.stop() @kafka_versions("all") def test_batched_simple_producer__triggers_by_time(self): - start_offset0 = self.current_offset(self.topic, 0) - start_offset1 = self.current_offset(self.topic, 1) + partitions = self.client.get_partition_ids_for_topic(self.topic) + start_offsets = [self.current_offset(self.topic, p) for p in partitions] + batch_interval = 5 producer = SimpleProducer(self.client, batch_send=True, batch_send_every_n=100, - batch_send_every_t=5, + batch_send_every_t=batch_interval, random_start=False) # Send 5 messages and do a fetch @@ -358,8 +291,8 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assertEqual(len(resp), 0) # It hasn't sent yet - self.assert_fetch_offset(0, start_offset0, []) - self.assert_fetch_offset(1, start_offset1, []) + self.assert_fetch_offset(partitions[0], start_offsets[0], []) + self.assert_fetch_offset(partitions[1], start_offsets[1], []) resp = producer.send_messages(self.topic, self.msg("five"), @@ -371,16 +304,16 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assertEqual(len(resp), 0) # Wait the timeout out - time.sleep(5) + time.sleep(batch_interval) - self.assert_fetch_offset(0, start_offset0, [ + self.assert_fetch_offset(partitions[0], start_offsets[0], [ self.msg("one"), self.msg("two"), self.msg("three"), self.msg("four"), ]) - self.assert_fetch_offset(1, start_offset1, [ + self.assert_fetch_offset(partitions[1], start_offsets[1], [ self.msg("five"), self.msg("six"), self.msg("seven"), @@ -388,40 +321,146 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): producer.stop() + + ############################ + # KeyedProducer Tests # + ############################ + @kafka_versions("all") - def test_async_simple_producer(self): - start_offset0 = self.current_offset(self.topic, 0) + def test_round_robin_partitioner(self): + partitions = self.client.get_partition_ids_for_topic(self.topic) + start_offsets = [self.current_offset(self.topic, p) for p in partitions] - producer = SimpleProducer(self.client, async=True, random_start=False) - resp = producer.send_messages(self.topic, self.msg("one")) - self.assertEqual(len(resp), 0) + producer = KeyedProducer(self.client, partitioner=RoundRobinPartitioner) + resp1 = producer.send(self.topic, self.key("key1"), self.msg("one")) + resp2 = producer.send(self.topic, self.key("key2"), self.msg("two")) + resp3 = producer.send(self.topic, self.key("key3"), self.msg("three")) + resp4 = producer.send(self.topic, self.key("key4"), self.msg("four")) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) + self.assert_produce_response(resp1, start_offsets[0]+0) + self.assert_produce_response(resp2, start_offsets[1]+0) + self.assert_produce_response(resp3, start_offsets[0]+1) + self.assert_produce_response(resp4, start_offsets[1]+1) + + self.assert_fetch_offset(partitions[0], start_offsets[0], [ self.msg("one"), self.msg("three") ]) + self.assert_fetch_offset(partitions[1], start_offsets[1], [ self.msg("two"), self.msg("four") ]) + + producer.stop() + + @kafka_versions("all") + def test_hashed_partitioner(self): + partitions = self.client.get_partition_ids_for_topic(self.topic) + start_offsets = [self.current_offset(self.topic, p) for p in partitions] + + producer = KeyedProducer(self.client, partitioner=HashedPartitioner) + resp1 = producer.send(self.topic, self.key("1"), self.msg("one")) + resp2 = producer.send(self.topic, self.key("2"), self.msg("two")) + resp3 = producer.send(self.topic, self.key("3"), self.msg("three")) + resp4 = producer.send(self.topic, self.key("3"), self.msg("four")) + resp5 = producer.send(self.topic, self.key("4"), self.msg("five")) + + offsets = {partitions[0]: start_offsets[0], partitions[1]: start_offsets[1]} + messages = {partitions[0]: [], partitions[1]: []} + + keys = [self.key(k) for k in ["1", "2", "3", "3", "4"]] + resps = [resp1, resp2, resp3, resp4, resp5] + msgs = [self.msg(m) for m in ["one", "two", "three", "four", "five"]] + + for key, resp, msg in zip(keys, resps, msgs): + k = hash(key) % 2 + partition = partitions[k] + offset = offsets[partition] + self.assert_produce_response(resp, offset) + offsets[partition] += 1 + messages[partition].append(msg) + + self.assert_fetch_offset(partitions[0], start_offsets[0], messages[partitions[0]]) + self.assert_fetch_offset(partitions[1], start_offsets[1], messages[partitions[1]]) producer.stop() @kafka_versions("all") def test_async_keyed_producer(self): - start_offset0 = self.current_offset(self.topic, 0) + partition = self.client.get_partition_ids_for_topic(self.topic)[0] + start_offset = self.current_offset(self.topic, partition) producer = KeyedProducer(self.client, partitioner = RoundRobinPartitioner, async=True) resp = producer.send(self.topic, self.key("key1"), self.msg("one")) self.assertEqual(len(resp), 0) - self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) + # wait for the server to report a new highwatermark + while self.current_offset(self.topic, partition) == start_offset: + time.sleep(0.1) + + self.assert_fetch_offset(partition, start_offset, [ self.msg("one") ]) producer.stop() - def assert_produce_request(self, messages, initial_offset, message_ct): - produce = ProduceRequest(self.topic, 0, messages=messages) + ############################ + # Producer ACK Tests # + ############################ + + @kafka_versions("all") + def test_acks_none(self): + partition = self.client.get_partition_ids_for_topic(self.topic)[0] + start_offset = self.current_offset(self.topic, partition) + + producer = Producer( + self.client, + req_acks=Producer.ACK_NOT_REQUIRED, + ) + resp = producer.send_messages(self.topic, partition, self.msg("one")) + + # No response from produce request with no acks required + self.assertEqual(len(resp), 0) + + # But the message should still have been delivered + self.assert_fetch_offset(partition, start_offset, [ self.msg("one") ]) + producer.stop() + + @kafka_versions("all") + def test_acks_local_write(self): + partition = self.client.get_partition_ids_for_topic(self.topic)[0] + start_offset = self.current_offset(self.topic, partition) + + producer = Producer( + self.client, + req_acks=Producer.ACK_AFTER_LOCAL_WRITE, + ) + resp = producer.send_messages(self.topic, partition, self.msg("one")) + + self.assert_produce_response(resp, start_offset) + self.assert_fetch_offset(partition, start_offset, [ self.msg("one") ]) + + producer.stop() + + @kafka_versions("all") + def test_acks_cluster_commit(self): + partition = self.client.get_partition_ids_for_topic(self.topic)[0] + start_offset = self.current_offset(self.topic, partition) + + producer = Producer( + self.client, + req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT, + ) + + resp = producer.send_messages(self.topic, partition, self.msg("one")) + self.assert_produce_response(resp, start_offset) + self.assert_fetch_offset(partition, start_offset, [ self.msg("one") ]) + + producer.stop() + + def assert_produce_request(self, messages, initial_offset, message_ct, + partition=0): + produce = ProduceRequest(self.topic, partition, messages=messages) # There should only be one response message from the server. # This will throw an exception if there's more than one. resp = self.client.send_produce_request([ produce ]) self.assert_produce_response(resp, initial_offset) - self.assertEqual(self.current_offset(self.topic, 0), initial_offset + message_ct) + self.assertEqual(self.current_offset(self.topic, partition), initial_offset + message_ct) def assert_produce_response(self, resp, initial_offset): self.assertEqual(len(resp), 1) From c88e3cedf44c257a9dc4c00017114fca3f4e58c0 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Mon, 9 Feb 2015 13:50:29 -0800 Subject: [PATCH 021/250] Allow better usage of docs locally. This will default to the rtd theme and introduces tox -e docs for generating docs locally. --- docs/conf.py | 7 +++++++ docs/index.rst | 3 --- tox.ini | 12 +++++++++++- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/docs/conf.py b/docs/conf.py index 9e95f79..25b3d4d 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -262,3 +262,10 @@ texinfo_documents = [ # If true, do not generate a @detailmenu in the "Top" node's menu. #texinfo_no_detailmenu = False + +on_rtd = os.environ.get('READTHEDOCS', None) == 'True' + +if not on_rtd: # only import and set the theme if we're building docs locally + import sphinx_rtd_theme + html_theme = 'sphinx_rtd_theme' + html_theme_path = [sphinx_rtd_theme.get_html_theme_path()] diff --git a/docs/index.rst b/docs/index.rst index 79167b0..cdcddbc 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,4 +1,3 @@ - kafka-python ============ @@ -51,11 +50,9 @@ Contents usage api_reference - Indices and tables ================== * :ref:`genindex` * :ref:`modindex` * :ref:`search` - diff --git a/tox.ini b/tox.ini index 71c8993..9ce36b1 100644 --- a/tox.ini +++ b/tox.ini @@ -1,5 +1,6 @@ [tox] -envlist = lint, py26, py27, pypy, py33, py34 +envlist = lint, py26, py27, pypy, py33, py34, docs + [testenv] deps = six @@ -37,3 +38,12 @@ deps = mock pylint commands = pylint {posargs: -E kafka test} + +[testenv:docs] +deps = + sphinxcontrib-napoleon + sphinx_rtd_theme + sphinx + +commands = + sphinx-build -b html docs/ docs/_build From aa19d71f9804f431199f4f68087ad4a53b0cbab9 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Mon, 9 Feb 2015 14:27:42 -0800 Subject: [PATCH 022/250] Updated documentation for Consumers to prefer KafkaConsumer instead --- docs/usage.rst | 8 +++++--- kafka/consumer/kafka.py | 7 +++++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/docs/usage.rst b/docs/usage.rst index 5f3fcea..141cf93 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -6,7 +6,7 @@ High level .. code:: python - from kafka import KafkaClient, SimpleProducer, SimpleConsumer + from kafka import SimpleProducer, KafkaClient, KafkaConsumer # To send messages synchronously kafka = KafkaClient("localhost:9092") @@ -52,7 +52,8 @@ High level batch_send_every_t=60) # To consume messages - consumer = SimpleConsumer(kafka, "my-group", "my-topic") + consumer = KafkaConsumer("my-topic", group_id="my_group", + metadata_broker_list=["localhost:9092"]) for message in consumer: # message is raw byte string -- decode if necessary! # e.g., for unicode: `message.decode('utf-8')` @@ -66,7 +67,8 @@ Keyed messages .. code:: python - from kafka import KafkaClient, KeyedProducer, HashedPartitioner, RoundRobinPartitioner + from kafka import (KafkaClient, KeyedProducer, HashedPartitioner, + RoundRobinPartitioner) kafka = KafkaClient("localhost:9092") diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index ae0f0b9..cd3cc4a 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -57,7 +57,8 @@ class KafkaConsumer(object): .. code:: python # A very basic 'tail' consumer, with no stored offset management - kafka = KafkaConsumer('topic1') + kafka = KafkaConsumer('topic1', + metadata_broker_list=['localhost:9092']) for m in kafka: print m @@ -73,8 +74,10 @@ class KafkaConsumer(object): .. code:: python - # more advanced consumer -- multiple topics w/ auto commit offset management + # more advanced consumer -- multiple topics w/ auto commit offset + # management kafka = KafkaConsumer('topic1', 'topic2', + metadata_broker_list=['localhost:9092'], group_id='my_consumer_group', auto_commit_enable=True, auto_commit_interval_ms=30 * 1000, From e7f52f2aba6fc59c0221ae08fd931fc5276d7eef Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 08:59:28 -0800 Subject: [PATCH 023/250] Use sphinx-apidoc to to generate API docs --- .gitignore | 1 + docs/api_reference.rst | 64 ------------------------------------------ docs/conf.py | 3 +- docs/index.rst | 2 +- tox.ini | 1 + 5 files changed, 5 insertions(+), 66 deletions(-) delete mode 100644 docs/api_reference.rst diff --git a/.gitignore b/.gitignore index 30d663d..aeeb474 100644 --- a/.gitignore +++ b/.gitignore @@ -9,3 +9,4 @@ servers/*/kafka-bin .coverage .noseids docs/_build +docs/apidoc diff --git a/docs/api_reference.rst b/docs/api_reference.rst deleted file mode 100644 index 8804664..0000000 --- a/docs/api_reference.rst +++ /dev/null @@ -1,64 +0,0 @@ -API Reference -============= - -kafka ------ -.. automodule:: kafka.client - :members: - -.. automodule:: kafka.codec - :members: - -.. automodule:: kafka.common - :members: - -.. automodule:: kafka.conn - :members: - -.. automodule:: kafka.context - :members: - -.. automodule:: kafka.protocol - :members: - -.. automodule:: kafka.util - :members: - - -kafka.consumer --------------- -.. automodule:: kafka.consumer.base - :members: - -.. automodule:: kafka.consumer.kafka - :members: - -.. automodule:: kafka.consumer.multiprocess - :members: - -.. automodule:: kafka.consumer.simple - :members: - - -kafka.partitioner ------------------ -.. automodule:: kafka.partitioner.base - :members: - -.. automodule:: kafka.partitioner.hashed - :members: - -.. automodule:: kafka.partitioner.roundrobin - :members: - - -kafka.producer --------------- -.. automodule:: kafka.producer.base - :members: - -.. automodule:: kafka.producer.keyed - :members: - -.. automodule:: kafka.producer.simple - :members: diff --git a/docs/conf.py b/docs/conf.py index 25b3d4d..ea223c2 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -30,6 +30,7 @@ import os # ones. extensions = [ 'sphinx.ext.autodoc', + 'sphinx.ext.intersphinx', 'sphinx.ext.viewcode', 'sphinxcontrib.napoleon', ] @@ -57,7 +58,7 @@ copyright = u'2015, David Arthur' # The short X.Y version. with open('../VERSION') as version_file: version = version_file.read() - + # The full version, including alpha/beta/rc tags. release = version diff --git a/docs/index.rst b/docs/index.rst index cdcddbc..e4a9ac7 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -48,7 +48,7 @@ Contents install tests usage - api_reference + API reference Indices and tables ================== diff --git a/tox.ini b/tox.ini index 9ce36b1..71565fd 100644 --- a/tox.ini +++ b/tox.ini @@ -46,4 +46,5 @@ deps = sphinx commands = + sphinx-apidoc -o docs/apidoc/ kafka/ sphinx-build -b html docs/ docs/_build From 70b3af1b3c742973609151f408b60ebb8aff5701 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 09:29:32 -0800 Subject: [PATCH 024/250] Need to commit apidoc for RTD --- .gitignore | 1 - docs/apidoc/kafka.consumer.rst | 46 ++++++++++++++++++ docs/apidoc/kafka.partitioner.rst | 38 +++++++++++++++ docs/apidoc/kafka.producer.rst | 38 +++++++++++++++ docs/apidoc/kafka.rst | 79 +++++++++++++++++++++++++++++++ docs/apidoc/modules.rst | 7 +++ 6 files changed, 208 insertions(+), 1 deletion(-) create mode 100644 docs/apidoc/kafka.consumer.rst create mode 100644 docs/apidoc/kafka.partitioner.rst create mode 100644 docs/apidoc/kafka.producer.rst create mode 100644 docs/apidoc/kafka.rst create mode 100644 docs/apidoc/modules.rst diff --git a/.gitignore b/.gitignore index aeeb474..30d663d 100644 --- a/.gitignore +++ b/.gitignore @@ -9,4 +9,3 @@ servers/*/kafka-bin .coverage .noseids docs/_build -docs/apidoc diff --git a/docs/apidoc/kafka.consumer.rst b/docs/apidoc/kafka.consumer.rst new file mode 100644 index 0000000..8595f99 --- /dev/null +++ b/docs/apidoc/kafka.consumer.rst @@ -0,0 +1,46 @@ +kafka.consumer package +====================== + +Submodules +---------- + +kafka.consumer.base module +-------------------------- + +.. automodule:: kafka.consumer.base + :members: + :undoc-members: + :show-inheritance: + +kafka.consumer.kafka module +--------------------------- + +.. automodule:: kafka.consumer.kafka + :members: + :undoc-members: + :show-inheritance: + +kafka.consumer.multiprocess module +---------------------------------- + +.. automodule:: kafka.consumer.multiprocess + :members: + :undoc-members: + :show-inheritance: + +kafka.consumer.simple module +---------------------------- + +.. automodule:: kafka.consumer.simple + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka.consumer + :members: + :undoc-members: + :show-inheritance: diff --git a/docs/apidoc/kafka.partitioner.rst b/docs/apidoc/kafka.partitioner.rst new file mode 100644 index 0000000..ea215f1 --- /dev/null +++ b/docs/apidoc/kafka.partitioner.rst @@ -0,0 +1,38 @@ +kafka.partitioner package +========================= + +Submodules +---------- + +kafka.partitioner.base module +----------------------------- + +.. automodule:: kafka.partitioner.base + :members: + :undoc-members: + :show-inheritance: + +kafka.partitioner.hashed module +------------------------------- + +.. automodule:: kafka.partitioner.hashed + :members: + :undoc-members: + :show-inheritance: + +kafka.partitioner.roundrobin module +----------------------------------- + +.. automodule:: kafka.partitioner.roundrobin + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka.partitioner + :members: + :undoc-members: + :show-inheritance: diff --git a/docs/apidoc/kafka.producer.rst b/docs/apidoc/kafka.producer.rst new file mode 100644 index 0000000..bd850bb --- /dev/null +++ b/docs/apidoc/kafka.producer.rst @@ -0,0 +1,38 @@ +kafka.producer package +====================== + +Submodules +---------- + +kafka.producer.base module +-------------------------- + +.. automodule:: kafka.producer.base + :members: + :undoc-members: + :show-inheritance: + +kafka.producer.keyed module +--------------------------- + +.. automodule:: kafka.producer.keyed + :members: + :undoc-members: + :show-inheritance: + +kafka.producer.simple module +---------------------------- + +.. automodule:: kafka.producer.simple + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka.producer + :members: + :undoc-members: + :show-inheritance: diff --git a/docs/apidoc/kafka.rst b/docs/apidoc/kafka.rst new file mode 100644 index 0000000..eb04c35 --- /dev/null +++ b/docs/apidoc/kafka.rst @@ -0,0 +1,79 @@ +kafka package +============= + +Subpackages +----------- + +.. toctree:: + + kafka.consumer + kafka.partitioner + kafka.producer + +Submodules +---------- + +kafka.client module +------------------- + +.. automodule:: kafka.client + :members: + :undoc-members: + :show-inheritance: + +kafka.codec module +------------------ + +.. automodule:: kafka.codec + :members: + :undoc-members: + :show-inheritance: + +kafka.common module +------------------- + +.. automodule:: kafka.common + :members: + :undoc-members: + :show-inheritance: + +kafka.conn module +----------------- + +.. automodule:: kafka.conn + :members: + :undoc-members: + :show-inheritance: + +kafka.context module +-------------------- + +.. automodule:: kafka.context + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol module +--------------------- + +.. automodule:: kafka.protocol + :members: + :undoc-members: + :show-inheritance: + +kafka.util module +----------------- + +.. automodule:: kafka.util + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka + :members: + :undoc-members: + :show-inheritance: diff --git a/docs/apidoc/modules.rst b/docs/apidoc/modules.rst new file mode 100644 index 0000000..db3e580 --- /dev/null +++ b/docs/apidoc/modules.rst @@ -0,0 +1,7 @@ +kafka +===== + +.. toctree:: + :maxdepth: 4 + + kafka From 960b7a94585e15b8fe599bcd15e9a2fae6862cad Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 09:40:06 -0800 Subject: [PATCH 025/250] This links to the main page not the mobile site --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 031382c..2bf1965 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,7 @@ http://kafka.apache.org/ On Freenode IRC at #kafka-python, as well as #apache-kafka For general discussion of kafka-client design and implementation (not python specific), -see https://groups.google.com/forum/m/#!forum/kafka-clients +see https://groups.google.com/forum/#!forum/kafka-clients # License From 3193d64e70c91d3cad64b0cec1b180a7a1c67ed0 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 11:26:56 -0800 Subject: [PATCH 026/250] Move to .rst and use it in setup.py for long description --- README.md => README.rst | 23 +++++++++++++++++------ setup.py | 15 +++++++-------- 2 files changed, 24 insertions(+), 14 deletions(-) rename README.md => README.rst (51%) diff --git a/README.md b/README.rst similarity index 51% rename from README.md rename to README.rst index 2bf1965..58814f0 100644 --- a/README.md +++ b/README.rst @@ -1,10 +1,18 @@ -# Kafka Python client +Kafka Python client +------------------------ +.. image:: https://api.travis-ci.org/mumrah/kafka-python.png?branch=master + :target: https://travis-ci.org/mumrah/kafka-pytho + :alt: Build Status -[![Build Status](https://api.travis-ci.org/mumrah/kafka-python.png?branch=master)](https://travis-ci.org/mumrah/kafka-python) -[![Coverage Status](https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master)](https://coveralls.io/r/mumrah/kafka-python?branch=master) -[![Full documentation available on ReadTheDocs](https://readthedocs.org/projects/kafka-python/badge/?version=latest)](https://readthedocs.org/projects/kafka-python/?badge=latest) +.. image:: https://api.travis-ci.org/mumrah/kafka-python.png?branch=master + :target: https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master + :alt: Coverage Status -[Full documentation available on ReadTheDocs](http://kafka-python.readthedocs.org/en/latest/) +.. image:: https://readthedocs.org/projects/kafka-python/badge/?version=latest + :target: http://kafka-python.readthedocs.org/en/latest/ + :alt: Full documentation available on ReadTheDocs + +`Full documentation available on ReadTheDoc`_ This module provides low-level protocol support for Apache Kafka as well as high-level consumer and producer classes. Request batching is supported by the @@ -24,7 +32,7 @@ Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE` # Status -The current stable version of this package is [**0.9.3**](https://github.com/mumrah/kafka-python/releases/tag/v0.9.3) and is compatible with +The current stable version of this package is **`0.9.3`_** and is compatible with: Kafka broker versions - 0.8.2.0 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] @@ -38,3 +46,6 @@ Python versions - 3.3 (tested on 3.3.5) - 3.4 (tested on 3.4.2) - pypy (tested on pypy 2.4.0 / python 2.7.8) + +.. _Full documentation available on ReadTheDocs: http://kafka-python.readthedocs.org/en/latest/ +.. _0.9.3: https://github.com/mumrah/kafka-python/releases/tag/v0.9.3 diff --git a/setup.py b/setup.py index 4b1e18e..5bdff65 100644 --- a/setup.py +++ b/setup.py @@ -1,5 +1,5 @@ import sys - +import os from setuptools import setup, Command with open('VERSION', 'r') as v: @@ -26,6 +26,10 @@ test_require = ['tox', 'mock'] if sys.version_info < (2, 7): test_require.append('unittest2') +here = os.path.abspath(os.path.dirname(__file__)) + +with open(os.path.join(here, 'README.rst')) as f: + README = f.read() setup( name="kafka-python", @@ -46,15 +50,10 @@ setup( url="https://github.com/mumrah/kafka-python", license="Apache License 2.0", description="Pure Python client for Apache Kafka", - long_description=""" -This module provides low-level protocol support for Apache Kafka as well as -high-level consumer and producer classes. Request batching is supported by the -protocol as well as broker-aware request routing. Gzip and Snappy compression -is also supported for message sets. -""", + long_description=README, keywords="apache kafka", install_requires=['six'], - classifiers = [ + classifiers=[ "Development Status :: 4 - Beta", "Intended Audience :: Developers", "License :: OSI Approved :: Apache Software License", From f6c67aef4f38b6adebe060fd67e150895ba2db4d Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 11:30:21 -0800 Subject: [PATCH 027/250] Cleaned up some formatting --- README.rst | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/README.rst b/README.rst index 58814f0..5f128c8 100644 --- a/README.rst +++ b/README.rst @@ -1,7 +1,7 @@ Kafka Python client ------------------------ .. image:: https://api.travis-ci.org/mumrah/kafka-python.png?branch=master - :target: https://travis-ci.org/mumrah/kafka-pytho + :target: https://travis-ci.org/mumrah/kafka-python :alt: Build Status .. image:: https://api.travis-ci.org/mumrah/kafka-python.png?branch=master @@ -12,7 +12,7 @@ Kafka Python client :target: http://kafka-python.readthedocs.org/en/latest/ :alt: Full documentation available on ReadTheDocs -`Full documentation available on ReadTheDoc`_ +`Full documentation available on ReadTheDocs`_ This module provides low-level protocol support for Apache Kafka as well as high-level consumer and producer classes. Request batching is supported by the @@ -26,13 +26,13 @@ On Freenode IRC at #kafka-python, as well as #apache-kafka For general discussion of kafka-client design and implementation (not python specific), see https://groups.google.com/forum/#!forum/kafka-clients -# License - +License +---------- Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE` -# Status - -The current stable version of this package is **`0.9.3`_** and is compatible with: +Status +---------- +The current stable version of this package is **0.9.3_** and is compatible with: Kafka broker versions - 0.8.2.0 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] From 2462b878539a0ff56600dfd831b76c28d911d840 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 11:32:40 -0800 Subject: [PATCH 028/250] More formatting cleanup --- README.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.rst b/README.rst index 5f128c8..51a866d 100644 --- a/README.rst +++ b/README.rst @@ -4,7 +4,7 @@ Kafka Python client :target: https://travis-ci.org/mumrah/kafka-python :alt: Build Status -.. image:: https://api.travis-ci.org/mumrah/kafka-python.png?branch=master +.. image:: https://coveralls.io/r/mumrah/kafka-python?branch=master :target: https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master :alt: Coverage Status @@ -32,7 +32,7 @@ Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE` Status ---------- -The current stable version of this package is **0.9.3_** and is compatible with: +The current stable version of this package is `0.9.3`_ and is compatible with: Kafka broker versions - 0.8.2.0 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] From 55f5e77421478c635a778ac98b88fb53faba6396 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 11:34:12 -0800 Subject: [PATCH 029/250] Final formatting --- README.rst | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/README.rst b/README.rst index 51a866d..5405f92 100644 --- a/README.rst +++ b/README.rst @@ -4,8 +4,8 @@ Kafka Python client :target: https://travis-ci.org/mumrah/kafka-python :alt: Build Status -.. image:: https://coveralls.io/r/mumrah/kafka-python?branch=master - :target: https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master +.. image:: https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master + :target: https://coveralls.io/r/mumrah/kafka-python?branch=master :alt: Coverage Status .. image:: https://readthedocs.org/projects/kafka-python/badge/?version=latest @@ -35,12 +35,14 @@ Status The current stable version of this package is `0.9.3`_ and is compatible with: Kafka broker versions + - 0.8.2.0 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] - 0.8.1.1 - 0.8.1 - 0.8.0 Python versions + - 2.6 (tested on 2.6.9) - 2.7 (tested on 2.7.9) - 3.3 (tested on 3.3.5) From 96bc453da96287915a25989ac525c8c8f10b5b99 Mon Sep 17 00:00:00 2001 From: John Anderson Date: Tue, 10 Feb 2015 11:37:55 -0800 Subject: [PATCH 030/250] Added py3 to classifiers --- setup.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/setup.py b/setup.py index 5bdff65..f1c1954 100644 --- a/setup.py +++ b/setup.py @@ -61,6 +61,9 @@ setup( "Programming Language :: Python :: 2", "Programming Language :: Python :: 2.6", "Programming Language :: Python :: 2.7", + "Programming Language :: Python :: 3", + "Programming Language :: Python :: 3.3", + "Programming Language :: Python :: 3.4", "Programming Language :: Python :: Implementation :: PyPy", "Topic :: Software Development :: Libraries :: Python Modules", ] From 25ad88cbe68c816cf41ae12d6d6bfc7c2a0926e8 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Fri, 20 Feb 2015 11:21:12 +0300 Subject: [PATCH 031/250] Correct message keys for async batching mode --- kafka/producer/base.py | 4 +- kafka/protocol.py | 6 +-- test/test_producer_integration.py | 16 ++++---- test/test_protocol.py | 66 +++++++++++++++++++++++++++++-- 4 files changed, 76 insertions(+), 16 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 5b41bc9..4d56134 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -62,7 +62,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # Adjust the timeout to match the remaining period count -= 1 timeout = send_at - time.time() - msgset[topic_partition].append(msg) + msgset[topic_partition].append((msg, key)) # Send collected requests upstream reqs = [] @@ -191,7 +191,7 @@ class Producer(object): self.queue.put((TopicAndPartition(topic, partition), m, key)) resp = [] else: - messages = create_message_set(msg, self.codec, key) + messages = create_message_set([(m, key) for m in msg], self.codec, key) req = ProduceRequest(topic, partition, messages) try: resp = self.client.send_produce_request([req], acks=self.req_acks, diff --git a/kafka/protocol.py b/kafka/protocol.py index 2a39de6..b34a95d 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -559,7 +559,7 @@ def create_gzip_message(payloads, key=None): """ message_set = KafkaProtocol._encode_message_set( - [create_message(payload, key) for payload in payloads]) + [create_message(payload, pl_key) for payload, pl_key in payloads]) gzipped = gzip_encode(message_set) codec = ATTRIBUTE_CODEC_MASK & CODEC_GZIP @@ -580,7 +580,7 @@ def create_snappy_message(payloads, key=None): """ message_set = KafkaProtocol._encode_message_set( - [create_message(payload, key) for payload in payloads]) + [create_message(payload, pl_key) for payload, pl_key in payloads]) snapped = snappy_encode(message_set) codec = ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY @@ -595,7 +595,7 @@ def create_message_set(messages, codec=CODEC_NONE, key=None): return a list containing a single codec-encoded message. """ if codec == CODEC_NONE: - return [create_message(m, key) for m in messages] + return [create_message(m, k) for m, k in messages] elif codec == CODEC_GZIP: return [create_gzip_message(messages, key)] elif codec == CODEC_SNAPPY: diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 38df69f..1804af0 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -71,9 +71,9 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): start_offset = self.current_offset(self.topic, 0) message1 = create_gzip_message([ - ("Gzipped 1 %d" % i).encode('utf-8') for i in range(100)]) + (("Gzipped 1 %d" % i).encode('utf-8'), None) for i in range(100)]) message2 = create_gzip_message([ - ("Gzipped 2 %d" % i).encode('utf-8') for i in range(100)]) + (("Gzipped 2 %d" % i).encode('utf-8'), None) for i in range(100)]) self.assert_produce_request( [ message1, message2 ], @@ -87,8 +87,8 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): start_offset = self.current_offset(self.topic, 0) self.assert_produce_request([ - create_snappy_message(["Snappy 1 %d" % i for i in range(100)]), - create_snappy_message(["Snappy 2 %d" % i for i in range(100)]), + create_snappy_message([("Snappy 1 %d" % i, None) for i in range(100)]), + create_snappy_message([("Snappy 2 %d" % i, None) for i in range(100)]), ], start_offset, 200, @@ -102,13 +102,13 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): messages = [ create_message(b"Just a plain message"), create_gzip_message([ - ("Gzipped %d" % i).encode('utf-8') for i in range(100)]), + (("Gzipped %d" % i).encode('utf-8'), None) for i in range(100)]), ] # All snappy integration tests fail with nosnappyjava if False and has_snappy(): msg_count += 100 - messages.append(create_snappy_message(["Snappy %d" % i for i in range(100)])) + messages.append(create_snappy_message([("Snappy %d" % i, None) for i in range(100)])) self.assert_produce_request(messages, start_offset, msg_count) @@ -118,7 +118,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assert_produce_request([ create_gzip_message([ - ("Gzipped batch 1, message %d" % i).encode('utf-8') + (("Gzipped batch 1, message %d" % i).encode('utf-8'), None) for i in range(50000)]) ], start_offset, @@ -127,7 +127,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assert_produce_request([ create_gzip_message([ - ("Gzipped batch 1, message %d" % i).encode('utf-8') + (("Gzipped batch 1, message %d" % i).encode('utf-8'), None) for i in range(50000)]) ], start_offset+50000, diff --git a/test/test_protocol.py b/test/test_protocol.py index d20f591..0938228 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -32,7 +32,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(msg.value, payload) def test_create_gzip(self): - payloads = [b"v1", b"v2"] + payloads = [(b"v1", None), (b"v2", None)] msg = create_gzip_message(payloads) self.assertEqual(msg.magic, 0) self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_GZIP) @@ -59,9 +59,39 @@ class TestProtocol(unittest.TestCase): self.assertEqual(decoded, expect) + def test_create_gzip_keyed(self): + payloads = [(b"v1", b"k1"), (b"v2", b"k2")] + msg = create_gzip_message(payloads) + self.assertEqual(msg.magic, 0) + self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_GZIP) + self.assertEqual(msg.key, None) + # Need to decode to check since gzipped payload is non-deterministic + decoded = gzip_decode(msg.value) + expect = b"".join([ + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 2), # Length of value + b"v2", # Value + ]) + + self.assertEqual(decoded, expect) + @unittest.skipUnless(has_snappy(), "Snappy not available") def test_create_snappy(self): - payloads = [b"v1", b"v2"] + payloads = [(b"v1", None), (b"v2", None)] msg = create_snappy_message(payloads) self.assertEqual(msg.magic, 0) self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY) @@ -87,6 +117,36 @@ class TestProtocol(unittest.TestCase): self.assertEqual(decoded, expect) + @unittest.skipUnless(has_snappy(), "Snappy not available") + def test_create_snappy_keyed(self): + payloads = [(b"v1", b"k1"), (b"v2", b"k2")] + msg = create_snappy_message(payloads) + self.assertEqual(msg.magic, 0) + self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY) + self.assertEqual(msg.key, None) + decoded = snappy_decode(msg.value) + expect = b"".join([ + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 2), # Length of value + b"v2", # Value + ]) + + self.assertEqual(decoded, expect) + def test_encode_message_header(self): expect = b"".join([ struct.pack(">h", 10), # API Key @@ -701,7 +761,7 @@ class TestProtocol(unittest.TestCase): yield def test_create_message_set(self): - messages = [1, 2, 3] + messages = [(1, "k1"), (2, "k2"), (3, "k3")] # Default codec is CODEC_NONE. Expect list of regular messages. expect = [sentinel.message] * len(messages) From 250778453fcece7316615bd62795e9293c62e525 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 24 Feb 2015 21:31:47 +0300 Subject: [PATCH 032/250] Trying to use threading for async batching --- kafka/producer/base.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 5b41bc9..a9288d9 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -3,12 +3,14 @@ from __future__ import absolute_import import logging import time +from Queue import Queue try: from queue import Empty except ImportError: from Queue import Empty from collections import defaultdict -from multiprocessing import Queue, Process + +from threading import Thread import six @@ -140,7 +142,7 @@ class Producer(object): log.warning("Current implementation does not retry Failed messages") log.warning("Use at your own risk! (or help improve with a PR!)") self.queue = Queue() # Messages are sent through this queue - self.proc = Process(target=_send_upstream, + self.proc = Thread(target=_send_upstream, args=(self.queue, self.client.copy(), self.codec, @@ -211,4 +213,4 @@ class Producer(object): self.proc.join(timeout) if self.proc.is_alive(): - self.proc.terminate() + raise SystemError("Can't join Kafka async thread") From bc0d5c1e275805d2b207900923ef5c87c45127d9 Mon Sep 17 00:00:00 2001 From: Alexey Borzenkov Date: Thu, 3 Apr 2014 21:39:19 +0400 Subject: [PATCH 033/250] Make KafkaConnection copies usable across threads --- kafka/conn.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/kafka/conn.py b/kafka/conn.py index 30debec..8142c45 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -62,6 +62,9 @@ class KafkaConnection(local): self.reinit() + def __getnewargs__(self): + return (self.host, self.port, self.timeout) + def __repr__(self): return "" % (self.host, self.port) @@ -167,6 +170,7 @@ class KafkaConnection(local): c.port = copy.copy(self.port) c.timeout = copy.copy(self.timeout) c._sock = None + c._dirty = True return c def close(self): From 1cce28715798cc5dbe84c793e306cd15769afd7c Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 25 Feb 2015 10:45:47 +0300 Subject: [PATCH 034/250] Returned original tests, rm dirty flag, name fixes --- kafka/conn.py | 1 - kafka/producer/base.py | 39 +++++++++++++++---------------------- test/test_conn.py | 44 ++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 60 insertions(+), 24 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 8142c45..ea55481 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -170,7 +170,6 @@ class KafkaConnection(local): c.port = copy.copy(self.port) c.timeout = copy.copy(self.timeout) c._sock = None - c._dirty = True return c def close(self): diff --git a/kafka/producer/base.py b/kafka/producer/base.py index a9288d9..bb7fd43 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -3,11 +3,10 @@ from __future__ import absolute_import import logging import time -from Queue import Queue try: - from queue import Empty + from queue import Empty, Queue except ImportError: - from Queue import Empty + from Queue import Empty, Queue from collections import defaultdict from threading import Thread @@ -33,13 +32,8 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, Listen on the queue for a specified number of messages or till a specified timeout and send them upstream to the brokers in one request - - NOTE: Ideally, this should have been a method inside the Producer - class. However, multiprocessing module has issues in windows. The - functionality breaks unless this function is kept outside of a class """ stop = False - client.reinit() while not stop: timeout = batch_time @@ -142,18 +136,20 @@ class Producer(object): log.warning("Current implementation does not retry Failed messages") log.warning("Use at your own risk! (or help improve with a PR!)") self.queue = Queue() # Messages are sent through this queue - self.proc = Thread(target=_send_upstream, - args=(self.queue, - self.client.copy(), - self.codec, - batch_send_every_t, - batch_send_every_n, - self.req_acks, - self.ack_timeout)) + self.thread = Thread(target=_send_upstream, + args=(self.queue, + self.client.copy(), + self.codec, + batch_send_every_t, + batch_send_every_n, + self.req_acks, + self.ack_timeout)) + + # Thread will die if main thread exits + self.thread.daemon = True + self.thread.start() + - # Process will die if main thread exits - self.proc.daemon = True - self.proc.start() def send_messages(self, topic, partition, *msg): """ @@ -210,7 +206,4 @@ class Producer(object): """ if self.async: self.queue.put((STOP_ASYNC_PRODUCER, None, None)) - self.proc.join(timeout) - - if self.proc.is_alive(): - raise SystemError("Can't join Kafka async thread") + self.thread.join(timeout) diff --git a/test/test_conn.py b/test/test_conn.py index 2c8f3b2..c4f219b 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -1,5 +1,6 @@ import socket import struct +from threading import Thread import mock from . import unittest @@ -162,3 +163,46 @@ class ConnTest(unittest.TestCase): self.conn.send(self.config['request_id'], self.config['payload']) self.assertEqual(self.MockCreateConn.call_count, 1) self.conn._sock.sendall.assert_called_with(self.config['payload']) + + +class TestKafkaConnection(unittest.TestCase): + + @mock.patch('socket.create_connection') + def test_copy(self, socket): + """KafkaConnection copies work as expected""" + + conn = KafkaConnection('kafka', 9092) + self.assertEqual(socket.call_count, 1) + + copy = conn.copy() + self.assertEqual(socket.call_count, 1) + self.assertEqual(copy.host, 'kafka') + self.assertEqual(copy.port, 9092) + self.assertEqual(copy._sock, None) + + copy.reinit() + self.assertEqual(socket.call_count, 2) + self.assertNotEqual(copy._sock, None) + + @mock.patch('socket.create_connection') + def test_copy_thread(self, socket): + """KafkaConnection copies work in other threads""" + + err = [] + copy = KafkaConnection('kafka', 9092).copy() + + def thread_func(err, copy): + try: + self.assertEqual(copy.host, 'kafka') + self.assertEqual(copy.port, 9092) + self.assertNotEqual(copy._sock, None) + except Exception as e: + err.append(e) + else: + err.append(None) + thread = Thread(target=thread_func, args=(err, copy)) + thread.start() + thread.join() + + self.assertEqual(err, [None]) + self.assertEqual(socket.call_count, 2) From aedbbb39be4d207ba9eaf99811980276d44f39a5 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Feb 2015 13:02:48 +0300 Subject: [PATCH 035/250] Using threading.Event to stop async producer thread --- kafka/producer/base.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index bb7fd43..cf5abac 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -9,7 +9,7 @@ except ImportError: from Queue import Empty, Queue from collections import defaultdict -from threading import Thread +from threading import Thread, Event import six @@ -27,7 +27,7 @@ STOP_ASYNC_PRODUCER = -1 def _send_upstream(queue, client, codec, batch_time, batch_size, - req_acks, ack_timeout): + req_acks, ack_timeout, stop_event): """ Listen on the queue for a specified number of messages or till a specified timeout and send them upstream to the brokers in one @@ -35,7 +35,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, """ stop = False - while not stop: + while not stop_event.is_set(): timeout = batch_time count = batch_size send_at = time.time() + timeout @@ -52,7 +52,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # Check if the controller has requested us to stop if topic_partition == STOP_ASYNC_PRODUCER: - stop = True + stop_event.set() break # Adjust the timeout to match the remaining period @@ -136,6 +136,7 @@ class Producer(object): log.warning("Current implementation does not retry Failed messages") log.warning("Use at your own risk! (or help improve with a PR!)") self.queue = Queue() # Messages are sent through this queue + self.thread_stop_event = Event() self.thread = Thread(target=_send_upstream, args=(self.queue, self.client.copy(), @@ -143,7 +144,8 @@ class Producer(object): batch_send_every_t, batch_send_every_n, self.req_acks, - self.ack_timeout)) + self.ack_timeout, + self.thread_stop_event)) # Thread will die if main thread exits self.thread.daemon = True @@ -207,3 +209,6 @@ class Producer(object): if self.async: self.queue.put((STOP_ASYNC_PRODUCER, None, None)) self.thread.join(timeout) + + if self.thread.is_alive(): + self.thread_stop_event.set() From 5137163fa44b4a6a8a315c30f959e816f657e921 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Feb 2015 13:33:40 +0300 Subject: [PATCH 036/250] Async producer stop() fix --- kafka/producer/base.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index cf5abac..357bccd 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -210,5 +210,5 @@ class Producer(object): self.queue.put((STOP_ASYNC_PRODUCER, None, None)) self.thread.join(timeout) - if self.thread.is_alive(): - self.thread_stop_event.set() + if self.thread.is_alive(): + self.thread_stop_event.set() From 9f7d61258bc878cb70fa2f46d542e9aeb96ea462 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 1 Mar 2015 11:04:18 -0800 Subject: [PATCH 037/250] Accept KafkaClient client_id as string or bytes; encode string client_id as utf-8 bytes internally --- kafka/client.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index f8fe555..3c2a727 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -15,6 +15,7 @@ from kafka.common import (TopicAndPartition, BrokerMetadata, from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS from kafka.protocol import KafkaProtocol +from kafka.util import kafka_bytestring log = logging.getLogger("kafka") @@ -30,7 +31,7 @@ class KafkaClient(object): def __init__(self, hosts, client_id=CLIENT_ID, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): # We need one connection to bootstrap - self.client_id = client_id + self.client_id = kafka_bytestring(client_id) self.timeout = timeout self.hosts = collect_hosts(hosts) From 0f04eba0977c77fd9edd0dc336d4934cd9de1348 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 1 Mar 2015 11:05:36 -0800 Subject: [PATCH 038/250] Remove KafkaConsumer.BYTES_CONFIGURATION_KEYS - rely on KafkaClient to handle client_id string/bytes encoding - accept either string or bytes for group_id - convert group_id string to utf-8 bytes internally --- kafka/consumer/kafka.py | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index cd3cc4a..0093381 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -47,8 +47,6 @@ DEFAULT_CONSUMER_CONFIG = { 'rebalance_backoff_ms': 2000, } -BYTES_CONFIGURATION_KEYS = ('client_id', 'group_id') - class KafkaConsumer(object): """ @@ -171,13 +169,6 @@ class KafkaConsumer(object): raise KafkaConfigurationError('Unknown configuration key(s): ' + str(list(configs.keys()))) - # Handle str/bytes conversions - for config_key in BYTES_CONFIGURATION_KEYS: - 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') - 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)') @@ -554,7 +545,7 @@ class KafkaConsumer(object): 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(kafka_bytestring(self._config['group_id']), commits, fail_on_error=False) @@ -618,7 +609,7 @@ class KafkaConsumer(object): logger.info("Consumer fetching stored offsets") for topic_partition in self._topics: (resp,) = self._client.send_offset_fetch_request( - self._config['group_id'], + kafka_bytestring(self._config['group_id']), [OffsetFetchRequest(topic_partition[0], topic_partition[1])], fail_on_error=False) try: From 082cda6a7235df15d38102556f7bfe09e43b39fa Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 2 Mar 2015 09:34:06 -0800 Subject: [PATCH 039/250] Avoid topic_partitions KeyError in KafkaClient --- kafka/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index 3c2a727..53b6ad0 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -86,7 +86,7 @@ class KafkaClient(object): self.load_metadata_for_topics(topic) # If the partition doesn't actually exist, raise - if partition not in self.topic_partitions[topic]: + if partition not in self.topic_partitions.get(topic, []): raise UnknownTopicOrPartitionError(key) # If there's no leader for the partition, raise From 83d28ba79a5453abf0530e403c125e51b9d3641c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 2 Mar 2015 09:35:04 -0800 Subject: [PATCH 040/250] Add comment about no server response case in _send_broker_aware_request --- kafka/client.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/kafka/client.py b/kafka/client.py index 53b6ad0..6c6e74f 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -178,8 +178,13 @@ class KafkaClient(object): # Send the request, recv the response try: conn.send(requestId, request) + + # decoder_fn=None signal that the server is expected to not + # send a response. This probably only applies to + # ProduceRequest w/ acks = 0 if decoder_fn is None: continue + try: response = conn.recv(requestId) except ConnectionError as e: From c80fbd1451f0042a28eba537c7b804c92bfbd612 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 2 Mar 2015 09:37:55 -0800 Subject: [PATCH 041/250] KafkaClient.get_partition_ids_for_topic now returns empty list for unknown topics --- kafka/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index 6c6e74f..48a534e 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -265,7 +265,7 @@ class KafkaClient(object): def get_partition_ids_for_topic(self, topic): if topic not in self.topic_partitions: - return None + return [] return sorted(list(self.topic_partitions[topic])) From 6de9444b960b931d9def09e6d95ac6866bb34466 Mon Sep 17 00:00:00 2001 From: wkiser Date: Mon, 23 Feb 2015 12:58:20 -0500 Subject: [PATCH 042/250] Fixes consumer/kafka and consumer/simple to only yield messages if the message's offset is greater than or equal to the consumer offset. --- kafka/consumer/kafka.py | 4 ++++ kafka/consumer/simple.py | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index cd3cc4a..53ba0a7 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -430,6 +430,10 @@ class KafkaConsumer(object): offset, message.key, self._config['deserializer_class'](message.value)) + if offset < self._offsets.fetch[topic_partition]: + logger.debug('Skipping message %s because its offset is less than the consumer offset', + msg) + continue # Only increment fetch offset if we safely got the message and deserialized self._offsets.fetch[topic_partition] = offset + 1 diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 000fcd9..4f76bd6 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -305,6 +305,10 @@ class SimpleConsumer(Consumer): buffer_size = partitions[partition] try: for message in resp.messages: + if message.offset < self.fetch_offsets[partition]: + log.debug('Skipping message %s because its offset is less than the consumer offset', + message) + continue # Put the message in our queue self.queue.put((partition, message)) self.fetch_offsets[partition] = message.offset + 1 From 04979248afa82af77d02f76fbf12456bb83ae334 Mon Sep 17 00:00:00 2001 From: Christophe-Marie Duquesne Date: Tue, 17 Feb 2015 00:50:51 +0100 Subject: [PATCH 043/250] Stopping daemonized threads when references to these threads are lost --- kafka/producer/base.py | 6 ++++++ kafka/util.py | 3 +++ 2 files changed, 9 insertions(+) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 5b41bc9..695f195 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -127,6 +127,7 @@ class Producer(object): self.async = async self.req_acks = req_acks self.ack_timeout = ack_timeout + self.stopped = False if codec is None: codec = CODEC_NONE @@ -212,3 +213,8 @@ class Producer(object): if self.proc.is_alive(): self.proc.terminate() + self.stopped = True + + def __del__(self): + if not self.stopped: + self.stop() diff --git a/kafka/util.py b/kafka/util.py index 14d2b2c..78c3607 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -151,3 +151,6 @@ class ReentrantTimer(object): # noinspection PyAttributeOutsideInit self.timer = None self.fn = None + + def __del__(self): + self.stop() From ac66fe9159723c6707258bbb4c6727cf66a78219 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 3 Mar 2015 15:32:32 +0300 Subject: [PATCH 044/250] Fixing distribution for MP Consumer --- kafka/consumer/multiprocess.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 4dc04dc..40aecf8 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -131,7 +131,7 @@ class MultiProcessConsumer(Consumer): # * we have an even distribution of partitions among processes if not partitions_per_proc: partitions_per_proc = round(len(partitions) * 1.0 / num_procs) - if partitions_per_proc < num_procs * 0.5: + if partitions_per_proc * num_procs < len(partitions): partitions_per_proc += 1 # The final set of chunks From 95737d9b89bd9b8afa85c12e9aa174fda3ee0982 Mon Sep 17 00:00:00 2001 From: Job Evers Date: Thu, 22 Jan 2015 17:25:02 -0600 Subject: [PATCH 045/250] Add failed_payloads attribute to FailedPayloadsError --- kafka/common.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/common.py b/kafka/common.py index b7bb06c..f40c9fa 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -161,7 +161,9 @@ class KafkaTimeoutError(KafkaError): class FailedPayloadsError(KafkaError): - pass + def __init__(self, failed_payloads, *args): + super(FailedPayloadsError, self).__init__(*args) + self.failed_payloads = failed_payloads class ConnectionError(KafkaError): From 610f01e96d3bfd9f632371de5bd6cf911a8e71ef Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 20:01:57 -0700 Subject: [PATCH 046/250] Use context managers in gzip_encode / gzip_decode --- kafka/codec.py | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/kafka/codec.py b/kafka/codec.py index 2279200..7883158 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -24,22 +24,17 @@ def has_snappy(): def gzip_encode(payload): - buffer = BytesIO() - handle = gzip.GzipFile(fileobj=buffer, mode="w") - handle.write(payload) - handle.close() - buffer.seek(0) - result = buffer.read() - buffer.close() + with BytesIO() as buf: + with gzip.GzipFile(fileobj=buf, mode="w") as gzipper: + gzipper.write(payload) + result = buf.getvalue() return result def gzip_decode(payload): - buffer = BytesIO(payload) - handle = gzip.GzipFile(fileobj=buffer, mode='r') - result = handle.read() - handle.close() - buffer.close() + with BytesIO(payload) as buf: + with gzip.GzipFile(fileobj=buf, mode='r') as gzipper: + result = gzipper.read() return result From 83cee06a33ec822d747b3259f859179c63c5f39a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 20:13:13 -0700 Subject: [PATCH 047/250] Fixup indentation in kafka/common.py; add links to protocol definitions --- kafka/common.py | 51 ++++++++++++++++++++++++++----------------------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index f40c9fa..8207bec 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -6,43 +6,46 @@ from collections import namedtuple # Structs # ############### -# Request payloads -ProduceRequest = namedtuple("ProduceRequest", - ["topic", "partition", "messages"]) - -FetchRequest = namedtuple("FetchRequest", - ["topic", "partition", "offset", "max_bytes"]) - -OffsetRequest = namedtuple("OffsetRequest", - ["topic", "partition", "time", "max_offsets"]) - -OffsetCommitRequest = namedtuple("OffsetCommitRequest", - ["topic", "partition", "offset", "metadata"]) - +# https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI MetadataRequest = namedtuple("MetadataRequest", ["topics"]) -OffsetFetchRequest = namedtuple("OffsetFetchRequest", ["topic", "partition"]) - MetadataResponse = namedtuple("MetadataResponse", ["brokers", "topics"]) -# Response payloads -ProduceResponse = namedtuple("ProduceResponse", - ["topic", "partition", "error", "offset"]) +# https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ProduceAPI +ProduceRequest = namedtuple("ProduceRequest", + ["topic", "partition", "messages"]) -FetchResponse = namedtuple("FetchResponse", ["topic", "partition", "error", - "highwaterMark", "messages"]) +ProduceResponse = namedtuple("ProduceResponse", + ["topic", "partition", "error", "offset"]) + +# https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-FetchAPI +FetchRequest = namedtuple("FetchRequest", + ["topic", "partition", "offset", "max_bytes"]) + +FetchResponse = namedtuple("FetchResponse", + ["topic", "partition", "error", "highwaterMark", "messages"]) + +# https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI +OffsetRequest = namedtuple("OffsetRequest", + ["topic", "partition", "time", "max_offsets"]) OffsetResponse = namedtuple("OffsetResponse", - ["topic", "partition", "error", "offsets"]) + ["topic", "partition", "error", "offsets"]) + +# https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI +OffsetCommitRequest = namedtuple("OffsetCommitRequest", + ["topic", "partition", "offset", "metadata"]) OffsetCommitResponse = namedtuple("OffsetCommitResponse", - ["topic", "partition", "error"]) + ["topic", "partition", "error"]) + +OffsetFetchRequest = namedtuple("OffsetFetchRequest", + ["topic", "partition"]) OffsetFetchResponse = namedtuple("OffsetFetchResponse", - ["topic", "partition", "offset", - "metadata", "error"]) + ["topic", "partition", "offset", "metadata", "error"]) From 4d59678dd38393fcdc2ef627ca717d5e87d0e744 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 23:41:10 -0700 Subject: [PATCH 048/250] Gzip context manager not supported in py2.6, so use try/finally instead --- kafka/codec.py | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/kafka/codec.py b/kafka/codec.py index 7883158..56689ce 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -25,16 +25,31 @@ def has_snappy(): def gzip_encode(payload): with BytesIO() as buf: - with gzip.GzipFile(fileobj=buf, mode="w") as gzipper: + + # Gzip context manager introduced in python 2.6 + # so old-fashioned way until we decide to not support 2.6 + gzipper = gzip.GzipFile(fileobj=buf, mode="w") + try: gzipper.write(payload) + finally: + gzipper.close() + result = buf.getvalue() + return result def gzip_decode(payload): with BytesIO(payload) as buf: - with gzip.GzipFile(fileobj=buf, mode='r') as gzipper: + + # Gzip context manager introduced in python 2.6 + # so old-fashioned way until we decide to not support 2.6 + gzipper = gzip.GzipFile(fileobj=buf, mode='r') + try: result = gzipper.read() + finally: + gzipper.close() + return result From fa4338821976a66134b08eabab79bedefc2a0c67 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 23:41:46 -0700 Subject: [PATCH 049/250] Take the linter to kafka/codec.py --- kafka/codec.py | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/kafka/codec.py b/kafka/codec.py index 56689ce..19f405b 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -1,8 +1,7 @@ -from io import BytesIO import gzip +from io import BytesIO import struct -import six from six.moves import xrange _XERIAL_V1_HEADER = (-126, b'S', b'N', b'A', b'P', b'P', b'Y', 0, 1, 1) @@ -10,9 +9,9 @@ _XERIAL_V1_FORMAT = 'bccccccBii' try: import snappy - _has_snappy = True + _HAS_SNAPPY = True except ImportError: - _has_snappy = False + _HAS_SNAPPY = False def has_gzip(): @@ -20,7 +19,7 @@ def has_gzip(): def has_snappy(): - return _has_snappy + return _HAS_SNAPPY def gzip_encode(payload): @@ -57,8 +56,8 @@ def snappy_encode(payload, xerial_compatible=False, xerial_blocksize=32 * 1024): """Encodes the given data with snappy if xerial_compatible is set then the stream is encoded in a fashion compatible with the xerial snappy library - The block size (xerial_blocksize) controls how frequent the blocking occurs - 32k is the default in the xerial library. + The block size (xerial_blocksize) controls how frequent the blocking + occurs 32k is the default in the xerial library. The format winds up being +-------------+------------+--------------+------------+--------------+ @@ -73,7 +72,7 @@ def snappy_encode(payload, xerial_compatible=False, xerial_blocksize=32 * 1024): length will always be <= blocksize. """ - if not _has_snappy: + if not has_snappy(): raise NotImplementedError("Snappy codec is not available") if xerial_compatible: @@ -84,7 +83,7 @@ def snappy_encode(payload, xerial_compatible=False, xerial_blocksize=32 * 1024): out = BytesIO() header = b''.join([struct.pack('!' + fmt, dat) for fmt, dat - in zip(_XERIAL_V1_FORMAT, _XERIAL_V1_HEADER)]) + in zip(_XERIAL_V1_FORMAT, _XERIAL_V1_HEADER)]) out.write(header) for chunk in _chunker(): @@ -123,13 +122,13 @@ def _detect_xerial_stream(payload): """ if len(payload) > 16: - header = header = struct.unpack('!' + _XERIAL_V1_FORMAT, bytes(payload)[:16]) + header = struct.unpack('!' + _XERIAL_V1_FORMAT, bytes(payload)[:16]) return header == _XERIAL_V1_HEADER return False def snappy_decode(payload): - if not _has_snappy: + if not has_snappy(): raise NotImplementedError("Snappy codec is not available") if _detect_xerial_stream(payload): From 4bab2fa5d1bc67e18b2f7791ff5fbb8e73143a5e Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 11 Mar 2015 13:51:07 +0300 Subject: [PATCH 050/250] Cleaned code for MP consumer chunking --- kafka/consumer/multiprocess.py | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 40aecf8..db59f7b 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -123,26 +123,25 @@ class MultiProcessConsumer(Consumer): self.pause = Event() # Requests the consumers to pause fetch self.size = Value('i', 0) # Indicator of number of messages to fetch - partitions = self.offsets.keys() + partitions = list(self.offsets.keys()) - # If unspecified, start one consumer per partition + # By default, start one consumer process for all partitions # The logic below ensures that # * we do not cross the num_procs limit # * we have an even distribution of partitions among processes - if not partitions_per_proc: - partitions_per_proc = round(len(partitions) * 1.0 / num_procs) - if partitions_per_proc * num_procs < len(partitions): - partitions_per_proc += 1 + + if partitions_per_proc: + num_procs = len(partitions) / partitions_per_proc + if num_procs * partitions_per_proc < len(partitions): + num_procs += 1 # The final set of chunks - chunker = lambda *x: [] + list(x) - chunks = map(chunker, *[iter(partitions)] * int(partitions_per_proc)) + chunks = [partitions[proc::num_procs] for proc in range(num_procs)] self.procs = [] for chunk in chunks: - chunk = filter(lambda x: x is not None, chunk) args = (client.copy(), - group, topic, list(chunk), + group, topic, chunk, self.queue, self.start, self.exit, self.pause, self.size) From 01ea3bf968c76a5f7a1999cfca36766d9bbff5e7 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 12 Mar 2015 11:33:07 +0300 Subject: [PATCH 051/250] Used thread-safe dict.copy().keys() for MP consumer partitions --- kafka/consumer/multiprocess.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index db59f7b..bec3100 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -123,7 +123,10 @@ class MultiProcessConsumer(Consumer): self.pause = Event() # Requests the consumers to pause fetch self.size = Value('i', 0) # Indicator of number of messages to fetch - partitions = list(self.offsets.keys()) + # dict.keys() returns a view in py3 + it's not a thread-safe operation + # http://blog.labix.org/2008/06/27/watch-out-for-listdictkeys-in-python-3 + # It's safer to copy dict as it only runs during the init. + partitions = list(self.offsets.copy().keys()) # By default, start one consumer process for all partitions # The logic below ensures that From 09028f0b22389a6319a2c1283c902b5f83b81917 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 3 Mar 2015 13:37:37 +0300 Subject: [PATCH 052/250] Using additional params for MP consumer child process - Moved the events params to a separate param for consistency - Passing additional params to internal SimpleConsumer worker for multiprocessing high-level consumer. It allows to use non-default consumer settings (fetch_size_bytes, buffer_size, max_buffer_size). --- kafka/consumer/multiprocess.py | 58 +++++++++++++++++++--------------- 1 file changed, 32 insertions(+), 26 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index bec3100..2bb97f3 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -2,6 +2,8 @@ from __future__ import absolute_import import logging import time + +from collections import namedtuple from multiprocessing import Process, Queue as MPQueue, Event, Value try: @@ -15,10 +17,11 @@ from .base import ( ) from .simple import Consumer, SimpleConsumer +Events = namedtuple("Events", ["start", "pause", "exit"]) + log = logging.getLogger("kafka") - -def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): +def _mp_consume(client, group, topic, queue, size, events, consumer_options): """ A child process worker which consumes messages based on the notifications given by the controller process @@ -34,20 +37,20 @@ def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): # We will start consumers without auto-commit. Auto-commit will be # done by the master controller process. consumer = SimpleConsumer(client, group, topic, - partitions=chunk, auto_commit=False, auto_commit_every_n=None, - auto_commit_every_t=None) + auto_commit_every_t=None, + **consumer_options) # Ensure that the consumer provides the partition information consumer.provide_partition_info() while True: # Wait till the controller indicates us to start consumption - start.wait() + events.start.wait() # If we are asked to quit, do so - if exit.is_set(): + if events.exit.is_set(): break # Consume messages and add them to the queue. If the controller @@ -65,7 +68,7 @@ def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): # loop consuming all available messages before the controller # can reset the 'start' event if count == size.value: - pause.wait() + events.pause.wait() else: # In case we did not receive any message, give up the CPU for @@ -105,7 +108,8 @@ class MultiProcessConsumer(Consumer): def __init__(self, client, group, topic, auto_commit=True, auto_commit_every_n=AUTO_COMMIT_MSG_COUNT, auto_commit_every_t=AUTO_COMMIT_INTERVAL, - num_procs=1, partitions_per_proc=0): + num_procs=1, partitions_per_proc=0, + simple_consumer_options=None): # Initiate the base consumer class super(MultiProcessConsumer, self).__init__( @@ -118,9 +122,10 @@ class MultiProcessConsumer(Consumer): # Variables for managing and controlling the data flow from # consumer child process to master self.queue = MPQueue(1024) # Child consumers dump messages into this - self.start = Event() # Indicates the consumers to start fetch - self.exit = Event() # Requests the consumers to shutdown - self.pause = Event() # Requests the consumers to pause fetch + self.events = Events( + start = Event(), # Indicates the consumers to start fetch + exit = Event(), # Requests the consumers to shutdown + pause = Event()) # Requests the consumers to pause fetch self.size = Value('i', 0) # Indicator of number of messages to fetch # dict.keys() returns a view in py3 + it's not a thread-safe operation @@ -143,11 +148,12 @@ class MultiProcessConsumer(Consumer): self.procs = [] for chunk in chunks: - args = (client.copy(), - group, topic, chunk, - self.queue, self.start, self.exit, - self.pause, self.size) + options = {'partitions': list(chunk)} + if simple_consumer_options: + options.update(simple_consumer_options) + args = (client.copy(), group, topic, self.queue, + self.size, self.events, options) proc = Process(target=_mp_consume, args=args) proc.daemon = True proc.start() @@ -159,9 +165,9 @@ class MultiProcessConsumer(Consumer): def stop(self): # Set exit and start off all waiting consumers - self.exit.set() - self.pause.set() - self.start.set() + self.events.exit.set() + self.events.pause.set() + self.events.start.set() for proc in self.procs: proc.join() @@ -176,10 +182,10 @@ class MultiProcessConsumer(Consumer): # Trigger the consumer procs to start off. # We will iterate till there are no more messages available self.size.value = 0 - self.pause.set() + self.events.pause.set() while True: - self.start.set() + self.events.start.set() try: # We will block for a small while so that the consumers get # a chance to run and put some messages in the queue @@ -191,12 +197,12 @@ class MultiProcessConsumer(Consumer): # Count, check and commit messages if necessary self.offsets[partition] = message.offset + 1 - self.start.clear() + self.events.start.clear() self.count_since_commit += 1 self._auto_commit() yield message - self.start.clear() + self.events.start.clear() def get_messages(self, count=1, block=True, timeout=10): """ @@ -216,7 +222,7 @@ class MultiProcessConsumer(Consumer): # necessary, but these will not be committed to kafka. Also, the extra # messages can be provided in subsequent runs self.size.value = count - self.pause.clear() + self.events.pause.clear() if timeout is not None: max_time = time.time() + timeout @@ -228,7 +234,7 @@ class MultiProcessConsumer(Consumer): # go into overdrive and keep consuming thousands of # messages when the user might need only a few if self.queue.empty(): - self.start.set() + self.events.start.set() try: partition, message = self.queue.get(block, timeout) @@ -242,8 +248,8 @@ class MultiProcessConsumer(Consumer): timeout = max_time - time.time() self.size.value = 0 - self.start.clear() - self.pause.set() + self.events.start.clear() + self.events.pause.set() # Update and commit offsets if necessary self.offsets.update(new_offsets) From f8012c1a74a60623d541d30ff5a21e86fd48c81a Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 11 Mar 2015 12:41:34 +0300 Subject: [PATCH 053/250] Moved additional MP consumer options to **kwargs --- kafka/consumer/multiprocess.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 2bb97f3..cdfaeeb 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -21,7 +21,7 @@ Events = namedtuple("Events", ["start", "pause", "exit"]) log = logging.getLogger("kafka") -def _mp_consume(client, group, topic, queue, size, events, consumer_options): +def _mp_consume(client, group, topic, queue, size, events, **consumer_options): """ A child process worker which consumes messages based on the notifications given by the controller process @@ -153,8 +153,8 @@ class MultiProcessConsumer(Consumer): options.update(simple_consumer_options) args = (client.copy(), group, topic, self.queue, - self.size, self.events, options) - proc = Process(target=_mp_consume, args=args) + self.size, self.events) + proc = Process(target=_mp_consume, args=args, kwargs=options) proc.daemon = True proc.start() self.procs.append(proc) From e43f405a0cf2912841bf4734d67384a8074e8616 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Fri, 13 Mar 2015 14:15:21 +0300 Subject: [PATCH 054/250] Using iter_timeout=0 for MP inner consumer in tests --- test/test_consumer_integration.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 9c89190..7dd4a79 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -63,6 +63,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): if consumer_class == SimpleConsumer: kwargs.setdefault('iter_timeout', 0) + elif consumer_class == MultiProcessConsumer: + kwargs.setdefault('simple_consumer_options', {'iter_timeout': 0}) return consumer_class(self.client, group, topic, **kwargs) @@ -243,7 +245,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(0, range(0, 10)) self.send_messages(1, range(10, 20)) - consumer = MultiProcessConsumer(self.client, "group1", self.topic, auto_commit=False) + consumer = MultiProcessConsumer(self.client, "group1", self.topic, auto_commit=False, + simple_consumer_options={'iter_timeout': 0}) self.assertEqual(consumer.pending(), 20) self.assertEqual(consumer.pending(partitions=[0]), 10) From a049e19a1b3220d8eb85e380a564942e02543ef5 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Fri, 20 Mar 2015 15:29:19 -0700 Subject: [PATCH 055/250] Increase count_since_commit on seek When auto_commit is False this change enables an explicit call to commit() to actually commit the offsets. Otherwise a consumer won't be able to commit until at least one message is read. --- kafka/consumer/simple.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 3d250ea..b50de61 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -214,8 +214,8 @@ class SimpleConsumer(Consumer): # Reset queue and fetch offsets since they are invalid self.fetch_offsets = self.offsets.copy() + self.count_since_commit += 1 if self.auto_commit: - self.count_since_commit += 1 self.commit() self.queue = Queue() From 3b1ccc68b324cf05740b7b02f8568889c750e4bf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 22:57:34 -0700 Subject: [PATCH 056/250] Deprecate KafkaConsumer config metadata_broker_list in favor of bootstrap_servers --- kafka/consumer/kafka.py | 34 ++++++++++++++++++++++++++-------- 1 file changed, 26 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 49ffa7b..f03d15e 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -25,7 +25,7 @@ OffsetsStruct = namedtuple("OffsetsStruct", ["fetch", "highwater", "commit", "ta DEFAULT_CONSUMER_CONFIG = { 'client_id': __name__, 'group_id': None, - 'metadata_broker_list': None, + 'bootstrap_servers': [], 'socket_timeout_ms': 30 * 1000, 'fetch_message_max_bytes': 1024 * 1024, 'auto_offset_reset': 'largest', @@ -47,6 +47,9 @@ DEFAULT_CONSUMER_CONFIG = { 'rebalance_backoff_ms': 2000, } +DEPRECATED_CONFIG_KEYS = { + 'metadata_broker_list': 'bootstrap_servers', +} class KafkaConsumer(object): """ @@ -56,7 +59,7 @@ class KafkaConsumer(object): # A very basic 'tail' consumer, with no stored offset management kafka = KafkaConsumer('topic1', - metadata_broker_list=['localhost:9092']) + bootstrap_servers=['localhost:9092']) for m in kafka: print m @@ -75,7 +78,7 @@ class KafkaConsumer(object): # more advanced consumer -- multiple topics w/ auto commit offset # management kafka = KafkaConsumer('topic1', 'topic2', - metadata_broker_list=['localhost:9092'], + bootstrap_servers=['localhost:9092'], group_id='my_consumer_group', auto_commit_enable=True, auto_commit_interval_ms=30 * 1000, @@ -120,7 +123,7 @@ class KafkaConsumer(object): fetch_min_bytes=1, fetch_wait_max_ms=100, refresh_leader_backoff_ms=200, - metadata_broker_list=None, + bootstrap_servers=[], socket_timeout_ms=30*1000, auto_offset_reset='largest', deserializer_class=lambda msg: msg, @@ -149,7 +152,7 @@ class KafkaConsumer(object): fetch_min_bytes=1, fetch_wait_max_ms=100, refresh_leader_backoff_ms=200, - metadata_broker_list=None, + bootstrap_servers=[], socket_timeout_ms=30*1000, auto_offset_reset='largest', deserializer_class=lambda msg: msg, @@ -161,6 +164,7 @@ class KafkaConsumer(object): Configuration parameters are described in more detail at http://kafka.apache.org/documentation.html#highlevelconsumerapi """ + configs = self._deprecate_configs(**configs) self._config = {} for key in DEFAULT_CONSUMER_CONFIG: self._config[key] = configs.pop(key, DEFAULT_CONSUMER_CONFIG[key]) @@ -178,11 +182,11 @@ class KafkaConsumer(object): logger.info("Configuring consumer to auto-commit offsets") self._reset_auto_commit() - if self._config['metadata_broker_list'] is None: - raise KafkaConfigurationError('metadata_broker_list required to ' + if not self._config['bootstrap_servers']: + raise KafkaConfigurationError('bootstrap_servers required to ' 'configure KafkaConsumer') - self._client = KafkaClient(self._config['metadata_broker_list'], + self._client = KafkaClient(self._config['bootstrap_servers'], client_id=self._config['client_id'], timeout=(self._config['socket_timeout_ms'] / 1000.0)) @@ -751,3 +755,17 @@ class KafkaConsumer(object): return '' % ', '.join(["%s-%d" % topic_partition for topic_partition in self._topics]) + + # + # other private methods + # + + def _deprecate_configs(self, **configs): + for old, new in six.iteritems(DEPRECATED_CONFIG_KEYS): + if old in configs: + logger.warning('Deprecated Kafka Consumer configuration: %s. ' + 'Please use %s instead.', old, new) + old_value = configs.pop(old) + if new not in configs: + configs[new] = old_value + return configs From 9641e9fa296a035e73838f07b77310cb5c9eb655 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 24 Mar 2015 12:49:44 +0300 Subject: [PATCH 057/250] Moving to **kwargs for MPConsumer options --- kafka/consumer/multiprocess.py | 3 ++- test/test_consumer_integration.py | 8 +++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index cdfaeeb..5bc04cc 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -109,7 +109,7 @@ class MultiProcessConsumer(Consumer): auto_commit_every_n=AUTO_COMMIT_MSG_COUNT, auto_commit_every_t=AUTO_COMMIT_INTERVAL, num_procs=1, partitions_per_proc=0, - simple_consumer_options=None): + **simple_consumer_options): # Initiate the base consumer class super(MultiProcessConsumer, self).__init__( @@ -150,6 +150,7 @@ class MultiProcessConsumer(Consumer): for chunk in chunks: options = {'partitions': list(chunk)} if simple_consumer_options: + simple_consumer_options.pop('partitions', None) options.update(simple_consumer_options) args = (client.copy(), group, topic, self.queue, diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 7dd4a79..d3df56a 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -61,10 +61,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): group = kwargs.pop('group', self.id().encode('utf-8')) topic = kwargs.pop('topic', self.topic) - if consumer_class == SimpleConsumer: + if consumer_class in [SimpleConsumer, MultiProcessConsumer]: kwargs.setdefault('iter_timeout', 0) - elif consumer_class == MultiProcessConsumer: - kwargs.setdefault('simple_consumer_options', {'iter_timeout': 0}) return consumer_class(self.client, group, topic, **kwargs) @@ -245,8 +243,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(0, range(0, 10)) self.send_messages(1, range(10, 20)) - consumer = MultiProcessConsumer(self.client, "group1", self.topic, auto_commit=False, - simple_consumer_options={'iter_timeout': 0}) + consumer = MultiProcessConsumer(self.client, "group1", self.topic, + auto_commit=False, iter_timeout=0) self.assertEqual(consumer.pending(), 20) self.assertEqual(consumer.pending(partitions=[0]), 10) From 88465f70ef75c13bd6317496f1f8a40d0455b091 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 24 Mar 2015 14:08:25 +0300 Subject: [PATCH 058/250] Using mp.manager to solve the issue with join for MPConsumer --- kafka/consumer/multiprocess.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 5bc04cc..a63b090 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -4,7 +4,7 @@ import logging import time from collections import namedtuple -from multiprocessing import Process, Queue as MPQueue, Event, Value +from multiprocessing import Process, Manager as MPManager try: from Queue import Empty @@ -121,12 +121,13 @@ class MultiProcessConsumer(Consumer): # Variables for managing and controlling the data flow from # consumer child process to master - self.queue = MPQueue(1024) # Child consumers dump messages into this + manager = MPManager() + self.queue = manager.Queue(1024) # Child consumers dump messages into this self.events = Events( - start = Event(), # Indicates the consumers to start fetch - exit = Event(), # Requests the consumers to shutdown - pause = Event()) # Requests the consumers to pause fetch - self.size = Value('i', 0) # Indicator of number of messages to fetch + start = manager.Event(), # Indicates the consumers to start fetch + exit = manager.Event(), # Requests the consumers to shutdown + pause = manager.Event()) # Requests the consumers to pause fetch + self.size = manager.Value('i', 0) # Indicator of number of messages to fetch # dict.keys() returns a view in py3 + it's not a thread-safe operation # http://blog.labix.org/2008/06/27/watch-out-for-listdictkeys-in-python-3 From 9d5c93e902f093872c7475bb010f8e4a5b97aa40 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 24 Mar 2015 14:45:33 +0300 Subject: [PATCH 059/250] Added pylint.rc: ignoring SyncManager for pylint --- pylint.rc | 2 ++ tox.ini | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 pylint.rc diff --git a/pylint.rc b/pylint.rc new file mode 100644 index 0000000..1e76d8c --- /dev/null +++ b/pylint.rc @@ -0,0 +1,2 @@ +[TYPECHECK] +ignored-classes=SyncManager diff --git a/tox.ini b/tox.ini index 71565fd..fba7d8e 100644 --- a/tox.ini +++ b/tox.ini @@ -37,7 +37,7 @@ deps = unittest2 mock pylint -commands = pylint {posargs: -E kafka test} +commands = pylint --rcfile=pylint.rc {posargs: -E kafka test} [testenv:docs] deps = From fb118fb75c818a32d0bb81fe725faca0a714b580 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 25 Mar 2015 14:35:23 +0300 Subject: [PATCH 060/250] Manageable queue.put() operation for MPConsumer processes --- kafka/consumer/base.py | 1 + kafka/consumer/multiprocess.py | 15 +++++++++++---- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 9cdcf89..efc9404 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -25,6 +25,7 @@ MAX_FETCH_BUFFER_SIZE_BYTES = FETCH_BUFFER_SIZE_BYTES * 8 ITER_TIMEOUT_SECONDS = 60 NO_MESSAGES_WAIT_TIME_SECONDS = 0.1 +FULL_QUEUE_WAIT_TIME_SECONDS = 0.1 class Consumer(object): diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index a63b090..5ce8b4d 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -7,13 +7,14 @@ from collections import namedtuple from multiprocessing import Process, Manager as MPManager try: - from Queue import Empty + from Queue import Empty, Full except ImportError: # python 2 - from queue import Empty + from queue import Empty, Full from .base import ( AUTO_COMMIT_MSG_COUNT, AUTO_COMMIT_INTERVAL, - NO_MESSAGES_WAIT_TIME_SECONDS + NO_MESSAGES_WAIT_TIME_SECONDS, + FULL_QUEUE_WAIT_TIME_SECONDS ) from .simple import Consumer, SimpleConsumer @@ -59,7 +60,13 @@ def _mp_consume(client, group, topic, queue, size, events, **consumer_options): message = consumer.get_message() if message: - queue.put(message) + while True: + try: + queue.put(message, timeout=FULL_QUEUE_WAIT_TIME_SECONDS) + break + except Full: + if events.exit.is_set(): break + count += 1 # We have reached the required size. The controller might have From eb5fd4aaa1b0f099235ad29784e3068e3f29e131 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 23:00:00 -0700 Subject: [PATCH 061/250] Improve KafkaConsumer docstrings --- kafka/consumer/kafka.py | 228 +++++++++++++++++----------------------- 1 file changed, 94 insertions(+), 134 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index f03d15e..5d98f81 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -52,114 +52,59 @@ DEPRECATED_CONFIG_KEYS = { } class KafkaConsumer(object): - """ - A simpler kafka consumer - - .. code:: python - - # A very basic 'tail' consumer, with no stored offset management - kafka = KafkaConsumer('topic1', - bootstrap_servers=['localhost:9092']) - for m in kafka: - print m - - # Alternate interface: next() - 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!" - - - .. code:: python - - # more advanced consumer -- multiple topics w/ auto commit offset - # management - kafka = KafkaConsumer('topic1', 'topic2', - bootstrap_servers=['localhost:9092'], - 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() - 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 - while True: - for m in kafka.fetch_messages(): - process_message(m) - kafka.task_done(m) - - - 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 raw bytes) - - Configuration settings can be passed to constructor, - otherwise defaults will be used: - - .. code:: python - - client_id='kafka.consumer.kafka', - group_id=None, - fetch_message_max_bytes=1024*1024, - fetch_min_bytes=1, - fetch_wait_max_ms=100, - refresh_leader_backoff_ms=200, - bootstrap_servers=[], - socket_timeout_ms=30*1000, - auto_offset_reset='largest', - 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 - """ + """A simpler kafka consumer""" def __init__(self, *topics, **configs): self.configure(**configs) self.set_topic_partitions(*topics) def configure(self, **configs): - """ + """Configure the consumer instance + Configuration settings can be passed to constructor, otherwise defaults will be used: - .. code:: python - - client_id='kafka.consumer.kafka', - group_id=None, - fetch_message_max_bytes=1024*1024, - fetch_min_bytes=1, - fetch_wait_max_ms=100, - refresh_leader_backoff_ms=200, - bootstrap_servers=[], - socket_timeout_ms=30*1000, - auto_offset_reset='largest', - deserializer_class=lambda msg: msg, - auto_commit_enable=False, - auto_commit_interval_ms=60 * 1000, - auto_commit_interval_messages=None, - consumer_timeout_ms=-1 + Keyword Arguments: + bootstrap_servers (list): List of initial broker nodes the consumer + should contact to bootstrap initial cluster metadata. This does + not have to be the full node list. It just needs to have at + least one broker that will respond to a Metadata API Request. + client_id (str): a unique name for this client. Defaults to + 'kafka.consumer.kafka'. + group_id (str): the name of the consumer group to join, + Offsets are fetched / committed to this group name. + fetch_message_max_bytes (int, optional): Maximum bytes for each + topic/partition fetch request. Defaults to 1024*1024. + fetch_min_bytes (int, optional): Minimum amount of data the server + should return for a fetch request, otherwise wait up to + fetch_wait_max_ms for more data to accumulate. Defaults to 1. + fetch_wait_max_ms (int, optional): Maximum time for the server to + block waiting for fetch_min_bytes messages to accumulate. + Defaults to 100. + refresh_leader_backoff_ms (int, optional): Milliseconds to backoff + when refreshing metadata on errors (subject to random jitter). + Defaults to 200. + socket_timeout_ms (int, optional): TCP socket timeout in + milliseconds. Defaults to 30*1000. + auto_offset_reset (str, optional): A policy for resetting offsets on + OffsetOutOfRange errors. 'smallest' will move to the oldest + available message, 'largest' will move to the most recent. Any + ofther value will raise the exception. Defaults to 'smallest'. + deserializer_class (callable, optional): Any callable that takes a + raw message value and returns a deserialized value. Defaults to + lambda msg: msg. + auto_commit_enable (bool, optional): Enabling auto-commit will cause + the KafkaConsumer to periodically commit offsets without an + explicit call to commit(). Defaults to False. + auto_commit_interval_ms (int, optional): If auto_commit_enabled, + the milliseconds between automatic offset commits. Defaults to + 60 * 1000. + auto_commit_interval_messages (int, optional): If + auto_commit_enabled, a number of messages consumed between + automatic offset commits. Defaults to None (disabled). + consumer_timeout_ms (int, optional): number of millisecond to throw + a timeout exception to the consumer if no message is available + for consumption. Defaults to -1 (dont throw exception). Configuration parameters are described in more detail at http://kafka.apache.org/documentation.html#highlevelconsumerapi @@ -316,18 +261,18 @@ class KafkaConsumer(object): self._reset_message_iterator() 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 + """Return the next available message - Note that this is also the method called internally during iteration: + Blocks indefinitely unless consumer_timeout_ms > 0 - .. code:: python + Returns: + a single KafkaMessage from the message iterator - for m in consumer: - pass + Raises: + ConsumerTimeout after consumer_timeout_ms and no message + + Note: + This is also the method called internally during iteration """ self._set_consumer_timeout_start() @@ -343,21 +288,24 @@ class KafkaConsumer(object): 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` + """Sends FetchRequests for all topic/partitions set for consumption - Refreshes metadata on errors, and resets fetch offset on - OffsetOutOfRange, per the configured `auto_offset_reset` policy + Returns: + Generator that yields KafkaMessage structs + after deserializing with the configured `deserializer_class` - Key configuration parameters: + Note: + Refreshes metadata on errors, and resets fetch offset on + OffsetOutOfRange, per the configured `auto_offset_reset` policy + + See Also: + Key KafkaConsumer configuration parameters: + * `fetch_message_max_bytes` + * `fetch_max_wait_ms` + * `fetch_min_bytes` + * `deserializer_class` + * `auto_offset_reset` - * `fetch_message_max_bytes` - * `fetch_max_wait_ms` - * `fetch_min_bytes` - * `deserializer_class` - * `auto_offset_reset` """ max_bytes = self._config['fetch_message_max_bytes'] @@ -436,21 +384,22 @@ class KafkaConsumer(object): yield msg def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): - """ - Request available fetch offsets for a single topic/partition + """Request available fetch offsets for a single topic/partition - Arguments: - topic (str) - partition (int) + Keyword Arguments: + topic (str): topic for offset request + partition (int): partition for offset request 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. - max_num_offsets (int) + max_num_offsets (int): Maximum offsets to include in the OffsetResponse Returns: - offsets (list) + a list of offsets in the OffsetResponse submitted for the provided + topic / partition. See: + https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI """ reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] @@ -466,7 +415,8 @@ class KafkaConsumer(object): return resp.offsets def offsets(self, group=None): - """ + """Get internal consumer offset values + Keyword Arguments: group: Either "fetch", "commit", "task_done", or "highwater". If no group specified, returns all groups. @@ -485,10 +435,17 @@ class KafkaConsumer(object): return dict(deepcopy(getattr(self._offsets, group))) def task_done(self, message): - """ - Mark a fetched message as consumed. + """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() + + Arguments: + message (KafkaMessage): the message to mark as complete + + Returns: + Nothing + """ topic_partition = (message.topic, message.partition) offset = message.offset @@ -516,12 +473,15 @@ class KafkaConsumer(object): self.commit() def commit(self): - """ - Store consumed message offsets (marked via task_done()) + """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 `this wiki page `_. + Returns: + True on success, or False if no offsets were found for commit + + Note: + this functionality requires server version >=0.8.1.1 + 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!') From 484f1a4722145620c0dcd0cb8c72bcb8a3834020 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 23:00:50 -0700 Subject: [PATCH 062/250] Move KafkaConsumer usage examples to docs/usage; Put KeyedProducer usage right after SimpleProducer --- docs/usage.rst | 119 +++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 105 insertions(+), 14 deletions(-) diff --git a/docs/usage.rst b/docs/usage.rst index 141cf93..acd52dc 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -1,12 +1,12 @@ Usage ===== -High level ----------- +SimpleProducer +-------------- .. code:: python - from kafka import SimpleProducer, KafkaClient, KafkaConsumer + from kafka import SimpleProducer, KafkaClient # To send messages synchronously kafka = KafkaClient("localhost:9092") @@ -51,17 +51,6 @@ High level batch_send_every_n=20, batch_send_every_t=60) - # To consume messages - consumer = KafkaConsumer("my-topic", group_id="my_group", - metadata_broker_list=["localhost:9092"]) - for message in consumer: - # message is raw byte string -- decode if necessary! - # e.g., for unicode: `message.decode('utf-8')` - print(message) - - kafka.close() - - Keyed messages -------------- @@ -80,6 +69,108 @@ Keyed messages producer = KeyedProducer(kafka, partitioner=RoundRobinPartitioner) + +KafkaConsumer +------------- + +.. code:: python + + from kafka import KafkaConsumer + + # To consume messages + consumer = KafkaConsumer("my-topic", + group_id="my_group", + metadata_broker_list=["localhost:9092"]) + for message in consumer: + # message is raw byte string -- decode if necessary! + # e.g., for unicode: `message.decode('utf-8')` + print(message) + + kafka.close() + +.. code:: python + + from kafka import KafkaConsumer + + # A very basic 'tail' consumer, with no stored offset management + kafka = KafkaConsumer('topic1', + metadata_broker_list=['localhost:9092']) + for m in kafka: + print m + + # Alternate interface: next() + 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!" + + +.. code:: python + + from kafka import KafkaConsumer + + # more advanced consumer -- multiple topics w/ auto commit offset + # management + kafka = KafkaConsumer('topic1', 'topic2', + metadata_broker_list=['localhost:9092'], + 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() + 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 + while True: + for m in kafka.fetch_messages(): + process_message(m) + kafka.task_done(m) + + + 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 raw bytes) + + Configuration settings can be passed to constructor, + otherwise defaults will be used: + +.. code:: python + + client_id='kafka.consumer.kafka', + 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=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 + Multiprocess consumer --------------------- From 9be4146532e7f7c52b5a47caa3d3c5fe625ed69c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 8 Mar 2015 23:54:54 -0700 Subject: [PATCH 063/250] fixup kafka consumer docstring --- 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 5d98f81..6f5bcdd 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -89,7 +89,7 @@ class KafkaConsumer(object): auto_offset_reset (str, optional): A policy for resetting offsets on OffsetOutOfRange errors. 'smallest' will move to the oldest available message, 'largest' will move to the most recent. Any - ofther value will raise the exception. Defaults to 'smallest'. + ofther value will raise the exception. Defaults to 'largest'. deserializer_class (callable, optional): Any callable that takes a raw message value and returns a deserialized value. Defaults to lambda msg: msg. From c1ca3bf97ed91f25bd3d0489ca4f3a7d61ab95b8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 9 Mar 2015 00:00:49 -0700 Subject: [PATCH 064/250] Updates to KafkaConsumer usage docs --- docs/usage.rst | 68 +++++++++++++++++++------------------------------- 1 file changed, 26 insertions(+), 42 deletions(-) diff --git a/docs/usage.rst b/docs/usage.rst index acd52dc..150d121 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -80,32 +80,24 @@ KafkaConsumer # To consume messages consumer = KafkaConsumer("my-topic", group_id="my_group", - metadata_broker_list=["localhost:9092"]) + bootstrap_servers=["localhost:9092"]) for message in consumer: - # message is raw byte string -- decode if necessary! - # e.g., for unicode: `message.decode('utf-8')` - print(message) + # message value is raw byte string -- decode if necessary! + # e.g., for unicode: `message.value.decode('utf-8')` + print("%s:%d:%d: key=%s value=%s" % (message.topic, message.partition, + message.offset, message.key, + message.value)) kafka.close() -.. code:: python - from kafka import KafkaConsumer +messages (m) are namedtuples with attributes: - # A very basic 'tail' consumer, with no stored offset management - kafka = KafkaConsumer('topic1', - metadata_broker_list=['localhost:9092']) - for m in kafka: - print m - - # Alternate interface: next() - 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!" + * `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 raw bytes) .. code:: python @@ -114,22 +106,22 @@ KafkaConsumer # more advanced consumer -- multiple topics w/ auto commit offset # management - kafka = KafkaConsumer('topic1', 'topic2', - metadata_broker_list=['localhost:9092'], - group_id='my_consumer_group', - auto_commit_enable=True, - auto_commit_interval_ms=30 * 1000, - auto_offset_reset='smallest') + consumer = KafkaConsumer('topic1', 'topic2', + bootstrap_servers=['localhost:9092'], + 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) + for m in consumer: + do_some_work(m) - # Alternate interface: next() - m = kafka.next() - process_message(m) - kafka.task_done(m) + # Mark this message as fully consumed + # so it can be included in the next commit + # + # **messages that are not marked w/ task_done currently do not commit! + kafka.task_done(m) # If auto_commit_enable is False, remember to commit() periodically kafka.commit() @@ -141,14 +133,6 @@ KafkaConsumer kafka.task_done(m) - 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 raw bytes) - Configuration settings can be passed to constructor, otherwise defaults will be used: @@ -160,7 +144,7 @@ KafkaConsumer fetch_min_bytes=1, fetch_wait_max_ms=100, refresh_leader_backoff_ms=200, - metadata_broker_list=None, + bootstrap_servers=[], socket_timeout_ms=30*1000, auto_offset_reset='largest', deserializer_class=lambda msg: msg, From 5652f595632bff728c7880de8892d8cb0a99cb45 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 9 Mar 2015 00:01:00 -0700 Subject: [PATCH 065/250] Add sphinx_rtd_theme to docs requirements.txt --- docs/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/requirements.txt b/docs/requirements.txt index 86b4f05..d32365f 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -1,5 +1,6 @@ sphinx sphinxcontrib-napoleon +sphinx_rtd_theme # Install kafka-python in editable mode # This allows the sphinx autodoc module From 35b8f5b5d8b0888806d5d6c9ec02910327c3a671 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Mar 2015 16:38:47 -0700 Subject: [PATCH 066/250] Use bootstrap_servers interface in tests --- test/test_consumer_integration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index d3df56a..17a8ac9 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -69,7 +69,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def kafka_consumer(self, **configs): brokers = '%s:%d' % (self.server.host, self.server.port) consumer = KafkaConsumer(self.topic, - metadata_broker_list=brokers, + bootstrap_servers=brokers, **configs) return consumer From a4b439141b2ef35951e46716696e4c01bb88661c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Mar 2015 16:45:20 -0700 Subject: [PATCH 067/250] Rollover KafkaClient correlation ids at 2**31 to keep within int32 protocol encoding --- kafka/client.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 48a534e..c36cd08 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -2,7 +2,6 @@ import binascii import collections import copy import functools -import itertools import logging import time import kafka.common @@ -23,17 +22,18 @@ log = logging.getLogger("kafka") class KafkaClient(object): CLIENT_ID = b"kafka-python" - ID_GEN = itertools.count() # NOTE: The timeout given to the client should always be greater than the # one passed to SimpleConsumer.get_message(), otherwise you can get a # socket timeout. def __init__(self, hosts, client_id=CLIENT_ID, - timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): + timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS, + correlation_id=0): # We need one connection to bootstrap self.client_id = kafka_bytestring(client_id) self.timeout = timeout self.hosts = collect_hosts(hosts) + self.correlation_id = correlation_id # create connections only when we need them self.conns = {} @@ -98,10 +98,10 @@ class KafkaClient(object): return self.brokers[meta.leader] def _next_id(self): - """ - Generate a new correlation id - """ - return next(KafkaClient.ID_GEN) + """Generate a new correlation id""" + # modulo to keep w/i int32 + self.correlation_id = (self.correlation_id + 1) % 2**31 + return self.correlation_id def _send_broker_unaware_request(self, payloads, encoder_fn, decoder_fn): """ From 1313388662d509ade01f71d0740cd0efe263c01f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Mar 2015 18:52:12 -0700 Subject: [PATCH 068/250] Add test for correlation_id rollover --- test/test_client.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/test/test_client.py b/test/test_client.py index c522d9a..abda421 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -401,3 +401,11 @@ class TestKafkaClient(unittest.TestCase): with self.assertRaises(ConnectionError): KafkaConnection("nowhere", 1234, 1.0) self.assertGreaterEqual(t.interval, 1.0) + + def test_correlation_rollover(self): + with patch.object(KafkaClient, 'load_metadata_for_topics'): + big_num = 2**31 - 3 + client = KafkaClient(hosts=[], correlation_id=big_num) + self.assertEqual(big_num + 1, client._next_id()) + self.assertEqual(big_num + 2, client._next_id()) + self.assertEqual(0, client._next_id()) From bb68965b9e530de7413c6ac98ef13c06158e2d91 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 30 Mar 2015 11:05:48 -0700 Subject: [PATCH 069/250] Stop consumers on delete --- kafka/consumer/base.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index efc9404..bde3c1a 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -143,6 +143,9 @@ class Consumer(object): if self.count_since_commit >= self.auto_commit_every_n: self.commit() + def __del__(self): + self.stop() + def stop(self): if self.commit_timer is not None: self.commit_timer.stop() From 4bc30a2ec8665b1faef0e668c12138c3cc52e38c Mon Sep 17 00:00:00 2001 From: Ali-Akber Saifee Date: Wed, 18 Mar 2015 10:27:04 +0800 Subject: [PATCH 070/250] Add test case for MP Consumer auto commit Tweak MP Consumer test to use iterator --- test/test_consumer_integration.py | 35 +++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 17a8ac9..e5e2148 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -327,6 +327,41 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer1.stop() consumer2.stop() + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + def test_multi_process_offset_behavior__resuming_behavior(self): + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) + + # Start a consumer + consumer1 = self.consumer( + consumer=MultiProcessConsumer, + auto_commit_every_t = None, + auto_commit_every_n = 20, + ) + + # Grab the first 195 messages + output_msgs1 = [] + idx = 0 + for message in consumer1: + output_msgs1.append(message.message.value) + idx += 1 + if idx >= 195: + break + self.assert_message_count(output_msgs1, 195) + + # The total offset across both partitions should be at 180 + consumer2 = self.consumer( + consumer=MultiProcessConsumer, + auto_commit_every_t = None, + auto_commit_every_n = 20, + ) + + # 181-200 + self.assert_message_count([ message for message in consumer2 ], 20) + + consumer1.stop() + consumer2.stop() + # TODO: Make this a unit test -- should not require integration @kafka_versions("all") def test_fetch_buffer_size(self): From 92a3737a6b5267c7b643a9163c81b85ee0b0da58 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Mar 2015 14:04:38 +0300 Subject: [PATCH 071/250] Added basic tests for load_initial_offsets option --- test/test_consumer_integration.py | 35 +++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index e5e2148..70d5109 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -127,6 +127,23 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): with self.assertRaises(OffsetOutOfRangeError): consumer.get_message() + @kafka_versions('all') + def test_simple_consumer_load_initial_offsets(self): + self.send_messages(0, range(0, 100)) + self.send_messages(1, range(100, 200)) + + # Create 1st consumer and change offsets + consumer = self.consumer() + self.assertEqual(consumer.offsets, {0: 0, 1: 0}) + consumer.offsets.update({0:51, 1:101}) + # Update counter after manual offsets update + consumer.count_since_commit += 1 + consumer.commit() + + # Create 2nd consumer and check initial offsets + consumer = self.consumer(auto_commit=False) + self.assertEqual(consumer.offsets, {0: 51, 1: 101}) + @kafka_versions("all") def test_simple_consumer__seek(self): self.send_messages(0, range(0, 100)) @@ -252,6 +269,24 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer.stop() + @kafka_versions("all") + def test_multi_process_consumer_load_initial_offsets(self): + self.send_messages(0, range(0, 10)) + self.send_messages(1, range(10, 20)) + + # Create 1st consumer and change offsets + consumer = self.consumer() + self.assertEqual(consumer.offsets, {0: 0, 1: 0}) + consumer.offsets.update({0:5, 1:15}) + # Update counter after manual offsets update + consumer.count_since_commit += 1 + consumer.commit() + + # Create 2nd consumer and check initial offsets + consumer = self.consumer(consumer = MultiProcessConsumer, + auto_commit=False) + self.assertEqual(consumer.offsets, {0: 5, 1: 15}) + @kafka_versions("all") def test_large_messages(self): # Produce 10 "normal" size messages From 32dd817aac4130a019339afac7ef52f2b9b7acd4 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Mar 2015 15:21:46 +0300 Subject: [PATCH 072/250] Skip these tests: no OffsetCommitRequest for 0.8.0 --- test/test_consumer_integration.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 70d5109..fd62d9b 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -127,7 +127,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): with self.assertRaises(OffsetOutOfRangeError): consumer.get_message() - @kafka_versions('all') + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") def test_simple_consumer_load_initial_offsets(self): self.send_messages(0, range(0, 100)) self.send_messages(1, range(100, 200)) @@ -269,7 +269,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer.stop() - @kafka_versions("all") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") def test_multi_process_consumer_load_initial_offsets(self): self.send_messages(0, range(0, 10)) self.send_messages(1, range(10, 20)) From b6d032cc3f1b53a6d5b395f9b14de62f547c8f1c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Mar 2015 17:24:56 -0700 Subject: [PATCH 073/250] Fetch previously committed offsets in base consumer class so long as a group is configured (but document that group must be None for old servers). This fixes multiprocessor consumer issue that prevented access to commit offsets if auto_commit is disabled. Also refactor fetch_last_known_offsets based on KafkaConsumer While still setting unknown offsets to 0 --- kafka/consumer/base.py | 41 ++++++++++++++++++++----------- kafka/consumer/multiprocess.py | 2 ++ kafka/consumer/simple.py | 2 ++ test/test_consumer_integration.py | 5 +++- test/test_failover_integration.py | 2 +- 5 files changed, 36 insertions(+), 16 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index bde3c1a..91ad82f 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -7,7 +7,7 @@ from threading import Lock import kafka.common from kafka.common import ( OffsetRequest, OffsetCommitRequest, OffsetFetchRequest, - UnknownTopicOrPartitionError + UnknownTopicOrPartitionError, check_error ) from kafka.util import ReentrantTimer @@ -68,29 +68,42 @@ class Consumer(object): self.commit) self.commit_timer.start() - if auto_commit: + # Set initial offsets + if self.group is not None: self.fetch_last_known_offsets(partitions) else: for partition in partitions: self.offsets[partition] = 0 + def fetch_last_known_offsets(self, partitions=None): + if self.group is None: + raise ValueError('KafkaClient.group must not be None') + if not partitions: partitions = self.client.get_partition_ids_for_topic(self.topic) - def get_or_init_offset(resp): - try: - kafka.common.check_error(resp) - return resp.offset - except UnknownTopicOrPartitionError: - return 0 - for partition in partitions: - req = OffsetFetchRequest(self.topic, partition) - (resp,) = self.client.send_offset_fetch_request(self.group, [req], - fail_on_error=False) - self.offsets[partition] = get_or_init_offset(resp) - self.fetch_offsets = self.offsets.copy() + (resp,) = self.client.send_offset_fetch_request( + self.group, + [OffsetFetchRequest(self.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[partition] = 0 + + # Otherwise we committed the stored offset + # and need to fetch the next one + else: + self.offsets[partition] = resp.offset def commit(self, partitions=None): """ diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 5ce8b4d..3acd470 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -93,6 +93,8 @@ class MultiProcessConsumer(Consumer): Arguments: client: a connected KafkaClient group: a name for this consumer, used for offset storage and must be unique + If you are connecting to a server that does not support offset + commit/fetch (any prior to 0.8.1.1), then you *must* set this to None topic: the topic to consume Keyword Arguments: diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index b50de61..ae00dab 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -73,6 +73,8 @@ class SimpleConsumer(Consumer): Arguments: client: a connected KafkaClient group: a name for this consumer, used for offset storage and must be unique + If you are connecting to a server that does not support offset + commit/fetch (any prior to 0.8.1.1), then you *must* set this to None topic: the topic to consume Keyword Arguments: diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index fd62d9b..403ce0f 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -53,6 +53,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): 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['group'] = None kwargs['auto_commit'] = False else: kwargs.setdefault('auto_commit', True) @@ -260,7 +261,9 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(0, range(0, 10)) self.send_messages(1, range(10, 20)) - consumer = MultiProcessConsumer(self.client, "group1", self.topic, + # set group to None and auto_commit to False to avoid interactions w/ + # offset commit/fetch apis + consumer = MultiProcessConsumer(self.client, None, self.topic, auto_commit=False, iter_timeout=0) self.assertEqual(consumer.pending(), 20) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 7d27526..15f0338 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -183,7 +183,7 @@ class TestFailover(KafkaIntegrationTestCase): client = KafkaClient(hosts) group = random_string(10) - consumer = SimpleConsumer(client, group, topic, + consumer = SimpleConsumer(client, None, topic, partitions=partitions, auto_commit=False, iter_timeout=timeout) From 1d252bfc20c8b1058dc93a495c3bdb0f4ccdf590 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 30 Mar 2015 16:55:41 -0700 Subject: [PATCH 074/250] Bulk fetch offset partitions in base consumer -- suggested by ecanzonieri --- kafka/consumer/base.py | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 91ad82f..0bbf46c 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -83,12 +83,13 @@ class Consumer(object): if not partitions: partitions = self.client.get_partition_ids_for_topic(self.topic) - for partition in partitions: - (resp,) = self.client.send_offset_fetch_request( - self.group, - [OffsetFetchRequest(self.topic, partition)], - fail_on_error=False - ) + responses = self.client.send_offset_fetch_request( + self.group, + [OffsetFetchRequest(self.topic, p) for p in partitions], + fail_on_error=False + ) + + for resp in responses: try: check_error(resp) # API spec says server wont set an error here @@ -98,12 +99,12 @@ class Consumer(object): # -1 offset signals no commit is currently stored if resp.offset == -1: - self.offsets[partition] = 0 + self.offsets[resp.partition] = 0 # Otherwise we committed the stored offset # and need to fetch the next one else: - self.offsets[partition] = resp.offset + self.offsets[resp.partition] = resp.offset def commit(self, partitions=None): """ From 47989db113ff1603b081867f3914e0c0828dfc9c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 31 Mar 2015 10:29:55 -0700 Subject: [PATCH 075/250] Register atexit handlers for consumer and producer thread/multiprocess cleanup (not __del__) --- kafka/consumer/base.py | 28 +++++++++++++++++++++++++--- kafka/producer/base.py | 27 ++++++++++++++++++++++++++- 2 files changed, 51 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 0bbf46c..64d96ea 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -1,5 +1,6 @@ from __future__ import absolute_import +import atexit import logging import numbers from threading import Lock @@ -75,6 +76,11 @@ class Consumer(object): for partition in partitions: self.offsets[partition] = 0 + # Register a cleanup handler + def cleanup(obj): + obj.stop() + self._cleanup_func = cleanup + atexit.register(cleanup, self) def fetch_last_known_offsets(self, partitions=None): if self.group is None: @@ -157,14 +163,30 @@ class Consumer(object): if self.count_since_commit >= self.auto_commit_every_n: self.commit() - def __del__(self): - self.stop() - def stop(self): if self.commit_timer is not None: self.commit_timer.stop() self.commit() + if hasattr(self, '_cleanup_func'): + # Remove cleanup handler now that we've stopped + + # py3 supports unregistering + if hasattr(atexit, 'unregister'): + atexit.unregister(self._cleanup_func) # pylint: disable=no-member + + # py2 requires removing from private attribute... + else: + + # ValueError on list.remove() if the exithandler no longer + # exists is fine here + try: + atexit._exithandlers.remove((self._cleanup_func, (self,), {})) + except ValueError: + pass + + del self._cleanup_func + def pending(self, partitions=None): """ Gets the pending message count diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 6a5a94e..2344168 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -1,5 +1,6 @@ from __future__ import absolute_import +import atexit import logging import time @@ -152,7 +153,11 @@ class Producer(object): self.thread.daemon = True self.thread.start() - + def cleanup(obj): + if obj.stopped: + obj.stop() + self._cleanup_func = cleanup + atexit.register(cleanup, self) def send_messages(self, topic, partition, *msg): """ @@ -213,6 +218,26 @@ class Producer(object): if self.thread.is_alive(): self.thread_stop_event.set() + + if hasattr(self, '_cleanup_func'): + # Remove cleanup handler now that we've stopped + + # py3 supports unregistering + if hasattr(atexit, 'unregister'): + atexit.unregister(self._cleanup_func) # pylint: disable=no-member + + # py2 requires removing from private attribute... + else: + + # ValueError on list.remove() if the exithandler no longer exists + # but that is fine here + try: + atexit._exithandlers.remove((self._cleanup_func, (self,), {})) + except ValueError: + pass + + del self._cleanup_func + self.stopped = True def __del__(self): From 1c856e8400e1c4fe6dccd562fbcf4d1bde38755d Mon Sep 17 00:00:00 2001 From: Space Date: Tue, 31 Mar 2015 15:25:38 -0700 Subject: [PATCH 076/250] Make external API consistently support python3 strings for topic. --- kafka/client.py | 3 ++ kafka/consumer/base.py | 6 +-- kafka/consumer/multiprocess.py | 2 +- kafka/producer/base.py | 6 +++ kafka/producer/keyed.py | 4 ++ test/test_client.py | 70 ++++++++++++++++--------------- test/test_client_integration.py | 8 ++-- test/test_consumer_integration.py | 2 +- test/test_failover_integration.py | 5 ++- test/test_producer_integration.py | 4 +- test/testutil.py | 8 +++- 11 files changed, 69 insertions(+), 49 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index c36cd08..4cd9e24 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -258,12 +258,14 @@ class KafkaClient(object): self.topic_partitions.clear() def has_metadata_for_topic(self, topic): + topic = kafka_bytestring(topic) return ( topic in self.topic_partitions and len(self.topic_partitions[topic]) > 0 ) def get_partition_ids_for_topic(self, topic): + topic = kafka_bytestring(topic) if topic not in self.topic_partitions: return [] @@ -312,6 +314,7 @@ class KafkaClient(object): Partition-level errors will also not be raised here (a single partition w/o a leader, for example) """ + topics = [kafka_bytestring(t) for t in topics] resp = self.send_metadata_request(topics) log.debug("Broker metadata: %s", resp.brokers) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 0bbf46c..2bd42eb 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -10,7 +10,7 @@ from kafka.common import ( UnknownTopicOrPartitionError, check_error ) -from kafka.util import ReentrantTimer +from kafka.util import kafka_bytestring, ReentrantTimer log = logging.getLogger("kafka") @@ -44,8 +44,8 @@ class Consumer(object): auto_commit_every_t=AUTO_COMMIT_INTERVAL): self.client = client - self.topic = topic - self.group = group + self.topic = kafka_bytestring(topic) + self.group = None if group is None else kafka_bytestring(group) self.client.load_metadata_for_topics(topic) self.offsets = {} diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 3acd470..cfe0ef6 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -163,7 +163,7 @@ class MultiProcessConsumer(Consumer): simple_consumer_options.pop('partitions', None) options.update(simple_consumer_options) - args = (client.copy(), group, topic, self.queue, + args = (client.copy(), self.group, self.topic, self.queue, self.size, self.events) proc = Process(target=_mp_consume, args=args, kwargs=options) proc.daemon = True diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 6a5a94e..00c4d46 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -17,6 +17,7 @@ from kafka.common import ( ProduceRequest, TopicAndPartition, UnsupportedCodecError ) from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set +from kafka.util import kafka_bytestring log = logging.getLogger("kafka") @@ -170,6 +171,7 @@ class Producer(object): All messages produced via this method will set the message 'key' to Null """ + topic = kafka_bytestring(topic) return self._send_messages(topic, partition, *msg) def _send_messages(self, topic, partition, *msg, **kwargs): @@ -183,6 +185,10 @@ class Producer(object): if any(not isinstance(m, six.binary_type) for m in msg): raise TypeError("all produce message payloads must be type bytes") + # Raise TypeError if topic is not encoded as bytes + if not isinstance(topic, six.binary_type): + raise TypeError("the topic must be type bytes") + # Raise TypeError if the key is not encoded as bytes if key is not None and not isinstance(key, six.binary_type): raise TypeError("the key must be type bytes") diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 36328ed..333b6c0 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -3,6 +3,8 @@ from __future__ import absolute_import import logging from kafka.partitioner import HashedPartitioner +from kafka.util import kafka_bytestring + from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, BATCH_SEND_MSG_COUNT @@ -57,10 +59,12 @@ class KeyedProducer(Producer): return partitioner.partition(key) def send_messages(self,topic,key,*msg): + topic = kafka_bytestring(topic) partition = self._next_partition(topic, key) return self._send_messages(topic, partition, *msg,key=key) def send(self, topic, key, msg): + topic = kafka_bytestring(topic) partition = self._next_partition(topic, key) return self._send_messages(topic, partition, msg, key=key) diff --git a/test/test_client.py b/test/test_client.py index abda421..bab7916 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -117,21 +117,21 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_1', NO_ERROR, [ - PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2], NO_ERROR) + TopicMetadata(b'topic_1', NO_ERROR, [ + PartitionMetadata(b'topic_1', 0, 1, [1, 2], [1, 2], NO_ERROR) ]), - TopicMetadata('topic_noleader', NO_ERROR, [ - PartitionMetadata('topic_noleader', 0, -1, [], [], + TopicMetadata(b'topic_noleader', NO_ERROR, [ + PartitionMetadata(b'topic_noleader', 0, -1, [], [], NO_LEADER), - PartitionMetadata('topic_noleader', 1, -1, [], [], + PartitionMetadata(b'topic_noleader', 1, -1, [], [], NO_LEADER), ]), - TopicMetadata('topic_no_partitions', NO_LEADER, []), - TopicMetadata('topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), - TopicMetadata('topic_3', NO_ERROR, [ - PartitionMetadata('topic_3', 0, 0, [0, 1], [0, 1], NO_ERROR), - PartitionMetadata('topic_3', 1, 1, [1, 0], [1, 0], NO_ERROR), - PartitionMetadata('topic_3', 2, 0, [0, 1], [0, 1], NO_ERROR) + TopicMetadata(b'topic_no_partitions', NO_LEADER, []), + TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), + TopicMetadata(b'topic_3', NO_ERROR, [ + PartitionMetadata(b'topic_3', 0, 0, [0, 1], [0, 1], NO_ERROR), + PartitionMetadata(b'topic_3', 1, 1, [1, 0], [1, 0], NO_ERROR), + PartitionMetadata(b'topic_3', 2, 0, [0, 1], [0, 1], NO_ERROR) ]) ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -139,12 +139,12 @@ class TestKafkaClient(unittest.TestCase): # client loads metadata at init client = KafkaClient(hosts=['broker_1:4567']) self.assertDictEqual({ - TopicAndPartition('topic_1', 0): brokers[1], - TopicAndPartition('topic_noleader', 0): None, - TopicAndPartition('topic_noleader', 1): None, - TopicAndPartition('topic_3', 0): brokers[0], - TopicAndPartition('topic_3', 1): brokers[1], - TopicAndPartition('topic_3', 2): brokers[0]}, + TopicAndPartition(b'topic_1', 0): brokers[1], + TopicAndPartition(b'topic_noleader', 0): None, + TopicAndPartition(b'topic_noleader', 1): None, + TopicAndPartition(b'topic_3', 0): brokers[0], + TopicAndPartition(b'topic_3', 1): brokers[1], + TopicAndPartition(b'topic_3', 2): brokers[0]}, client.topics_to_brokers) # if we ask for metadata explicitly, it should raise errors @@ -156,6 +156,7 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.load_metadata_for_topics('topic_no_leader') + client.load_metadata_for_topics(b'topic_no_leader') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') @@ -169,11 +170,11 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_still_creating', NO_LEADER, []), - TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), - TopicMetadata('topic_noleaders', NO_ERROR, [ - PartitionMetadata('topic_noleaders', 0, -1, [], [], NO_LEADER), - PartitionMetadata('topic_noleaders', 1, -1, [], [], NO_LEADER), + TopicMetadata(b'topic_still_creating', NO_LEADER, []), + TopicMetadata(b'topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), + TopicMetadata(b'topic_noleaders', NO_ERROR, [ + PartitionMetadata(b'topic_noleaders', 0, -1, [], [], NO_LEADER), + PartitionMetadata(b'topic_noleaders', 1, -1, [], [], NO_LEADER), ]), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -188,8 +189,8 @@ class TestKafkaClient(unittest.TestCase): self.assertTrue(client.has_metadata_for_topic('topic_noleaders')) @patch('kafka.client.KafkaConnection') - @patch('kafka.client.KafkaProtocol') - def test_ensure_topic_exists(self, protocol, conn): + @patch('kafka.client.KafkaProtocol.decode_metadata_response') + def test_ensure_topic_exists(self, decode_metadata_response, conn): conn.recv.return_value = 'response' # anything but None @@ -199,14 +200,14 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_still_creating', NO_LEADER, []), - TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), - TopicMetadata('topic_noleaders', NO_ERROR, [ - PartitionMetadata('topic_noleaders', 0, -1, [], [], NO_LEADER), - PartitionMetadata('topic_noleaders', 1, -1, [], [], NO_LEADER), + TopicMetadata(b'topic_still_creating', NO_LEADER, []), + TopicMetadata(b'topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), + TopicMetadata(b'topic_noleaders', NO_ERROR, [ + PartitionMetadata(b'topic_noleaders', 0, -1, [], [], NO_LEADER), + PartitionMetadata(b'topic_noleaders', 1, -1, [], [], NO_LEADER), ]), ] - protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) + decode_metadata_response.return_value = MetadataResponse(brokers, topics) client = KafkaClient(hosts=['broker_1:4567']) @@ -218,6 +219,7 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.ensure_topic_exists('topic_noleaders', timeout=1) + client.ensure_topic_exists(b'topic_noleaders', timeout=1) @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') @@ -269,8 +271,8 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_no_partitions', NO_LEADER, []), - TopicMetadata('topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), + TopicMetadata(b'topic_no_partitions', NO_LEADER, []), + TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -279,10 +281,10 @@ class TestKafkaClient(unittest.TestCase): self.assertDictEqual({}, client.topics_to_brokers) with self.assertRaises(LeaderNotAvailableError): - client._get_leader_for_partition('topic_no_partitions', 0) + client._get_leader_for_partition(b'topic_no_partitions', 0) with self.assertRaises(UnknownTopicOrPartitionError): - client._get_leader_for_partition('topic_unknown', 0) + client._get_leader_for_partition(b'topic_unknown', 0) @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') diff --git a/test/test_client_integration.py b/test/test_client_integration.py index c0331ea..d963d85 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -29,11 +29,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_consume_none(self): - fetch = FetchRequest(self.topic, 0, 0, 1024) + fetch = FetchRequest(self.bytes_topic, 0, 0, 1024) fetch_resp, = self.client.send_fetch_request([fetch]) self.assertEqual(fetch_resp.error, 0) - self.assertEqual(fetch_resp.topic, self.topic) + self.assertEqual(fetch_resp.topic, self.bytes_topic) self.assertEqual(fetch_resp.partition, 0) messages = list(fetch_resp.messages) @@ -56,11 +56,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") def test_commit_fetch_offsets(self): - req = OffsetCommitRequest(self.topic, 0, 42, b"metadata") + req = OffsetCommitRequest(self.bytes_topic, 0, 42, b"metadata") (resp,) = self.client.send_offset_commit_request(b"group", [req]) self.assertEqual(resp.error, 0) - req = OffsetFetchRequest(self.topic, 0) + req = OffsetFetchRequest(self.bytes_topic, 0) (resp,) = self.client.send_offset_fetch_request(b"group", [req]) self.assertEqual(resp.error, 0) self.assertEqual(resp.offset, 42) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 403ce0f..ddb54a7 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -37,7 +37,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def send_messages(self, partition, messages): messages = [ create_message(self.msg(str(msg))) for msg in messages ] - produce = ProduceRequest(self.topic, partition, messages = messages) + produce = ProduceRequest(self.bytes_topic, partition, messages = messages) resp, = self.client.send_produce_request([produce]) self.assertEqual(resp.error, 0) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 15f0338..3be0189 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -8,6 +8,7 @@ from kafka import KafkaClient, SimpleConsumer from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer from kafka.producer import KeyedProducer +from kafka.util import kafka_bytestring from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( @@ -147,7 +148,7 @@ class TestFailover(KafkaIntegrationTestCase): key = random_string(3) msg = random_string(10) producer.send_messages(topic, key, msg) - if producer.partitioners[topic].partition(key) == 0: + if producer.partitioners[kafka_bytestring(topic)].partition(key) == 0: recovered = True except (FailedPayloadsError, ConnectionError): logging.debug("caught exception sending message -- will retry") @@ -172,7 +173,7 @@ class TestFailover(KafkaIntegrationTestCase): logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j) def _kill_leader(self, topic, partition): - leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] + leader = self.client.topics_to_brokers[TopicAndPartition(kafka_bytestring(topic), partition)] broker = self.brokers[leader.nodeId] broker.close() return broker diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 1804af0..e3f7767 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -453,7 +453,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): def assert_produce_request(self, messages, initial_offset, message_ct, partition=0): - produce = ProduceRequest(self.topic, partition, messages=messages) + produce = ProduceRequest(self.bytes_topic, partition, messages=messages) # There should only be one response message from the server. # This will throw an exception if there's more than one. @@ -471,7 +471,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # There should only be one response message from the server. # This will throw an exception if there's more than one. - resp, = self.client.send_fetch_request([ FetchRequest(self.topic, partition, start_offset, 1024) ]) + resp, = self.client.send_fetch_request([ FetchRequest(self.bytes_topic, partition, start_offset, 1024) ]) self.assertEqual(resp.error, 0) self.assertEqual(resp.partition, partition) diff --git a/test/testutil.py b/test/testutil.py index 7661cbc..e6947b4 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -12,6 +12,7 @@ from . import unittest from kafka import KafkaClient from kafka.common import OffsetRequest +from kafka.util import kafka_bytestring __all__ = [ 'random_string', @@ -50,6 +51,7 @@ def get_open_port(): class KafkaIntegrationTestCase(unittest.TestCase): create_client = True topic = None + bytes_topic = None server = None def setUp(self): @@ -59,7 +61,8 @@ class KafkaIntegrationTestCase(unittest.TestCase): if not self.topic: topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10).decode('utf-8')) - self.topic = topic.encode('utf-8') + self.topic = topic + self.bytes_topic = topic.encode('utf-8') if self.create_client: self.client = KafkaClient('%s:%d' % (self.server.host, self.server.port)) @@ -77,7 +80,8 @@ class KafkaIntegrationTestCase(unittest.TestCase): self.client.close() def current_offset(self, topic, partition): - offsets, = self.client.send_offset_request([ OffsetRequest(topic, partition, -1, 1) ]) + offsets, = self.client.send_offset_request([ OffsetRequest(kafka_bytestring(topic), + partition, -1, 1) ]) return offsets.offsets[0] def msgs(self, iterable): From e99a751081f59189ed818220100a1db2d8a13b6f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 4 Apr 2015 13:57:15 -0700 Subject: [PATCH 077/250] Fix python3 str/bytes bug in KafkaConsumer.set_topic_partitions({(topic, partition): offset, }) --- 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 6f5bcdd..59597d9 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -224,7 +224,7 @@ class KafkaConsumer(object): topic = kafka_bytestring(key[0]) partition = key[1] self._consume_topic_partition(topic, partition) - self._offsets.fetch[key] = value + self._offsets.fetch[(topic, partition)] = value else: raise KafkaConfigurationError('Unknown topic type (%s)' % type(arg)) From 87b2ca8e60832170a3c4ab3e391509ce40cb6faa Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 4 Apr 2015 20:19:40 -0700 Subject: [PATCH 078/250] call _consume_topic_partition() before storing offset in KafkaConsumer.set_topic_partitions((topic, partition, offset)) --- 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 59597d9..423ba63 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -194,10 +194,10 @@ class KafkaConsumer(object): elif isinstance(arg, tuple): topic = kafka_bytestring(arg[0]) partition = arg[1] + self._consume_topic_partition(topic, partition) 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): From 811fd4cbb903064e3961c60a0b39c43b9473c322 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 4 Apr 2015 21:04:58 -0700 Subject: [PATCH 079/250] Use list comprehension on _topics in KafkaConsumer.fetch_messages --- kafka/consumer/kafka.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 423ba63..79cee28 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -312,16 +312,16 @@ class KafkaConsumer(object): max_wait_time = self._config['fetch_wait_max_ms'] min_bytes = self._config['fetch_min_bytes'] - # Get current fetch offsets - offsets = self._offsets.fetch - if not offsets: - if not self._topics: - raise KafkaConfigurationError('No topics or partitions configured') + if not self._topics: + raise KafkaConfigurationError('No topics or partitions configured') + + if not self._offsets.fetch: raise KafkaConfigurationError('No fetch offsets found when calling fetch_messages') - fetches = [] - for topic_partition, offset in six.iteritems(offsets): - fetches.append(FetchRequest(topic_partition[0], topic_partition[1], offset, max_bytes)) + fetches = [FetchRequest(topic, partition, + self._offsets.fetch[(topic, partition)], + max_bytes) + for (topic, partition) in self._topics] # client.send_fetch_request will collect topic/partition requests by leader # and send each group as a single FetchRequest to the correct broker From c0fc334612f7a98ab98c0f970288ebe0023b42db Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 4 Apr 2015 21:34:24 -0700 Subject: [PATCH 080/250] Use kafka_bytestring when decoding message topics in KafkaConsumer.fetch_messages --- kafka/consumer/kafka.py | 38 +++++++++++++++++++++----------------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 79cee28..47a5b00 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -336,49 +336,53 @@ class KafkaConsumer(object): return for resp in responses: - topic_partition = (resp.topic, resp.partition) + topic = kafka_bytestring(resp.topic) + partition = 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) + logger.warning('OffsetOutOfRange: topic %s, partition %d, ' + 'offset %d (Highwatermark: %d)', + topic, partition, + self.offsets._fetch[(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", - resp.topic, resp.partition) + topic, partition) self._refresh_metadata_on_error() continue except RequestTimedOutError: logger.warning("RequestTimedOutError for %s - %d", - resp.topic, resp.partition) + topic, 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(resp.topic, - resp.partition, - offset, message.key, - self._config['deserializer_class'](message.value)) + val = self._config['deserializer_class'](message.value) + msg = KafkaMessage(topic, partition, offset, message.key, val) - if offset < self._offsets.fetch[topic_partition]: - logger.debug('Skipping message %s because its offset is less than the consumer offset', - msg) + # in some cases the server will return earlier messages + # than we requested. skip them per kafka spec + if offset < self._offsets.fetch[(topic, partition)]: + logger.debug('message offset less than fetched offset ' + 'skipping: %s', msg) continue # 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 From 5cc051353eff67713bd4885372e1aee12fab22f0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 5 Apr 2015 17:46:13 -0700 Subject: [PATCH 081/250] Refactor KafkaClient._send_broker_aware_request to return a list of responses and include individual (unraised) FailedPayloadsError instances rather than always raising a FailedPayloadsError. This should allow producers to determine which payloads succeeded and which failed, specifically. --- kafka/client.py | 130 +++++++++++++++++++++--------------------------- 1 file changed, 57 insertions(+), 73 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 4cd9e24..93eeb52 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -151,7 +151,7 @@ class KafkaClient(object): """ # Group the requests by topic+partition - original_keys = [] + brokers_for_payloads = [] payloads_by_broker = collections.defaultdict(list) for payload in payloads: @@ -159,67 +159,88 @@ class KafkaClient(object): payload.partition) payloads_by_broker[leader].append(payload) - original_keys.append((payload.topic, payload.partition)) - - # Accumulate the responses in a dictionary - acc = {} - - # keep a list of payloads that were failed to be sent to brokers - failed_payloads = [] + brokers_for_payloads.append(leader) # For each broker, send the list of request payloads + # and collect the responses and errors + responses_by_broker = collections.defaultdict(list) + broker_failures = [] for broker, payloads in payloads_by_broker.items(): conn = self._get_conn(broker.host.decode('utf-8'), broker.port) requestId = self._next_id() request = encoder_fn(client_id=self.client_id, correlation_id=requestId, payloads=payloads) - failed = False # Send the request, recv the response try: conn.send(requestId, request) + except ConnectionError as e: + broker_failures.append(broker) + log.warning("Could not send request [%s] to server %s: %s", + binascii.b2a_hex(request), conn, e) + + for payload in payloads: + responses_by_broker[broker].append(FailedPayloadsError(payload)) + + # No exception, try to get response + else: + # decoder_fn=None signal that the server is expected to not # send a response. This probably only applies to # ProduceRequest w/ acks = 0 if decoder_fn is None: + for payload in payloads: + responses_by_broker[broker].append(None) continue try: response = conn.recv(requestId) except ConnectionError as e: + broker_failures.append(broker) log.warning("Could not receive response to request [%s] " - "from server %s: %s", binascii.b2a_hex(request), conn, e) - failed = True - except ConnectionError as e: - log.warning("Could not send request [%s] to server %s: %s", - binascii.b2a_hex(request), conn, e) - failed = True + "from server %s: %s", + binascii.b2a_hex(request), conn, e) - if failed: - failed_payloads += payloads - self.reset_all_metadata() - continue + for payload in payloads: + responses_by_broker[broker].append(FailedPayloadsError(payload)) - for response in decoder_fn(response): - acc[(response.topic, response.partition)] = response + else: - if failed_payloads: - raise FailedPayloadsError(failed_payloads) + for payload_response in decoder_fn(response): + responses_by_broker[broker].append(payload_response) - # Order the accumulated responses by the original key order - return (acc[k] for k in original_keys) if acc else () + # Connection errors generally mean stale metadata + # although sometimes it means incorrect api request + # Unfortunately there is no good way to tell the difference + # so we'll just reset metadata on all errors to be safe + if broker_failures: + self.reset_all_metadata() + + # Return responses in the same order as provided + responses_by_payload = [responses_by_broker[broker].pop(0) + for broker in brokers_for_payloads] + return responses_by_payload def __repr__(self): return '' % (self.client_id) def _raise_on_response_error(self, resp): + + # Response can be an unraised exception object (FailedPayloadsError) + if isinstance(resp, Exception): + raise resp + + # Or a server api error response try: kafka.common.check_error(resp) except (UnknownTopicOrPartitionError, NotLeaderForPartitionError): self.reset_topic_metadata(resp.topic) raise + # Return False if no error to enable list comprehensions + return False + ################# # Public API # ################# @@ -419,16 +440,9 @@ class KafkaClient(object): resps = self._send_broker_aware_request(payloads, encoder, decoder) - out = [] - for resp in resps: - if fail_on_error is True: - self._raise_on_response_error(resp) - - if callback is not None: - out.append(callback(resp)) - else: - out.append(resp) - return out + return [resp if not callback else callback(resp) for resp in resps + if resp is not None and + (not fail_on_error or not self._raise_on_response_error(resp))] def send_fetch_request(self, payloads=[], fail_on_error=True, callback=None, max_wait_time=100, min_bytes=4096): @@ -447,16 +461,8 @@ class KafkaClient(object): payloads, encoder, KafkaProtocol.decode_fetch_response) - out = [] - for resp in resps: - if fail_on_error is True: - self._raise_on_response_error(resp) - - if callback is not None: - out.append(callback(resp)) - else: - out.append(resp) - return out + return [resp if not callback else callback(resp) for resp in resps + if not fail_on_error or not self._raise_on_response_error(resp)] def send_offset_request(self, payloads=[], fail_on_error=True, callback=None): @@ -465,15 +471,8 @@ class KafkaClient(object): KafkaProtocol.encode_offset_request, KafkaProtocol.decode_offset_response) - out = [] - for resp in resps: - if fail_on_error is True: - self._raise_on_response_error(resp) - if callback is not None: - out.append(callback(resp)) - else: - out.append(resp) - return out + return [resp if not callback else callback(resp) for resp in resps + if not fail_on_error or not self._raise_on_response_error(resp)] def send_offset_commit_request(self, group, payloads=[], fail_on_error=True, callback=None): @@ -482,16 +481,8 @@ class KafkaClient(object): decoder = KafkaProtocol.decode_offset_commit_response resps = self._send_broker_aware_request(payloads, encoder, decoder) - out = [] - for resp in resps: - if fail_on_error is True: - self._raise_on_response_error(resp) - - if callback is not None: - out.append(callback(resp)) - else: - out.append(resp) - return out + return [resp if not callback else callback(resp) for resp in resps + if not fail_on_error or not self._raise_on_response_error(resp)] def send_offset_fetch_request(self, group, payloads=[], fail_on_error=True, callback=None): @@ -501,12 +492,5 @@ class KafkaClient(object): decoder = KafkaProtocol.decode_offset_fetch_response resps = self._send_broker_aware_request(payloads, encoder, decoder) - out = [] - for resp in resps: - if fail_on_error is True: - self._raise_on_response_error(resp) - if callback is not None: - out.append(callback(resp)) - else: - out.append(resp) - return out + return [resp if not callback else callback(resp) for resp in resps + if not fail_on_error or not self._raise_on_response_error(resp)] From fdec16dc45c6e83d53538478282a00ab428b113c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 5 Apr 2015 18:32:23 -0700 Subject: [PATCH 082/250] Add debug logging for all broker-aware requests/responses --- kafka/client.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/client.py b/kafka/client.py index 93eeb52..08d74d7 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -150,6 +150,8 @@ class KafkaClient(object): List of response objects in the same order as the supplied payloads """ + log.debug("Sending Payloads: %s" % payloads) + # Group the requests by topic+partition brokers_for_payloads = [] payloads_by_broker = collections.defaultdict(list) @@ -220,6 +222,7 @@ class KafkaClient(object): # Return responses in the same order as provided responses_by_payload = [responses_by_broker[broker].pop(0) for broker in brokers_for_payloads] + log.debug('Responses: %s' % responses_by_payload) return responses_by_payload def __repr__(self): From 94f4c9e4c4a6503884527f90908ba092a7f7586c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 5 Apr 2015 18:32:47 -0700 Subject: [PATCH 083/250] Improve send_produce_request docstring --- kafka/client.py | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 08d74d7..9eb8a0d 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -420,14 +420,25 @@ class KafkaClient(object): same order as the list of payloads specified Arguments: - payloads: list of ProduceRequest - fail_on_error: boolean, should we raise an Exception if we - encounter an API error? - callback: function, instead of returning the ProduceResponse, - first pass it through this function + payloads (list of ProduceRequest): produce requests to send to kafka + acks (int, optional): how many acks the servers should receive from replica + brokers before responding to the request. If it is 0, the server + will not send any response. If it is 1, the server will wait + until the data is written to the local log before sending a + response. If it is -1, the server will wait until the message + is committed by all in-sync replicas before sending a response. + For any value > 1, the server will wait for this number of acks to + occur (but the server will never wait for more acknowledgements than + there are in-sync replicas). defaults to 1. + timeout (int, optional): maximum time in milliseconds the server can + await the receipt of the number of acks, defaults to 1000. + fail_on_error (bool, optional): raise exceptions on connection and + server response errors, defaults to True. + callback (function, optional): instead of returning the ProduceResponse, + first pass it through this function, defaults to None. Returns: - list of ProduceResponse or callback(ProduceResponse), in the + list of ProduceResponses, or callback results if supplied, in the order of input payloads """ From fbc242fac09e3913b3e5a7bb73b0440da1bc2c11 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 5 Apr 2015 19:36:25 -0700 Subject: [PATCH 084/250] Simplify client.reset_topic_metadata --- kafka/client.py | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 4cd9e24..a966f92 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -243,15 +243,11 @@ class KafkaClient(object): def reset_topic_metadata(self, *topics): for topic in topics: - try: - partitions = self.topic_partitions[topic] - except KeyError: - continue - - for partition in partitions: - self.topics_to_brokers.pop(TopicAndPartition(topic, partition), None) - - del self.topic_partitions[topic] + for topic_partition in list(self.topics_to_brokers.keys()): + if topic_partition.topic == topic: + del self.topics_to_brokers[topic_partition] + if topic in self.topic_partitions: + del self.topic_partitions[topic] def reset_all_metadata(self): self.topics_to_brokers.clear() From d5c9a11a574f06a1c9dd7ac4a98d0dc30452480c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 5 Apr 2015 19:38:18 -0700 Subject: [PATCH 085/250] clean local metadata before requesting refresh in load_metadata_for_topics() --- kafka/client.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index a966f92..f01c1c5 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -311,6 +311,13 @@ class KafkaClient(object): (a single partition w/o a leader, for example) """ topics = [kafka_bytestring(t) for t in topics] + + if topics: + for topic in topics: + self.reset_topic_metadata(topic) + else: + self.reset_all_metadata() + resp = self.send_metadata_request(topics) log.debug("Broker metadata: %s", resp.brokers) @@ -323,8 +330,6 @@ class KafkaClient(object): topic = topic_metadata.topic partitions = topic_metadata.partitions - self.reset_topic_metadata(topic) - # Errors expected for new topics try: kafka.common.check_error(topic_metadata) From 6326e18cd89d55fc8e83d313f365ddafea272601 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 5 Apr 2015 19:39:34 -0700 Subject: [PATCH 086/250] Update debug log messages in client.load_metadata_for_topics --- kafka/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index f01c1c5..7675ad4 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -320,8 +320,8 @@ class KafkaClient(object): resp = self.send_metadata_request(topics) - log.debug("Broker metadata: %s", resp.brokers) - log.debug("Topic metadata: %s", resp.topics) + log.debug("Received new broker metadata: %s", resp.brokers) + log.debug("Received new topic metadata: %s", resp.topics) self.brokers = dict([(broker.nodeId, broker) for broker in resp.brokers]) From 353c01eb1caac594e4c93af352921a48f12aa20a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arturo=20Filast=C3=B2?= Date: Thu, 14 May 2015 15:49:46 +0200 Subject: [PATCH 087/250] Fix calling of _offsets Previously you would see this error: ``` self.offsets._fetch[(topic, partition)], AttributeError: 'function' object has no attribute '_fetch' ``` --- 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 47a5b00..60f1a0b 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -344,7 +344,7 @@ class KafkaConsumer(object): logger.warning('OffsetOutOfRange: topic %s, partition %d, ' 'offset %d (Highwatermark: %d)', topic, partition, - self.offsets._fetch[(topic, partition)], + self._offsets.fetch[(topic, partition)], resp.highwaterMark) # Reset offset self._offsets.fetch[(topic, partition)] = ( From 45c263037cba15e120a103a9e21753c1c7a277b2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 17 May 2015 16:24:20 -0700 Subject: [PATCH 088/250] Deprecate KeyedProducer.send in favor of send_messages -- keep interface consistent --- docs/usage.rst | 4 ++-- kafka/producer/keyed.py | 11 ++++++----- test/test_producer_integration.py | 20 ++++++++++---------- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/docs/usage.rst b/docs/usage.rst index 150d121..3e5f434 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -63,8 +63,8 @@ Keyed messages # HashedPartitioner is default producer = KeyedProducer(kafka) - producer.send("my-topic", "key1", "some message") - producer.send("my-topic", "key2", "this methode") + producer.send_messages("my-topic", "key1", "some message") + producer.send_messages("my-topic", "key2", "this methode") producer = KeyedProducer(kafka, partitioner=RoundRobinPartitioner) diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 333b6c0..bc42803 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -1,6 +1,7 @@ from __future__ import absolute_import import logging +import warnings from kafka.partitioner import HashedPartitioner from kafka.util import kafka_bytestring @@ -58,15 +59,15 @@ class KeyedProducer(Producer): partitioner = self.partitioners[topic] return partitioner.partition(key) - def send_messages(self,topic,key,*msg): + def send_messages(self, topic, key, *msg): topic = kafka_bytestring(topic) partition = self._next_partition(topic, key) - return self._send_messages(topic, partition, *msg,key=key) + return self._send_messages(topic, partition, *msg, key=key) + # DEPRECATED def send(self, topic, key, msg): - topic = kafka_bytestring(topic) - partition = self._next_partition(topic, key) - return self._send_messages(topic, partition, msg, key=key) + warnings.warn("KeyedProducer.send is deprecated in favor of send_messages", DeprecationWarning) + return self.send_messages(topic, key, msg) def __repr__(self): return '' % self.async diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index e3f7767..c81716d 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -332,10 +332,10 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): start_offsets = [self.current_offset(self.topic, p) for p in partitions] producer = KeyedProducer(self.client, partitioner=RoundRobinPartitioner) - resp1 = producer.send(self.topic, self.key("key1"), self.msg("one")) - resp2 = producer.send(self.topic, self.key("key2"), self.msg("two")) - resp3 = producer.send(self.topic, self.key("key3"), self.msg("three")) - resp4 = producer.send(self.topic, self.key("key4"), self.msg("four")) + resp1 = producer.send_messages(self.topic, self.key("key1"), self.msg("one")) + resp2 = producer.send_messages(self.topic, self.key("key2"), self.msg("two")) + resp3 = producer.send_messages(self.topic, self.key("key3"), self.msg("three")) + resp4 = producer.send_messages(self.topic, self.key("key4"), self.msg("four")) self.assert_produce_response(resp1, start_offsets[0]+0) self.assert_produce_response(resp2, start_offsets[1]+0) @@ -353,11 +353,11 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): start_offsets = [self.current_offset(self.topic, p) for p in partitions] producer = KeyedProducer(self.client, partitioner=HashedPartitioner) - resp1 = producer.send(self.topic, self.key("1"), self.msg("one")) - resp2 = producer.send(self.topic, self.key("2"), self.msg("two")) - resp3 = producer.send(self.topic, self.key("3"), self.msg("three")) - resp4 = producer.send(self.topic, self.key("3"), self.msg("four")) - resp5 = producer.send(self.topic, self.key("4"), self.msg("five")) + resp1 = producer.send_messages(self.topic, self.key("1"), self.msg("one")) + resp2 = producer.send_messages(self.topic, self.key("2"), self.msg("two")) + resp3 = producer.send_messages(self.topic, self.key("3"), self.msg("three")) + resp4 = producer.send_messages(self.topic, self.key("3"), self.msg("four")) + resp5 = producer.send_messages(self.topic, self.key("4"), self.msg("five")) offsets = {partitions[0]: start_offsets[0], partitions[1]: start_offsets[1]} messages = {partitions[0]: [], partitions[1]: []} @@ -386,7 +386,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): producer = KeyedProducer(self.client, partitioner = RoundRobinPartitioner, async=True) - resp = producer.send(self.topic, self.key("key1"), self.msg("one")) + resp = producer.send_messages(self.topic, self.key("key1"), self.msg("one")) self.assertEqual(len(resp), 0) # wait for the server to report a new highwatermark From 01eb735af858f5bb1c67bf57cbdb0f0af665ba9b Mon Sep 17 00:00:00 2001 From: William Ting Date: Mon, 9 Mar 2015 14:23:26 -0700 Subject: [PATCH 089/250] Check vs None directly. --- kafka/consumer/base.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index aef898a..f53217f 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -50,7 +50,7 @@ class Consumer(object): self.client.load_metadata_for_topics(topic) self.offsets = {} - if not partitions: + if partitions is None: partitions = self.client.get_partition_ids_for_topic(topic) else: assert all(isinstance(x, numbers.Integral) for x in partitions) @@ -86,7 +86,7 @@ class Consumer(object): if self.group is None: raise ValueError('KafkaClient.group must not be None') - if not partitions: + if partitions is None: partitions = self.client.get_partition_ids_for_topic(self.topic) responses = self.client.send_offset_fetch_request( @@ -133,7 +133,7 @@ class Consumer(object): return reqs = [] - if not partitions: # commit all partitions + if partitions is None: # commit all partitions partitions = self.offsets.keys() for partition in partitions: @@ -194,7 +194,7 @@ class Consumer(object): Keyword Arguments: partitions (list): list of partitions to check for, default is to check all """ - if not partitions: + if partitions is None: partitions = self.offsets.keys() total = 0 From e298081bed97731b6c22ceec49806034aa249be5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 17 May 2015 18:47:41 -0700 Subject: [PATCH 090/250] Support optional partitions kwarg in MultiProcessConsumer --- kafka/consumer/multiprocess.py | 10 +++++++--- test/test_consumer.py | 13 +++++++++++-- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index cfe0ef6..891af04 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -98,6 +98,7 @@ class MultiProcessConsumer(Consumer): topic: the topic to consume Keyword Arguments: + partitions: An optional list of partitions to consume the data from auto_commit: default True. Whether or not to auto commit the offsets auto_commit_every_n: default 100. How many messages to consume before a commit @@ -114,16 +115,19 @@ class MultiProcessConsumer(Consumer): commit method on this class. A manual call to commit will also reset these triggers """ - def __init__(self, client, group, topic, auto_commit=True, + def __init__(self, client, group, topic, + partitions=None, + auto_commit=True, auto_commit_every_n=AUTO_COMMIT_MSG_COUNT, auto_commit_every_t=AUTO_COMMIT_INTERVAL, - num_procs=1, partitions_per_proc=0, + num_procs=1, + partitions_per_proc=0, **simple_consumer_options): # Initiate the base consumer class super(MultiProcessConsumer, self).__init__( client, group, topic, - partitions=None, + partitions=partitions, auto_commit=auto_commit, auto_commit_every_n=auto_commit_every_n, auto_commit_every_t=auto_commit_every_t) diff --git a/test/test_consumer.py b/test/test_consumer.py index 7b8f370..a3d09a8 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -1,8 +1,8 @@ -from mock import MagicMock +from mock import MagicMock, patch from . import unittest -from kafka import SimpleConsumer, KafkaConsumer +from kafka import SimpleConsumer, KafkaConsumer, MultiProcessConsumer from kafka.common import KafkaConfigurationError class TestKafkaConsumer(unittest.TestCase): @@ -13,3 +13,12 @@ class TestKafkaConsumer(unittest.TestCase): def test_broker_list_required(self): with self.assertRaises(KafkaConfigurationError): KafkaConsumer() + +class TestMultiProcessConsumer(unittest.TestCase): + def test_partition_list(self): + client = MagicMock() + partitions = (0,) + with patch.object(MultiProcessConsumer, 'fetch_last_known_offsets') as fetch_last_known_offsets: + consumer = MultiProcessConsumer(client, 'testing-group', 'testing-topic', partitions=partitions) + self.assertEqual(fetch_last_known_offsets.call_args[0], (partitions,) ) + self.assertEqual(client.get_partition_ids_for_topic.call_count, 0) # pylint: disable=no-member From b82f94f5a409a237222a6486a870751fa17da254 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 25 Feb 2015 15:04:55 +0300 Subject: [PATCH 091/250] Retries for async batching --- kafka/common.py | 12 ++++++++++-- kafka/producer/base.py | 31 ++++++++++++++++++++++++------- kafka/producer/keyed.py | 11 ++++++++--- kafka/producer/simple.py | 11 ++++++++--- 4 files changed, 50 insertions(+), 15 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 8207bec..b3380d7 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -6,6 +6,7 @@ from collections import namedtuple # Structs # ############### +<<<<<<< HEAD # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI MetadataRequest = namedtuple("MetadataRequest", ["topics"]) @@ -14,8 +15,15 @@ MetadataResponse = namedtuple("MetadataResponse", ["brokers", "topics"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ProduceAPI -ProduceRequest = namedtuple("ProduceRequest", - ["topic", "partition", "messages"]) +_ProduceRequest = namedtuple("ProduceRequest", + ["topic", "partition", "messages", "retries"]) + + +class ProduceRequest(_ProduceRequest): + def __new__(cls, topic, partition, messages, retries=0): + return super(ProduceRequest, cls).__new__( + cls, topic, partition, messages, retries) + ProduceResponse = namedtuple("ProduceResponse", ["topic", "partition", "error", "offset"]) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 4bd3de4..a5af3d6 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -24,22 +24,26 @@ log = logging.getLogger("kafka") BATCH_SEND_DEFAULT_INTERVAL = 20 BATCH_SEND_MSG_COUNT = 20 +BATCH_RETRY_BACKOFF_MS = 300 +BATCH_RETRIES_LIMIT = 5 STOP_ASYNC_PRODUCER = -1 def _send_upstream(queue, client, codec, batch_time, batch_size, - req_acks, ack_timeout, stop_event): + req_acks, ack_timeout, retry_backoff, retries_limit, stop_event): """ Listen on the queue for a specified number of messages or till a specified timeout and send them upstream to the brokers in one request """ stop = False + reqs = [] + client.reinit() while not stop_event.is_set(): timeout = batch_time - count = batch_size + count = batch_size - len(reqs) send_at = time.time() + timeout msgset = defaultdict(list) @@ -48,7 +52,6 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, while count > 0 and timeout >= 0: try: topic_partition, msg, key = queue.get(timeout=timeout) - except Empty: break @@ -63,7 +66,6 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, msgset[topic_partition].append((msg, key)) # Send collected requests upstream - reqs = [] for topic_partition, msg in msgset.items(): messages = create_message_set(msg, codec, key) req = ProduceRequest(topic_partition.topic, @@ -75,8 +77,19 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, client.send_produce_request(reqs, acks=req_acks, timeout=ack_timeout) - except Exception: - log.exception("Unable to send message") + except FailedPayloadsError as ex: + log.exception("Failed payloads count %s" % len(ex.message)) + if retries_limit is None: + reqs = ex.message + continue + for req in ex.message: + if retries_limit and req.retries < retries_limit: + reqs.append(req._replace(retries=req.retries+1)) + except Exception as ex: + log.exception("Unable to send message: %s" % type(ex)) + + if reqs and retry_backoff: + time.sleep(float(retry_backoff) / 1000) class Producer(object): @@ -111,7 +124,9 @@ class Producer(object): codec=None, batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, - batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL): + batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, + batch_retry_backoff_ms=BATCH_RETRY_BACKOFF_MS, + batch_retries_limit=BATCH_RETRIES_LIMIT): if batch_send: async = True @@ -148,6 +163,8 @@ class Producer(object): batch_send_every_n, self.req_acks, self.ack_timeout, + batch_retry_backoff_ms, + batch_retries_limit, self.thread_stop_event)) # Thread will die if main thread exits diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 333b6c0..aa569b3 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -7,7 +7,8 @@ from kafka.util import kafka_bytestring from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT + BATCH_SEND_MSG_COUNT, + BATCH_RETRY_BACKOFF_MS, BATCH_RETRIES_LIMIT ) log = logging.getLogger("kafka") @@ -37,7 +38,9 @@ class KeyedProducer(Producer): codec=None, batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, - batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL): + batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, + batch_retry_backoff_ms=BATCH_RETRY_BACKOFF_MS, + batch_retries_limit=BATCH_RETRIES_LIMIT): if not partitioner: partitioner = HashedPartitioner self.partitioner_class = partitioner @@ -46,7 +49,9 @@ class KeyedProducer(Producer): super(KeyedProducer, self).__init__(client, async, req_acks, ack_timeout, codec, batch_send, batch_send_every_n, - batch_send_every_t) + batch_send_every_t, + batch_retry_backoff_ms, + batch_retries_limit) def _next_partition(self, topic, key): if topic not in self.partitioners: diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index 2699cf2..7391be0 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -10,7 +10,8 @@ from six.moves import xrange from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT + BATCH_SEND_MSG_COUNT, + BATCH_RETRY_BACKOFF_MS, BATCH_RETRIES_LIMIT ) log = logging.getLogger("kafka") @@ -45,13 +46,17 @@ class SimpleProducer(Producer): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - random_start=True): + random_start=True, + batch_retry_backoff_ms=BATCH_RETRY_BACKOFF_MS, + batch_retries_limit=BATCH_RETRIES_LIMIT): self.partition_cycles = {} self.random_start = random_start super(SimpleProducer, self).__init__(client, async, req_acks, ack_timeout, codec, batch_send, batch_send_every_n, - batch_send_every_t) + batch_send_every_t, + batch_retry_backoff_ms, + batch_retries_limit) def _next_partition(self, topic): if topic not in self.partition_cycles: From 81d868869fa2f7ab980df5477d82654dc2598356 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 25 Feb 2015 15:18:16 +0300 Subject: [PATCH 092/250] Fixed base producer imports --- kafka/producer/base.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index a5af3d6..34b1d04 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -15,7 +15,8 @@ from threading import Thread, Event import six from kafka.common import ( - ProduceRequest, TopicAndPartition, UnsupportedCodecError + ProduceRequest, TopicAndPartition, + UnsupportedCodecError, FailedPayloadsError ) from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set from kafka.util import kafka_bytestring From 4b8288a578c0cee696ef9d0523f9cec32e8b1f05 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Feb 2015 12:40:44 +0300 Subject: [PATCH 093/250] Producer _send_upstream fixes, added tests for retries --- kafka/producer/base.py | 26 ++++++-- test/test_producer.py | 137 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 157 insertions(+), 6 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 34b1d04..505e31b 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -44,6 +44,9 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, while not stop_event.is_set(): timeout = batch_time + + # it's a simplification: we're comparing message sets and + # messages: each set can contain [1..batch_size] messages count = batch_size - len(reqs) send_at = time.time() + timeout msgset = defaultdict(list) @@ -74,6 +77,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, messages) reqs.append(req) + if not reqs: + continue + + reqs_to_retry = [] try: client.send_produce_request(reqs, acks=req_acks, @@ -81,15 +88,22 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, except FailedPayloadsError as ex: log.exception("Failed payloads count %s" % len(ex.message)) if retries_limit is None: - reqs = ex.message - continue - for req in ex.message: - if retries_limit and req.retries < retries_limit: - reqs.append(req._replace(retries=req.retries+1)) + # retry all failed messages until success + reqs_to_retry = ex.message + elif not retries_limit < 0: + # + for req in ex.message: + if retries_limit and req.retries < retries_limit: + updated_req = req._replace(retries=req.retries+1) + reqs_to_retry.append(updated_req) except Exception as ex: log.exception("Unable to send message: %s" % type(ex)) + finally: + reqs = [] - if reqs and retry_backoff: + if reqs_to_retry and retry_backoff: + reqs = reqs_to_retry + log.warning("%s requests will be retried next call." % len(reqs)) time.sleep(float(retry_backoff) / 1000) diff --git a/test/test_producer.py b/test/test_producer.py index f6b3d6a..eecc7a7 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -1,11 +1,22 @@ # -*- coding: utf-8 -*- +import time import logging from mock import MagicMock from . import unittest +from kafka.common import TopicAndPartition, FailedPayloadsError from kafka.producer.base import Producer +from kafka.producer.base import _send_upstream +from kafka.protocol import CODEC_NONE + +import threading +import multiprocessing as mp +try: + from queue import Empty +except ImportError: + from Queue import Empty class TestKafkaProducer(unittest.TestCase): @@ -40,3 +51,129 @@ class TestKafkaProducer(unittest.TestCase): topic = b"test-topic" producer.send_messages(topic, b'hi') assert client.send_produce_request.called + + +class TestKafkaProducerSendUpstream(unittest.TestCase): + + def setUp(self): + + # create a multiprocessing Value to store call counter + # (magicmock counters don't work with other processes) + self.send_calls_count = mp.Value('i', 0) + + def send_side_effect(*args, **kwargs): + self.send_calls_count.value += 1 + + self.client = MagicMock() + self.client.send_produce_request.side_effect = send_side_effect + self.queue = mp.Queue() + + def _run_process(self, retries_limit=3, sleep_timeout=1): + # run _send_upstream process with the queue + self.process = mp.Process( + target=_send_upstream, + args=(self.queue, self.client, CODEC_NONE, + 0.3, # batch time (seconds) + 3, # batch length + Producer.ACK_AFTER_LOCAL_WRITE, + Producer.DEFAULT_ACK_TIMEOUT, + 50, # retry backoff (ms) + retries_limit)) + self.process.daemon = True + self.process.start() + time.sleep(sleep_timeout) + self.process.terminate() + + def test_wo_retries(self): + + # lets create a queue and add 10 messages for 1 partition + for i in range(10): + self.queue.put((TopicAndPartition("test", 0), "msg %i", "key %i")) + + self._run_process() + + # the queue should be void at the end of the test + self.assertEqual(self.queue.empty(), True) + + # there should be 4 non-void cals: + # 3 batches of 3 msgs each + 1 batch of 1 message + self.assertEqual(self.send_calls_count.value, 4) + + def test_first_send_failed(self): + + # lets create a queue and add 10 messages for 10 different partitions + # to show how retries should work ideally + for i in range(10): + self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + + flag = mp.Value('c', 'f') + def send_side_effect(reqs, *args, **kwargs): + self.send_calls_count.value += 1 + if flag.value == 'f': + flag.value = 't' + raise FailedPayloadsError(reqs) + + self.client.send_produce_request.side_effect = send_side_effect + + self._run_process(2) + + # the queue should be void at the end of the test + self.assertEqual(self.queue.empty(), True) + + # there should be 5 non-void cals: 1st failed batch of 3 msgs + # + 3 batches of 3 msgs each + 1 batch of 1 msg = 1 + 3 + 1 = 5 + self.assertEqual(self.send_calls_count.value, 5) + + def test_with_limited_retries(self): + + # lets create a queue and add 10 messages for 10 different partitions + # to show how retries should work ideally + for i in range(10): + self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + + def send_side_effect(reqs, *args, **kwargs): + self.send_calls_count.value += 1 + raise FailedPayloadsError(reqs) + + self.client.send_produce_request.side_effect = send_side_effect + + self._run_process(3, 2) + + # the queue should be void at the end of the test + self.assertEqual(self.queue.empty(), True) + + # there should be 16 non-void cals: + # 3 initial batches of 3 msgs each + 1 initial batch of 1 msg + + # 3 retries of the batches above = 4 + 3 * 4 = 16, all failed + self.assertEqual(self.send_calls_count.value, 16) + + + def test_with_unlimited_retries(self): + + # lets create a queue and add 10 messages for 10 different partitions + # to show how retries should work ideally + for i in range(10): + self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + + def send_side_effect(reqs, *args, **kwargs): + self.send_calls_count.value += 1 + raise FailedPayloadsError(reqs) + + self.client.send_produce_request.side_effect = send_side_effect + + self._run_process(None) + + # the queue should have 7 elements + # 3 batches of 1 msg each were retried all this time + self.assertEqual(self.queue.empty(), False) + left = 0 + for i in range(10): + try: + self.queue.get(timeout=0.01) + left += 1 + except Empty: + break + self.assertEqual(left, 7) + + # 1s / 50ms of backoff = 20 times + self.assertEqual(self.send_calls_count.value, 20) From a9324f343e97ae3ceaa2acd480764818bb2b171e Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Feb 2015 15:10:55 +0300 Subject: [PATCH 094/250] Fixed compatible issues with tests --- kafka/producer/base.py | 8 +++++--- test/test_producer.py | 23 +++++++++++------------ 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 505e31b..87d923a 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -86,13 +86,15 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, acks=req_acks, timeout=ack_timeout) except FailedPayloadsError as ex: - log.exception("Failed payloads count %s" % len(ex.message)) + failed_reqs = ex.args[0] + log.exception("Failed payloads count %s" % len(failed_reqs)) + if retries_limit is None: # retry all failed messages until success - reqs_to_retry = ex.message + reqs_to_retry = failed_reqs elif not retries_limit < 0: # - for req in ex.message: + for req in failed_reqs: if retries_limit and req.retries < retries_limit: updated_req = req._replace(retries=req.retries+1) reqs_to_retry.append(updated_req) diff --git a/test/test_producer.py b/test/test_producer.py index eecc7a7..51a74b5 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -106,11 +106,11 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): for i in range(10): self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) - flag = mp.Value('c', 'f') + is_first_time = mp.Value('b', True) def send_side_effect(reqs, *args, **kwargs): self.send_calls_count.value += 1 - if flag.value == 'f': - flag.value = 't' + if is_first_time.value: + is_first_time.value = False raise FailedPayloadsError(reqs) self.client.send_produce_request.side_effect = send_side_effect @@ -166,14 +166,13 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # the queue should have 7 elements # 3 batches of 1 msg each were retried all this time self.assertEqual(self.queue.empty(), False) - left = 0 - for i in range(10): - try: + try: + for i in range(7): self.queue.get(timeout=0.01) - left += 1 - except Empty: - break - self.assertEqual(left, 7) + except Empty: + self.fail("Should be 7 elems in the queue") + self.assertEqual(self.queue.empty(), True) - # 1s / 50ms of backoff = 20 times - self.assertEqual(self.send_calls_count.value, 20) + # 1s / 50ms of backoff = 20 times max + self.assertTrue(self.send_calls_count.value > 10) + self.assertTrue(self.send_calls_count.value <= 20) From 566e408da0aa7fb1bb74fae96231c94cc4a12e37 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Feb 2015 17:24:52 +0300 Subject: [PATCH 095/250] Fix: check failed reqs to retry only for positive limit --- kafka/producer/base.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 87d923a..8544d73 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -89,11 +89,11 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, failed_reqs = ex.args[0] log.exception("Failed payloads count %s" % len(failed_reqs)) + # if no limit, retry all failed messages until success if retries_limit is None: - # retry all failed messages until success reqs_to_retry = failed_reqs - elif not retries_limit < 0: - # + # makes sense to check failed reqs only if we have a limit > 0 + elif retries_limit > 0: for req in failed_reqs: if retries_limit and req.retries < retries_limit: updated_req = req._replace(retries=req.retries+1) From b0a04595c6aee7f6fcaa8927fcdfcd9a04a9b7d3 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Thu, 26 Feb 2015 17:26:56 +0300 Subject: [PATCH 096/250] Returned default behaviour with no retries --- kafka/producer/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 8544d73..44ffdf4 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -26,7 +26,7 @@ log = logging.getLogger("kafka") BATCH_SEND_DEFAULT_INTERVAL = 20 BATCH_SEND_MSG_COUNT = 20 BATCH_RETRY_BACKOFF_MS = 300 -BATCH_RETRIES_LIMIT = 5 +BATCH_RETRIES_LIMIT = 0 STOP_ASYNC_PRODUCER = -1 From 5e8dc6dcf55890a4e3a214a943ecc655faed3ecc Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 24 Mar 2015 22:21:13 +0300 Subject: [PATCH 097/250] Fixed tests and other issues after rebase --- kafka/common.py | 1 - kafka/producer/base.py | 2 +- test/test_producer.py | 49 ++++++++++++++++-------------------------- 3 files changed, 20 insertions(+), 32 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index b3380d7..5c2b788 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -6,7 +6,6 @@ from collections import namedtuple # Structs # ############### -<<<<<<< HEAD # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI MetadataRequest = namedtuple("MetadataRequest", ["topics"]) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 44ffdf4..9bfe98b 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -86,7 +86,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, acks=req_acks, timeout=ack_timeout) except FailedPayloadsError as ex: - failed_reqs = ex.args[0] + failed_reqs = ex.failed_payloads log.exception("Failed payloads count %s" % len(failed_reqs)) # if no limit, retry all failed messages until success diff --git a/test/test_producer.py b/test/test_producer.py index 51a74b5..cc58fe4 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -12,11 +12,10 @@ from kafka.producer.base import _send_upstream from kafka.protocol import CODEC_NONE import threading -import multiprocessing as mp try: - from queue import Empty + from queue import Empty, Queue except ImportError: - from Queue import Empty + from Queue import Empty, Queue class TestKafkaProducer(unittest.TestCase): @@ -56,21 +55,13 @@ class TestKafkaProducer(unittest.TestCase): class TestKafkaProducerSendUpstream(unittest.TestCase): def setUp(self): - - # create a multiprocessing Value to store call counter - # (magicmock counters don't work with other processes) - self.send_calls_count = mp.Value('i', 0) - - def send_side_effect(*args, **kwargs): - self.send_calls_count.value += 1 - self.client = MagicMock() - self.client.send_produce_request.side_effect = send_side_effect - self.queue = mp.Queue() + self.queue = Queue() def _run_process(self, retries_limit=3, sleep_timeout=1): # run _send_upstream process with the queue - self.process = mp.Process( + stop_event = threading.Event() + self.thread = threading.Thread( target=_send_upstream, args=(self.queue, self.client, CODEC_NONE, 0.3, # batch time (seconds) @@ -78,11 +69,12 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): Producer.ACK_AFTER_LOCAL_WRITE, Producer.DEFAULT_ACK_TIMEOUT, 50, # retry backoff (ms) - retries_limit)) - self.process.daemon = True - self.process.start() + retries_limit, + stop_event)) + self.thread.daemon = True + self.thread.start() time.sleep(sleep_timeout) - self.process.terminate() + stop_event.set() def test_wo_retries(self): @@ -97,7 +89,8 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # there should be 4 non-void cals: # 3 batches of 3 msgs each + 1 batch of 1 message - self.assertEqual(self.send_calls_count.value, 4) + self.assertEqual(self.client.send_produce_request.call_count, 4) + def test_first_send_failed(self): @@ -106,11 +99,10 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): for i in range(10): self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) - is_first_time = mp.Value('b', True) + self.client.is_first_time = True def send_side_effect(reqs, *args, **kwargs): - self.send_calls_count.value += 1 - if is_first_time.value: - is_first_time.value = False + if self.client.is_first_time: + self.client.is_first_time = False raise FailedPayloadsError(reqs) self.client.send_produce_request.side_effect = send_side_effect @@ -122,7 +114,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # there should be 5 non-void cals: 1st failed batch of 3 msgs # + 3 batches of 3 msgs each + 1 batch of 1 msg = 1 + 3 + 1 = 5 - self.assertEqual(self.send_calls_count.value, 5) + self.assertEqual(self.client.send_produce_request.call_count, 5) def test_with_limited_retries(self): @@ -132,7 +124,6 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) def send_side_effect(reqs, *args, **kwargs): - self.send_calls_count.value += 1 raise FailedPayloadsError(reqs) self.client.send_produce_request.side_effect = send_side_effect @@ -145,8 +136,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # there should be 16 non-void cals: # 3 initial batches of 3 msgs each + 1 initial batch of 1 msg + # 3 retries of the batches above = 4 + 3 * 4 = 16, all failed - self.assertEqual(self.send_calls_count.value, 16) - + self.assertEqual(self.client.send_produce_request.call_count, 16) def test_with_unlimited_retries(self): @@ -156,7 +146,6 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) def send_side_effect(reqs, *args, **kwargs): - self.send_calls_count.value += 1 raise FailedPayloadsError(reqs) self.client.send_produce_request.side_effect = send_side_effect @@ -174,5 +163,5 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.assertEqual(self.queue.empty(), True) # 1s / 50ms of backoff = 20 times max - self.assertTrue(self.send_calls_count.value > 10) - self.assertTrue(self.send_calls_count.value <= 20) + calls = self.client.send_produce_request.call_count + self.assertTrue(calls > 10 & calls <= 20) From 09c1c8b117a3c8217ca36c27eafb22999c20ef93 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 21 Apr 2015 13:30:46 +0300 Subject: [PATCH 098/250] Improved retry logic --- kafka/common.py | 3 ++ kafka/producer/base.py | 88 ++++++++++++++++++++++++++++++---------- kafka/producer/keyed.py | 6 +-- kafka/producer/simple.py | 9 ++-- test/test_producer.py | 6 +-- 5 files changed, 77 insertions(+), 35 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 5c2b788..cbb4013 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -78,6 +78,9 @@ TopicAndPartition = namedtuple("TopicAndPartition", KafkaMessage = namedtuple("KafkaMessage", ["topic", "partition", "offset", "key", "value"]) +RetryOptions = namedtuple("RetryOptions", + ["limit", "backoff_ms", "retry_on_timeouts"]) + ################# # Exceptions # diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 9bfe98b..ebeb82d 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -16,7 +16,10 @@ import six from kafka.common import ( ProduceRequest, TopicAndPartition, - UnsupportedCodecError, FailedPayloadsError + UnsupportedCodecError, FailedPayloadsError, RetryOptions, + RequestTimedOutError, KafkaUnavailableError, LeaderNotAvailableError, + UnknownTopicOrPartitionError, NotLeaderForPartitionError, ConnectionError, + InvalidMessageError, MessageSizeTooLargeError ) from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set from kafka.util import kafka_bytestring @@ -25,20 +28,19 @@ log = logging.getLogger("kafka") BATCH_SEND_DEFAULT_INTERVAL = 20 BATCH_SEND_MSG_COUNT = 20 -BATCH_RETRY_BACKOFF_MS = 300 -BATCH_RETRIES_LIMIT = 0 +BATCH_RETRY_OPTIONS = RetryOptions( + limit=0, backoff_ms=300, retry_on_timeouts=True) STOP_ASYNC_PRODUCER = -1 def _send_upstream(queue, client, codec, batch_time, batch_size, - req_acks, ack_timeout, retry_backoff, retries_limit, stop_event): + req_acks, ack_timeout, retry_options, stop_event): """ Listen on the queue for a specified number of messages or till a specified timeout and send them upstream to the brokers in one request """ - stop = False reqs = [] client.reinit() @@ -85,28 +87,71 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, client.send_produce_request(reqs, acks=req_acks, timeout=ack_timeout) - except FailedPayloadsError as ex: - failed_reqs = ex.failed_payloads - log.exception("Failed payloads count %s" % len(failed_reqs)) - # if no limit, retry all failed messages until success - if retries_limit is None: - reqs_to_retry = failed_reqs - # makes sense to check failed reqs only if we have a limit > 0 - elif retries_limit > 0: - for req in failed_reqs: - if retries_limit and req.retries < retries_limit: - updated_req = req._replace(retries=req.retries+1) - reqs_to_retry.append(updated_req) + except RequestTimedOutError as ex: + # should retry only if user is fine with duplicates + if retry_options.retry_on_timeouts: + reqs_to_retry = reqs + + except KafkaUnavailableError as ex: + # backoff + retry + do_backoff(retry_options) + reqs_to_retry = get_requests_for_retry(reqs, retry_options) + + except (NotLeaderForPartitionError, UnknownTopicOrPartitionError) as ex: + # refresh + retry + client.load_metadata_for_topics() + reqs_to_retry = get_requests_for_retry(reqs, retry_options) + + except (LeaderNotAvailableError, ConnectionError) as ex: + # backoff + refresh + retry + do_backoff(retry_options) + client.load_metadata_for_topics() + reqs_to_retry = get_requests_for_retry(reqs, retry_options) + + except FailedPayloadsError as ex: + # retry only failed messages with backoff + failed_reqs = ex.failed_payloads + do_backoff(retry_options) + reqs_to_retry = get_requests_for_retry(failed_reqs, retry_options) + + except (InvalidMessageError, MessageSizeTooLargeError) as ex: + # "bad" messages, doesn't make sense to retry + log.exception("Message error when sending: %s" % type(ex)) + except Exception as ex: log.exception("Unable to send message: %s" % type(ex)) + finally: reqs = [] - if reqs_to_retry and retry_backoff: + if reqs_to_retry: reqs = reqs_to_retry - log.warning("%s requests will be retried next call." % len(reqs)) - time.sleep(float(retry_backoff) / 1000) + + +def get_requests_for_retry(requests, retry_options): + log.exception("Failed payloads count %s" % len(requests)) + + # if no limit, retry all failed messages until success + if retry_options.limit is None: + return requests + + # makes sense to check failed reqs only if we have a limit > 0 + reqs_to_retry = [] + if retry_options.limit > 0: + for req in requests: + if req.retries < retry_options.limit: + updated_req = req._replace(retries=req.retries+1) + reqs_to_retry.append(updated_req) + + return reqs_to_retry + + +def do_backoff(retry_options): + if retry_options.backoff_ms: + log.warning("Doing backoff for %s(ms)." % retry_options.backoff_ms) + time.sleep(float(retry_options.backoff_ms) / 1000) + class Producer(object): @@ -142,8 +187,7 @@ class Producer(object): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - batch_retry_backoff_ms=BATCH_RETRY_BACKOFF_MS, - batch_retries_limit=BATCH_RETRIES_LIMIT): + batch_retry_options=BATCH_RETRY_OPTIONS): if batch_send: async = True diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index aa569b3..d11db52 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -7,8 +7,7 @@ from kafka.util import kafka_bytestring from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, - BATCH_RETRY_BACKOFF_MS, BATCH_RETRIES_LIMIT + BATCH_SEND_MSG_COUNT, BATCH_RETRY_OPTIONS ) log = logging.getLogger("kafka") @@ -39,8 +38,7 @@ class KeyedProducer(Producer): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - batch_retry_backoff_ms=BATCH_RETRY_BACKOFF_MS, - batch_retries_limit=BATCH_RETRIES_LIMIT): + batch_retry_options=BATCH_RETRY_OPTIONS): if not partitioner: partitioner = HashedPartitioner self.partitioner_class = partitioner diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index 7391be0..b869683 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -10,8 +10,7 @@ from six.moves import xrange from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, - BATCH_RETRY_BACKOFF_MS, BATCH_RETRIES_LIMIT + BATCH_SEND_MSG_COUNT, BATCH_RETRY_OPTIONS ) log = logging.getLogger("kafka") @@ -47,16 +46,14 @@ class SimpleProducer(Producer): batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, random_start=True, - batch_retry_backoff_ms=BATCH_RETRY_BACKOFF_MS, - batch_retries_limit=BATCH_RETRIES_LIMIT): + batch_retry_options=BATCH_RETRY_OPTIONS): self.partition_cycles = {} self.random_start = random_start super(SimpleProducer, self).__init__(client, async, req_acks, ack_timeout, codec, batch_send, batch_send_every_n, batch_send_every_t, - batch_retry_backoff_ms, - batch_retries_limit) + batch_retry_options) def _next_partition(self, topic): if topic not in self.partition_cycles: diff --git a/test/test_producer.py b/test/test_producer.py index cc58fe4..c9bdc47 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -6,7 +6,7 @@ import logging from mock import MagicMock from . import unittest -from kafka.common import TopicAndPartition, FailedPayloadsError +from kafka.common import TopicAndPartition, FailedPayloadsError, RetryOptions from kafka.producer.base import Producer from kafka.producer.base import _send_upstream from kafka.protocol import CODEC_NONE @@ -68,8 +68,8 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): 3, # batch length Producer.ACK_AFTER_LOCAL_WRITE, Producer.DEFAULT_ACK_TIMEOUT, - 50, # retry backoff (ms) - retries_limit, + RetryOptions(limit=retries_limit, backoff_ms=50, + retry_on_timeouts=True), stop_event)) self.thread.daemon = True self.thread.start() From 0e0f794802076db34e8e4dc597c38237e88f4b34 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 21 Apr 2015 13:37:16 +0300 Subject: [PATCH 099/250] Arg fixes for base/keyed producers --- kafka/producer/base.py | 3 +-- kafka/producer/keyed.py | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index ebeb82d..1d5e045 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -224,8 +224,7 @@ class Producer(object): batch_send_every_n, self.req_acks, self.ack_timeout, - batch_retry_backoff_ms, - batch_retries_limit, + batch_retry_options, self.thread_stop_event)) # Thread will die if main thread exits diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index d11db52..7bcc629 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -48,8 +48,7 @@ class KeyedProducer(Producer): ack_timeout, codec, batch_send, batch_send_every_n, batch_send_every_t, - batch_retry_backoff_ms, - batch_retries_limit) + batch_retry_options) def _next_partition(self, topic, key): if topic not in self.partitioners: From b31114520a15477da7ad660765a0240b6f348944 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 21 Apr 2015 17:47:55 +0300 Subject: [PATCH 100/250] Clean and simplify retry logic --- kafka/common.py | 17 +++++++++ kafka/producer/base.py | 82 ++++++++++++++---------------------------- test/test_producer.py | 8 +++-- 3 files changed, 49 insertions(+), 58 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index cbb4013..50f8a77 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -82,6 +82,8 @@ RetryOptions = namedtuple("RetryOptions", ["limit", "backoff_ms", "retry_on_timeouts"]) + + ################# # Exceptions # ################# @@ -228,3 +230,18 @@ def check_error(response): if response.error: error_class = kafka_errors.get(response.error, UnknownError) raise error_class(response) + + +RETRY_BACKOFF_ERROR_TYPES = ( + KafkaUnavailableError, LeaderNotAvailableError, + ConnectionError, FailedPayloadsError +) + + +RETRY_REFRESH_ERROR_TYPES = ( + NotLeaderForPartitionError, UnknownTopicOrPartitionError, + LeaderNotAvailableError, ConnectionError +) + + +RETRY_ERROR_TYPES = list(set(RETRY_BACKOFF_ERROR_TYPES + RETRY_REFRESH_ERROR_TYPES)) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 1d5e045..a989e3f 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -15,12 +15,12 @@ from threading import Thread, Event import six from kafka.common import ( - ProduceRequest, TopicAndPartition, - UnsupportedCodecError, FailedPayloadsError, RetryOptions, - RequestTimedOutError, KafkaUnavailableError, LeaderNotAvailableError, - UnknownTopicOrPartitionError, NotLeaderForPartitionError, ConnectionError, - InvalidMessageError, MessageSizeTooLargeError + ProduceRequest, TopicAndPartition, RetryOptions, + UnsupportedCodecError, FailedPayloadsError, RequestTimedOutError ) +from kafka.common import ( + RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES) + from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set from kafka.util import kafka_bytestring @@ -88,36 +88,33 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, acks=req_acks, timeout=ack_timeout) - except RequestTimedOutError as ex: - # should retry only if user is fine with duplicates - if retry_options.retry_on_timeouts: - reqs_to_retry = reqs + except tuple(RETRY_ERROR_TYPES) as ex: - except KafkaUnavailableError as ex: - # backoff + retry - do_backoff(retry_options) - reqs_to_retry = get_requests_for_retry(reqs, retry_options) + # by default, retry all sent messages + reqs_to_retry = reqs - except (NotLeaderForPartitionError, UnknownTopicOrPartitionError) as ex: - # refresh + retry - client.load_metadata_for_topics() - reqs_to_retry = get_requests_for_retry(reqs, retry_options) + if type(ex) == FailedPayloadsError: + reqs_to_retry = ex.failed_payloads - except (LeaderNotAvailableError, ConnectionError) as ex: - # backoff + refresh + retry - do_backoff(retry_options) - client.load_metadata_for_topics() - reqs_to_retry = get_requests_for_retry(reqs, retry_options) + elif (type(ex) == RequestTimedOutError and + not retry_options.retry_on_timeouts): + reqs_to_retry = [] - except FailedPayloadsError as ex: - # retry only failed messages with backoff - failed_reqs = ex.failed_payloads - do_backoff(retry_options) - reqs_to_retry = get_requests_for_retry(failed_reqs, retry_options) + # filter reqs_to_retry if there's a retry limit + if retry_options.limit and retry_options.limit > 0: + reqs_to_retry = [req._replace(retries=req.retries+1) + for req in reqs_to_retry + if req.retries < retry_options.limit] - except (InvalidMessageError, MessageSizeTooLargeError) as ex: - # "bad" messages, doesn't make sense to retry - log.exception("Message error when sending: %s" % type(ex)) + # doing backoff before next retry + if (reqs_to_retry and type(ex) in RETRY_BACKOFF_ERROR_TYPES + and retry_options.backoff_ms): + log.warning("Doing backoff for %s(ms)." % retry_options.backoff_ms) + time.sleep(float(retry_options.backoff_ms) / 1000) + + # refresh topic metadata before next retry + if reqs_to_retry and type(ex) in RETRY_REFRESH_ERROR_TYPES: + client.load_metadata_for_topics() except Exception as ex: log.exception("Unable to send message: %s" % type(ex)) @@ -129,31 +126,6 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, reqs = reqs_to_retry -def get_requests_for_retry(requests, retry_options): - log.exception("Failed payloads count %s" % len(requests)) - - # if no limit, retry all failed messages until success - if retry_options.limit is None: - return requests - - # makes sense to check failed reqs only if we have a limit > 0 - reqs_to_retry = [] - if retry_options.limit > 0: - for req in requests: - if req.retries < retry_options.limit: - updated_req = req._replace(retries=req.retries+1) - reqs_to_retry.append(updated_req) - - return reqs_to_retry - - -def do_backoff(retry_options): - if retry_options.backoff_ms: - log.warning("Doing backoff for %s(ms)." % retry_options.backoff_ms) - time.sleep(float(retry_options.backoff_ms) / 1000) - - - class Producer(object): """ Base class to be used by producers diff --git a/test/test_producer.py b/test/test_producer.py index c9bdc47..c0dc873 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -61,6 +61,9 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def _run_process(self, retries_limit=3, sleep_timeout=1): # run _send_upstream process with the queue stop_event = threading.Event() + retry_options = RetryOptions(limit=retries_limit, + backoff_ms=50, + retry_on_timeouts=False) self.thread = threading.Thread( target=_send_upstream, args=(self.queue, self.client, CODEC_NONE, @@ -68,8 +71,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): 3, # batch length Producer.ACK_AFTER_LOCAL_WRITE, Producer.DEFAULT_ACK_TIMEOUT, - RetryOptions(limit=retries_limit, backoff_ms=50, - retry_on_timeouts=True), + retry_options, stop_event)) self.thread.daemon = True self.thread.start() @@ -121,7 +123,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # lets create a queue and add 10 messages for 10 different partitions # to show how retries should work ideally for i in range(10): - self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + self.queue.put((TopicAndPartition("test", i), "msg %i" % i, "key %i" % i)) def send_side_effect(reqs, *args, **kwargs): raise FailedPayloadsError(reqs) From c165f17338c0a9260a91b816f73e5ce4ff7ed359 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 21 Apr 2015 17:51:39 +0300 Subject: [PATCH 101/250] Disable retry on timeouts by default (old behaviour) --- kafka/producer/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index a989e3f..331c71c 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -29,7 +29,7 @@ log = logging.getLogger("kafka") BATCH_SEND_DEFAULT_INTERVAL = 20 BATCH_SEND_MSG_COUNT = 20 BATCH_RETRY_OPTIONS = RetryOptions( - limit=0, backoff_ms=300, retry_on_timeouts=True) + limit=0, backoff_ms=300, retry_on_timeouts=False) STOP_ASYNC_PRODUCER = -1 From cf363089617de2d0b18cb83eba1e61adbc5d0144 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D1=81=D0=BA=D0=B0=D0=BD=D0=B4=D0=B0=D1=80=D0=BE?= =?UTF-8?q?=D0=B2=20=D0=AD=D0=B4=D1=83=D0=B0=D1=80=D0=B4?= Date: Fri, 23 Jan 2015 12:56:42 +0300 Subject: [PATCH 102/250] add producer send batch queue overfilled test --- kafka/producer/base.py | 8 +++++++- test/test_producer.py | 18 +++++++++++++++++- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 331c71c..a0d9ac1 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -249,7 +249,13 @@ class Producer(object): if self.async: for m in msg: - self.queue.put((TopicAndPartition(topic, partition), m, key)) + try: + item = (TopicAndPartition(topic, partition), m, key) + self.queue.put_nowait(item) + except Full: + raise BatchQueueOverfilledError( + 'Producer batch send queue overfilled. ' + 'Current queue size %d.' % self.queue.qsize()) resp = [] else: messages = create_message_set([(m, key) for m in msg], self.codec, key) diff --git a/test/test_producer.py b/test/test_producer.py index c0dc873..b57dfd8 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -3,10 +3,11 @@ import time import logging -from mock import MagicMock +from mock import MagicMock, patch from . import unittest from kafka.common import TopicAndPartition, FailedPayloadsError, RetryOptions +from kafka.common import BatchQueueOverfilledError from kafka.producer.base import Producer from kafka.producer.base import _send_upstream from kafka.protocol import CODEC_NONE @@ -51,6 +52,21 @@ class TestKafkaProducer(unittest.TestCase): producer.send_messages(topic, b'hi') assert client.send_produce_request.called + @patch('kafka.producer.base.Process') + def test_producer_batch_send_queue_overfilled(self, process_mock): + queue_size = 2 + producer = Producer(MagicMock(), batch_send=True, + batch_send_queue_maxsize=queue_size) + + topic = b'test-topic' + partition = 0 + + message = b'test-message' + with self.assertRaises(BatchQueueOverfilledError): + message_list = [message] * (queue_size + 1) + producer.send_messages(topic, partition, *message_list) + + class TestKafkaProducerSendUpstream(unittest.TestCase): From f41e5f3e4befda52a20f072f85b807d77361e64d Mon Sep 17 00:00:00 2001 From: Eduard Iskandarov Date: Sat, 24 Jan 2015 00:30:50 +0300 Subject: [PATCH 103/250] async queue: refactored code; add one more test --- kafka/producer/base.py | 9 +++++++-- test/test_producer.py | 20 +++++++++++++++++--- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index a0d9ac1..0e005c5 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -31,6 +31,8 @@ BATCH_SEND_MSG_COUNT = 20 BATCH_RETRY_OPTIONS = RetryOptions( limit=0, backoff_ms=300, retry_on_timeouts=False) +# unlimited +ASYNC_QUEUE_MAXSIZE = 0 STOP_ASYNC_PRODUCER = -1 @@ -159,12 +161,14 @@ class Producer(object): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - batch_retry_options=BATCH_RETRY_OPTIONS): + batch_retry_options=BATCH_RETRY_OPTIONS, + async_queue_maxsize=ASYNC_QUEUE_MAXSIZE): if batch_send: async = True assert batch_send_every_n > 0 assert batch_send_every_t > 0 + assert async_queue_maxsize >= 0 else: batch_send_every_n = 1 batch_send_every_t = 3600 @@ -186,7 +190,8 @@ class Producer(object): log.warning("async producer does not guarantee message delivery!") log.warning("Current implementation does not retry Failed messages") log.warning("Use at your own risk! (or help improve with a PR!)") - self.queue = Queue() # Messages are sent through this queue + # Messages are sent through this queue + self.queue = Queue(async_queue_maxsize) self.thread_stop_event = Event() self.thread = Thread(target=_send_upstream, args=(self.queue, diff --git a/test/test_producer.py b/test/test_producer.py index b57dfd8..627178d 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -53,15 +53,29 @@ class TestKafkaProducer(unittest.TestCase): assert client.send_produce_request.called @patch('kafka.producer.base.Process') - def test_producer_batch_send_queue_overfilled(self, process_mock): + def test_producer_async_queue_overfilled_batch_send(self, process_mock): queue_size = 2 producer = Producer(MagicMock(), batch_send=True, - batch_send_queue_maxsize=queue_size) + async_queue_maxsize=queue_size) topic = b'test-topic' partition = 0 - message = b'test-message' + + with self.assertRaises(BatchQueueOverfilledError): + message_list = [message] * (queue_size + 1) + producer.send_messages(topic, partition, *message_list) + + @patch('kafka.producer.base.Process') + def test_producer_async_queue_overfilled(self, process_mock): + queue_size = 2 + producer = Producer(MagicMock(), async=True, + async_queue_maxsize=queue_size) + + topic = b'test-topic' + partition = 0 + message = b'test-message' + with self.assertRaises(BatchQueueOverfilledError): message_list = [message] * (queue_size + 1) producer.send_messages(topic, partition, *message_list) From 948e046b5443e0f38f6062e13153b57d29915a68 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 21 Apr 2015 19:44:27 +0300 Subject: [PATCH 104/250] Fix small issues with names/tests --- kafka/common.py | 4 ++++ kafka/producer/base.py | 16 +++++++--------- test/test_producer.py | 12 +++++------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 50f8a77..0e769e4 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -217,6 +217,10 @@ class KafkaConfigurationError(KafkaError): pass +class AsyncProducerQueueFull(KafkaError): + pass + + def _iter_broker_errors(): for name, obj in inspect.getmembers(sys.modules[__name__]): if inspect.isclass(obj) and issubclass(obj, BrokerResponseError) and obj != BrokerResponseError: diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 0e005c5..3f0431c 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -5,9 +5,9 @@ import logging import time try: - from queue import Empty, Queue + from queue import Empty, Full, Queue except ImportError: - from Queue import Empty, Queue + from Queue import Empty, Full, Queue from collections import defaultdict from threading import Thread, Event @@ -16,7 +16,8 @@ import six from kafka.common import ( ProduceRequest, TopicAndPartition, RetryOptions, - UnsupportedCodecError, FailedPayloadsError, RequestTimedOutError + UnsupportedCodecError, FailedPayloadsError, + RequestTimedOutError, AsyncProducerQueueFull ) from kafka.common import ( RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES) @@ -187,11 +188,8 @@ class Producer(object): self.codec = codec if self.async: - log.warning("async producer does not guarantee message delivery!") - log.warning("Current implementation does not retry Failed messages") - log.warning("Use at your own risk! (or help improve with a PR!)") # Messages are sent through this queue - self.queue = Queue(async_queue_maxsize) + self.queue = Queue(async_queue_maxsize) self.thread_stop_event = Event() self.thread = Thread(target=_send_upstream, args=(self.queue, @@ -258,8 +256,8 @@ class Producer(object): item = (TopicAndPartition(topic, partition), m, key) self.queue.put_nowait(item) except Full: - raise BatchQueueOverfilledError( - 'Producer batch send queue overfilled. ' + raise AsyncProducerQueueFull( + 'Producer async queue overfilled. ' 'Current queue size %d.' % self.queue.qsize()) resp = [] else: diff --git a/test/test_producer.py b/test/test_producer.py index 627178d..de012b9 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -7,7 +7,7 @@ from mock import MagicMock, patch from . import unittest from kafka.common import TopicAndPartition, FailedPayloadsError, RetryOptions -from kafka.common import BatchQueueOverfilledError +from kafka.common import AsyncProducerQueueFull from kafka.producer.base import Producer from kafka.producer.base import _send_upstream from kafka.protocol import CODEC_NONE @@ -52,8 +52,7 @@ class TestKafkaProducer(unittest.TestCase): producer.send_messages(topic, b'hi') assert client.send_produce_request.called - @patch('kafka.producer.base.Process') - def test_producer_async_queue_overfilled_batch_send(self, process_mock): + def test_producer_async_queue_overfilled_batch_send(self): queue_size = 2 producer = Producer(MagicMock(), batch_send=True, async_queue_maxsize=queue_size) @@ -62,12 +61,11 @@ class TestKafkaProducer(unittest.TestCase): partition = 0 message = b'test-message' - with self.assertRaises(BatchQueueOverfilledError): + with self.assertRaises(AsyncProducerQueueFull): message_list = [message] * (queue_size + 1) producer.send_messages(topic, partition, *message_list) - @patch('kafka.producer.base.Process') - def test_producer_async_queue_overfilled(self, process_mock): + def test_producer_async_queue_overfilled(self): queue_size = 2 producer = Producer(MagicMock(), async=True, async_queue_maxsize=queue_size) @@ -76,7 +74,7 @@ class TestKafkaProducer(unittest.TestCase): partition = 0 message = b'test-message' - with self.assertRaises(BatchQueueOverfilledError): + with self.assertRaises(AsyncProducerQueueFull): message_list = [message] * (queue_size + 1) producer.send_messages(topic, partition, *message_list) From 7da48f62975385e15e4115df70986688837058b8 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 22 Apr 2015 11:21:19 +0300 Subject: [PATCH 105/250] Simplification of retry logic --- kafka/common.py | 2 +- kafka/producer/base.py | 58 ++++++++++++++++++++---------------------- 2 files changed, 29 insertions(+), 31 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 0e769e4..e327d02 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -248,4 +248,4 @@ RETRY_REFRESH_ERROR_TYPES = ( ) -RETRY_ERROR_TYPES = list(set(RETRY_BACKOFF_ERROR_TYPES + RETRY_REFRESH_ERROR_TYPES)) +RETRY_ERROR_TYPES = RETRY_BACKOFF_ERROR_TYPES + RETRY_REFRESH_ERROR_TYPES diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 3f0431c..fffea94 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -85,48 +85,46 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if not reqs: continue - reqs_to_retry = [] + reqs_to_retry, error_type = [], None try: client.send_produce_request(reqs, acks=req_acks, timeout=ack_timeout) - except tuple(RETRY_ERROR_TYPES) as ex: + except FailedPayloadsError as ex: + error_type = FailedPayloadsError + reqs_to_retry = ex.failed_payloads - # by default, retry all sent messages - reqs_to_retry = reqs - - if type(ex) == FailedPayloadsError: - reqs_to_retry = ex.failed_payloads - - elif (type(ex) == RequestTimedOutError and - not retry_options.retry_on_timeouts): - reqs_to_retry = [] - - # filter reqs_to_retry if there's a retry limit - if retry_options.limit and retry_options.limit > 0: - reqs_to_retry = [req._replace(retries=req.retries+1) - for req in reqs_to_retry - if req.retries < retry_options.limit] - - # doing backoff before next retry - if (reqs_to_retry and type(ex) in RETRY_BACKOFF_ERROR_TYPES - and retry_options.backoff_ms): - log.warning("Doing backoff for %s(ms)." % retry_options.backoff_ms) - time.sleep(float(retry_options.backoff_ms) / 1000) - - # refresh topic metadata before next retry - if reqs_to_retry and type(ex) in RETRY_REFRESH_ERROR_TYPES: - client.load_metadata_for_topics() + except RequestTimedOutError: + error_type = RequestTimedOutError + if retry_options.retry_on_timeouts: + reqs_to_retry = reqs except Exception as ex: - log.exception("Unable to send message: %s" % type(ex)) + error_type = type(ex) + if type(ex) in RETRY_ERROR_TYPES: + reqs_to_retry = reqs finally: reqs = [] - if reqs_to_retry: - reqs = reqs_to_retry + if not reqs_to_retry: + continue + + # doing backoff before next retry + if error_type in RETRY_BACKOFF_ERROR_TYPES and retry_options.backoff_ms: + log.warning("Doing backoff for %s(ms)." % retry_options.backoff_ms) + time.sleep(float(retry_options.backoff_ms) / 1000) + + # refresh topic metadata before next retry + if error_type in RETRY_REFRESH_ERROR_TYPES: + client.load_metadata_for_topics() + + reqs = reqs_to_retry + # filter reqs_to_retry if there's a retry limit + if retry_options.limit and retry_options.limit > 0: + reqs = [req._replace(retries=req.retries+1) + for req in reqs if req.retries < retry_options.limit] class Producer(object): From 5119bb605acc4b24e091778656b229a36f9cac11 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 22 Apr 2015 12:14:11 +0300 Subject: [PATCH 106/250] Fix names for async retries opts, add timeout for put --- kafka/common.py | 4 ++-- kafka/producer/base.py | 28 ++++++++++++++++------------ kafka/producer/keyed.py | 11 ++++++++--- kafka/producer/simple.py | 11 ++++++++--- 4 files changed, 34 insertions(+), 20 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index e327d02..87c29f0 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -78,12 +78,12 @@ TopicAndPartition = namedtuple("TopicAndPartition", KafkaMessage = namedtuple("KafkaMessage", ["topic", "partition", "offset", "key", "value"]) +# Define retry policy for async producer +# Limit corner values: None - infinite retries, 0 - no retries RetryOptions = namedtuple("RetryOptions", ["limit", "backoff_ms", "retry_on_timeouts"]) - - ################# # Exceptions # ################# diff --git a/kafka/producer/base.py b/kafka/producer/base.py index fffea94..0b31d18 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -29,11 +29,13 @@ log = logging.getLogger("kafka") BATCH_SEND_DEFAULT_INTERVAL = 20 BATCH_SEND_MSG_COUNT = 20 -BATCH_RETRY_OPTIONS = RetryOptions( - limit=0, backoff_ms=300, retry_on_timeouts=False) # unlimited ASYNC_QUEUE_MAXSIZE = 0 +ASYNC_QUEUE_PUT_TIMEOUT = 0 +# no retries by default +ASYNC_RETRY_OPTIONS = RetryOptions( + limit=0, backoff_ms=0, retry_on_timeouts=False) STOP_ASYNC_PRODUCER = -1 @@ -108,7 +110,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, finally: reqs = [] - if not reqs_to_retry: + if not reqs_to_retry or retry_options.limit == 0: continue # doing backoff before next retry @@ -120,11 +122,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if error_type in RETRY_REFRESH_ERROR_TYPES: client.load_metadata_for_topics() - reqs = reqs_to_retry - # filter reqs_to_retry if there's a retry limit - if retry_options.limit and retry_options.limit > 0: - reqs = [req._replace(retries=req.retries+1) - for req in reqs if req.retries < retry_options.limit] + reqs = [req._replace(retries=req.retries+1) + for req in reqs_to_retry + if not retry_options.limit or + (retry_options.limit and req.retries < retry_options.limit)] class Producer(object): @@ -160,8 +161,9 @@ class Producer(object): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - batch_retry_options=BATCH_RETRY_OPTIONS, - async_queue_maxsize=ASYNC_QUEUE_MAXSIZE): + async_retry_options=ASYNC_RETRY_OPTIONS, + async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, + async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): if batch_send: async = True @@ -188,6 +190,7 @@ class Producer(object): if self.async: # Messages are sent through this queue self.queue = Queue(async_queue_maxsize) + self.async_queue_put_timeout = async_queue_put_timeout self.thread_stop_event = Event() self.thread = Thread(target=_send_upstream, args=(self.queue, @@ -197,7 +200,7 @@ class Producer(object): batch_send_every_n, self.req_acks, self.ack_timeout, - batch_retry_options, + async_retry_options, self.thread_stop_event)) # Thread will die if main thread exits @@ -249,10 +252,11 @@ class Producer(object): raise TypeError("the key must be type bytes") if self.async: + put_timeout = self.async_queue_put_timeout for m in msg: try: item = (TopicAndPartition(topic, partition), m, key) - self.queue.put_nowait(item) + self.queue.put(item, bool(put_timeout), put_timeout) except Full: raise AsyncProducerQueueFull( 'Producer async queue overfilled. ' diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 7bcc629..0fdccd5 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -7,7 +7,8 @@ from kafka.util import kafka_bytestring from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, BATCH_RETRY_OPTIONS + BATCH_SEND_MSG_COUNT, ASYNC_RETRY_OPTIONS, + ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT ) log = logging.getLogger("kafka") @@ -38,7 +39,9 @@ class KeyedProducer(Producer): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - batch_retry_options=BATCH_RETRY_OPTIONS): + async_retry_options=ASYNC_RETRY_OPTIONS, + async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, + async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): if not partitioner: partitioner = HashedPartitioner self.partitioner_class = partitioner @@ -48,7 +51,9 @@ class KeyedProducer(Producer): ack_timeout, codec, batch_send, batch_send_every_n, batch_send_every_t, - batch_retry_options) + async_retry_options, + async_queue_maxsize, + async_queue_put_timeout) def _next_partition(self, topic, key): if topic not in self.partitioners: diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index b869683..f7dfc46 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -10,7 +10,8 @@ from six.moves import xrange from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, BATCH_RETRY_OPTIONS + BATCH_SEND_MSG_COUNT, ASYNC_RETRY_OPTIONS, + ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT ) log = logging.getLogger("kafka") @@ -46,14 +47,18 @@ class SimpleProducer(Producer): batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, random_start=True, - batch_retry_options=BATCH_RETRY_OPTIONS): + async_retry_options=ASYNC_RETRY_OPTIONS, + async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, + async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): self.partition_cycles = {} self.random_start = random_start super(SimpleProducer, self).__init__(client, async, req_acks, ack_timeout, codec, batch_send, batch_send_every_n, batch_send_every_t, - batch_retry_options) + async_retry_options, + async_queue_maxsize, + async_queue_put_timeout) def _next_partition(self, topic): if topic not in self.partition_cycles: From 91af27c64488a0029e960615d0f10d62532d6616 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 22 Apr 2015 12:40:04 +0300 Subject: [PATCH 107/250] Fix async producer queue put arguments --- kafka/producer/base.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 0b31d18..df391f7 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -252,11 +252,13 @@ class Producer(object): raise TypeError("the key must be type bytes") if self.async: - put_timeout = self.async_queue_put_timeout for m in msg: try: item = (TopicAndPartition(topic, partition), m, key) - self.queue.put(item, bool(put_timeout), put_timeout) + if self.async_queue_put_timeout == 0: + self.queue.put_nowait(item) + else: + self.queue.put(item, True, self.async_queue_put_timeout) except Full: raise AsyncProducerQueueFull( 'Producer async queue overfilled. ' From 9eed1698fdcdda7cd8efe7bbee559ccf8baf9e1e Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 22 Apr 2015 19:05:18 +0300 Subject: [PATCH 108/250] Change backoff message log level --- kafka/producer/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index df391f7..03ef2a7 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -115,7 +115,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # doing backoff before next retry if error_type in RETRY_BACKOFF_ERROR_TYPES and retry_options.backoff_ms: - log.warning("Doing backoff for %s(ms)." % retry_options.backoff_ms) + log.info("Doing backoff for %s(ms)." % retry_options.backoff_ms) time.sleep(float(retry_options.backoff_ms) / 1000) # refresh topic metadata before next retry From 4c682f3d4da6c5af8bfbb00700c431a272b37dc1 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 22 Apr 2015 19:08:13 +0300 Subject: [PATCH 109/250] Increase producer test timeout --- test/test_producer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_producer.py b/test/test_producer.py index de012b9..3004c2d 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -158,7 +158,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.client.send_produce_request.side_effect = send_side_effect - self._run_process(3, 2) + self._run_process(3, 3) # the queue should be void at the end of the test self.assertEqual(self.queue.empty(), True) From a3fb3225a27ba6ca1a9fdac519c1f4257754d4eb Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Fri, 15 May 2015 12:58:34 +0300 Subject: [PATCH 110/250] Improve async producer code: logic and style fixes - send_producer_request with fail_on_error=False to retry failed reqs only - using an internal dict with with namedtuple keys for retry counters - refresh metadata on refresh_error irrespective to retries options - removed infinite retries (retry_options.limit=None) as an over-feature - separate producer init args for retries options (limit,backoff,on_timeouts) - AsyncProducerQueueFull returns a list of failed messages - producer tests improved thanks to @rogaha and @toli --- kafka/common.py | 17 +++++------- kafka/producer/base.py | 57 +++++++++++++++++++++++----------------- kafka/producer/keyed.py | 12 ++++++--- kafka/producer/simple.py | 12 ++++++--- test/test_producer.py | 53 ++++++++++++++----------------------- 5 files changed, 75 insertions(+), 76 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 87c29f0..8c13798 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -14,15 +14,8 @@ MetadataResponse = namedtuple("MetadataResponse", ["brokers", "topics"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ProduceAPI -_ProduceRequest = namedtuple("ProduceRequest", - ["topic", "partition", "messages", "retries"]) - - -class ProduceRequest(_ProduceRequest): - def __new__(cls, topic, partition, messages, retries=0): - return super(ProduceRequest, cls).__new__( - cls, topic, partition, messages, retries) - +ProduceRequest = namedtuple("ProduceRequest", + ["topic", "partition", "messages"]) ProduceResponse = namedtuple("ProduceResponse", ["topic", "partition", "error", "offset"]) @@ -79,7 +72,7 @@ KafkaMessage = namedtuple("KafkaMessage", ["topic", "partition", "offset", "key", "value"]) # Define retry policy for async producer -# Limit corner values: None - infinite retries, 0 - no retries +# Limit value: int >= 0, 0 means no retries RetryOptions = namedtuple("RetryOptions", ["limit", "backoff_ms", "retry_on_timeouts"]) @@ -218,7 +211,9 @@ class KafkaConfigurationError(KafkaError): class AsyncProducerQueueFull(KafkaError): - pass + def __init__(self, failed_msgs, *args): + super(AsyncProducerQueueFull, self).__init__(*args) + self.failed_msgs = failed_msgs def _iter_broker_errors(): diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 03ef2a7..602e2ed 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -34,8 +34,10 @@ BATCH_SEND_MSG_COUNT = 20 ASYNC_QUEUE_MAXSIZE = 0 ASYNC_QUEUE_PUT_TIMEOUT = 0 # no retries by default -ASYNC_RETRY_OPTIONS = RetryOptions( - limit=0, backoff_ms=0, retry_on_timeouts=False) +ASYNC_RETRY_LIMIT = 0 +ASYNC_RETRY_BACKOFF_MS = 0 +ASYNC_RETRY_ON_TIMEOUTS = False + STOP_ASYNC_PRODUCER = -1 @@ -46,7 +48,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, a specified timeout and send them upstream to the brokers in one request """ - reqs = [] + reqs = {} client.reinit() while not stop_event.is_set(): @@ -81,36 +83,38 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, messages = create_message_set(msg, codec, key) req = ProduceRequest(topic_partition.topic, topic_partition.partition, - messages) - reqs.append(req) + tuple(messages)) + reqs[req] = 0 if not reqs: continue reqs_to_retry, error_type = [], None - try: - client.send_produce_request(reqs, - acks=req_acks, - timeout=ack_timeout) - except FailedPayloadsError as ex: - error_type = FailedPayloadsError - reqs_to_retry = ex.failed_payloads + try: + reply = client.send_produce_request(reqs.keys(), + acks=req_acks, + timeout=ack_timeout, + fail_on_error=False) + reqs_to_retry = [req for broker_responses in reply + for response in broker_responses + for req in response.failed_payloads + if isinstance(response, FailedPayloadsError)] + if reqs_to_retry: + error_type = FailedPayloadsError except RequestTimedOutError: error_type = RequestTimedOutError if retry_options.retry_on_timeouts: - reqs_to_retry = reqs + reqs_to_retry = reqs.keys() except Exception as ex: error_type = type(ex) if type(ex) in RETRY_ERROR_TYPES: - reqs_to_retry = reqs + reqs_to_retry = reqs.keys() - finally: - reqs = [] - - if not reqs_to_retry or retry_options.limit == 0: + if not reqs_to_retry: + reqs = {} continue # doing backoff before next retry @@ -122,10 +126,8 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if error_type in RETRY_REFRESH_ERROR_TYPES: client.load_metadata_for_topics() - reqs = [req._replace(retries=req.retries+1) - for req in reqs_to_retry - if not retry_options.limit or - (retry_options.limit and req.retries < retry_options.limit)] + reqs = {key: count + 1 for key, count in reqs.items() + if key in reqs_to_retry and count < retry_options.limit} class Producer(object): @@ -161,7 +163,9 @@ class Producer(object): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - async_retry_options=ASYNC_RETRY_OPTIONS, + async_retry_limit=ASYNC_RETRY_LIMIT, + async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS, + async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): @@ -191,6 +195,10 @@ class Producer(object): # Messages are sent through this queue self.queue = Queue(async_queue_maxsize) self.async_queue_put_timeout = async_queue_put_timeout + async_retry_options = RetryOptions( + limit=async_retry_limit, + backoff_ms=async_retry_backoff_ms, + retry_on_timeouts=async_retry_on_timeouts) self.thread_stop_event = Event() self.thread = Thread(target=_send_upstream, args=(self.queue, @@ -252,7 +260,7 @@ class Producer(object): raise TypeError("the key must be type bytes") if self.async: - for m in msg: + for idx, m in enumerate(msg): try: item = (TopicAndPartition(topic, partition), m, key) if self.async_queue_put_timeout == 0: @@ -261,6 +269,7 @@ class Producer(object): self.queue.put(item, True, self.async_queue_put_timeout) except Full: raise AsyncProducerQueueFull( + msg[idx:], 'Producer async queue overfilled. ' 'Current queue size %d.' % self.queue.qsize()) resp = [] diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 0fdccd5..5252976 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -7,8 +7,8 @@ from kafka.util import kafka_bytestring from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, ASYNC_RETRY_OPTIONS, - ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT + BATCH_SEND_MSG_COUNT, ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT, + ASYNC_RETRY_LIMIT, ASYNC_RETRY_BACKOFF_MS, ASYNC_RETRY_ON_TIMEOUTS ) log = logging.getLogger("kafka") @@ -39,7 +39,9 @@ class KeyedProducer(Producer): batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - async_retry_options=ASYNC_RETRY_OPTIONS, + async_retry_limit=ASYNC_RETRY_LIMIT, + async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS, + async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): if not partitioner: @@ -51,7 +53,9 @@ class KeyedProducer(Producer): ack_timeout, codec, batch_send, batch_send_every_n, batch_send_every_t, - async_retry_options, + async_retry_limit, + async_retry_backoff_ms, + async_retry_on_timeouts, async_queue_maxsize, async_queue_put_timeout) diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index f7dfc46..ded6eb6 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -10,8 +10,8 @@ from six.moves import xrange from .base import ( Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, ASYNC_RETRY_OPTIONS, - ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT + BATCH_SEND_MSG_COUNT, ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT, + ASYNC_RETRY_LIMIT, ASYNC_RETRY_BACKOFF_MS, ASYNC_RETRY_ON_TIMEOUTS ) log = logging.getLogger("kafka") @@ -47,7 +47,9 @@ class SimpleProducer(Producer): batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, random_start=True, - async_retry_options=ASYNC_RETRY_OPTIONS, + async_retry_limit=ASYNC_RETRY_LIMIT, + async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS, + async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): self.partition_cycles = {} @@ -56,7 +58,9 @@ class SimpleProducer(Producer): ack_timeout, codec, batch_send, batch_send_every_n, batch_send_every_t, - async_retry_options, + async_retry_limit, + async_retry_backoff_ms, + async_retry_on_timeouts, async_queue_maxsize, async_queue_put_timeout) diff --git a/test/test_producer.py b/test/test_producer.py index 3004c2d..a2ba877 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -17,6 +17,10 @@ try: from queue import Empty, Queue except ImportError: from Queue import Empty, Queue +try: + xrange +except NameError: + xrange = range class TestKafkaProducer(unittest.TestCase): @@ -52,7 +56,8 @@ class TestKafkaProducer(unittest.TestCase): producer.send_messages(topic, b'hi') assert client.send_produce_request.called - def test_producer_async_queue_overfilled_batch_send(self): + @patch('kafka.producer.base._send_upstream') + def test_producer_async_queue_overfilled_batch_send(self, mock): queue_size = 2 producer = Producer(MagicMock(), batch_send=True, async_queue_maxsize=queue_size) @@ -64,8 +69,12 @@ class TestKafkaProducer(unittest.TestCase): with self.assertRaises(AsyncProducerQueueFull): message_list = [message] * (queue_size + 1) producer.send_messages(topic, partition, *message_list) + self.assertEqual(producer.queue.qsize(), queue_size) + for _ in xrange(producer.queue.qsize()): + producer.queue.get() - def test_producer_async_queue_overfilled(self): + @patch('kafka.producer.base._send_upstream') + def test_producer_async_queue_overfilled(self, mock): queue_size = 2 producer = Producer(MagicMock(), async=True, async_queue_maxsize=queue_size) @@ -77,7 +86,9 @@ class TestKafkaProducer(unittest.TestCase): with self.assertRaises(AsyncProducerQueueFull): message_list = [message] * (queue_size + 1) producer.send_messages(topic, partition, *message_list) - + self.assertEqual(producer.queue.qsize(), queue_size) + for _ in xrange(producer.queue.qsize()): + producer.queue.get() class TestKafkaProducerSendUpstream(unittest.TestCase): @@ -121,7 +132,6 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # 3 batches of 3 msgs each + 1 batch of 1 message self.assertEqual(self.client.send_produce_request.call_count, 4) - def test_first_send_failed(self): # lets create a queue and add 10 messages for 10 different partitions @@ -133,7 +143,8 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def send_side_effect(reqs, *args, **kwargs): if self.client.is_first_time: self.client.is_first_time = False - raise FailedPayloadsError(reqs) + return [[FailedPayloadsError(reqs)]] + return [] self.client.send_produce_request.side_effect = send_side_effect @@ -154,7 +165,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.queue.put((TopicAndPartition("test", i), "msg %i" % i, "key %i" % i)) def send_side_effect(reqs, *args, **kwargs): - raise FailedPayloadsError(reqs) + return [[FailedPayloadsError(reqs)]] self.client.send_produce_request.side_effect = send_side_effect @@ -168,30 +179,6 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # 3 retries of the batches above = 4 + 3 * 4 = 16, all failed self.assertEqual(self.client.send_produce_request.call_count, 16) - def test_with_unlimited_retries(self): - - # lets create a queue and add 10 messages for 10 different partitions - # to show how retries should work ideally - for i in range(10): - self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) - - def send_side_effect(reqs, *args, **kwargs): - raise FailedPayloadsError(reqs) - - self.client.send_produce_request.side_effect = send_side_effect - - self._run_process(None) - - # the queue should have 7 elements - # 3 batches of 1 msg each were retried all this time - self.assertEqual(self.queue.empty(), False) - try: - for i in range(7): - self.queue.get(timeout=0.01) - except Empty: - self.fail("Should be 7 elems in the queue") - self.assertEqual(self.queue.empty(), True) - - # 1s / 50ms of backoff = 20 times max - calls = self.client.send_produce_request.call_count - self.assertTrue(calls > 10 & calls <= 20) + def tearDown(self): + for _ in xrange(self.queue.qsize()): + self.queue.get() From 4474a50162f73aaabe770ee6dce9bd9b2110b2d5 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Fri, 15 May 2015 13:52:59 +0300 Subject: [PATCH 111/250] Async producer: py2.6 backward compatibility fix --- kafka/producer/base.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 602e2ed..2e2f3c4 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -126,8 +126,8 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if error_type in RETRY_REFRESH_ERROR_TYPES: client.load_metadata_for_topics() - reqs = {key: count + 1 for key, count in reqs.items() - if key in reqs_to_retry and count < retry_options.limit} + reqs = dict((key, count + 1) for (key, count) in reqs.items() + if key in reqs_to_retry and count < retry_options.limit) class Producer(object): From 7d6f3f541e0c380c0600eb607d927ec8f8cc966f Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 3 Jun 2015 23:09:15 +0300 Subject: [PATCH 112/250] Check response.error for async producer --- kafka/producer/base.py | 43 +++++++++++++++++++++++------------------- test/test_producer.py | 4 ++-- tox.ini | 1 + 3 files changed, 27 insertions(+), 21 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 2e2f3c4..2edeace 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -16,8 +16,8 @@ import six from kafka.common import ( ProduceRequest, TopicAndPartition, RetryOptions, - UnsupportedCodecError, FailedPayloadsError, - RequestTimedOutError, AsyncProducerQueueFull + kafka_errors, UnsupportedCodecError, FailedPayloadsError, + RequestTimedOutError, AsyncProducerQueueFull, UnknownError ) from kafka.common import ( RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES) @@ -89,41 +89,46 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if not reqs: continue - reqs_to_retry, error_type = [], None + reqs_to_retry, error_cls = [], None + do_backoff, do_refresh = False, False + + def _handle_error(error_cls, reqs, all_retries): + if ((error_cls == RequestTimedOutError and + retry_options.retry_on_timeouts) or + error_cls in RETRY_ERROR_TYPES): + all_retries += reqs + if error_cls in RETRY_BACKOFF_ERROR_TYPES: + do_backoff = True + if error_cls in RETRY_REFRESH_ERROR_TYPES: + do_refresh = True try: reply = client.send_produce_request(reqs.keys(), acks=req_acks, timeout=ack_timeout, fail_on_error=False) - reqs_to_retry = [req for broker_responses in reply - for response in broker_responses - for req in response.failed_payloads - if isinstance(response, FailedPayloadsError)] - if reqs_to_retry: - error_type = FailedPayloadsError - - except RequestTimedOutError: - error_type = RequestTimedOutError - if retry_options.retry_on_timeouts: - reqs_to_retry = reqs.keys() + for i, response in enumerate(reply): + if isinstance(response, FailedPayloadsError): + _handle_error(FailedPayloadsError, response.failed_payloads, reqs_to_retry) + elif isinstance(response, ProduceResponse) and response.error: + error_cls = kafka_errors.get(response.error, UnknownError) + _handle_error(error_cls, [reqs.keys()[i]], reqs_to_retry) except Exception as ex: - error_type = type(ex) - if type(ex) in RETRY_ERROR_TYPES: - reqs_to_retry = reqs.keys() + error_cls = kafka_errors.get(type(ex), UnknownError) + _handle_error(error_cls, reqs.keys(), reqs_to_retry) if not reqs_to_retry: reqs = {} continue # doing backoff before next retry - if error_type in RETRY_BACKOFF_ERROR_TYPES and retry_options.backoff_ms: + if do_backoff and retry_options.backoff_ms: log.info("Doing backoff for %s(ms)." % retry_options.backoff_ms) time.sleep(float(retry_options.backoff_ms) / 1000) # refresh topic metadata before next retry - if error_type in RETRY_REFRESH_ERROR_TYPES: + if do_refresh: client.load_metadata_for_topics() reqs = dict((key, count + 1) for (key, count) in reqs.items() diff --git a/test/test_producer.py b/test/test_producer.py index a2ba877..258b9c3 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -143,7 +143,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def send_side_effect(reqs, *args, **kwargs): if self.client.is_first_time: self.client.is_first_time = False - return [[FailedPayloadsError(reqs)]] + return [FailedPayloadsError(reqs)] return [] self.client.send_produce_request.side_effect = send_side_effect @@ -165,7 +165,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.queue.put((TopicAndPartition("test", i), "msg %i" % i, "key %i" % i)) def send_side_effect(reqs, *args, **kwargs): - return [[FailedPayloadsError(reqs)]] + return [FailedPayloadsError(reqs)] self.client.send_produce_request.side_effect = send_side_effect diff --git a/tox.ini b/tox.ini index fba7d8e..e3e8568 100644 --- a/tox.ini +++ b/tox.ini @@ -14,6 +14,7 @@ commands = nosetests {posargs:-v --with-id --id-file={envdir}/.noseids --with-timer --timer-top-n 10 --with-coverage --cover-erase --cover-package kafka} setenv = PROJECT_ROOT = {toxinidir} +passenv = KAFKA_VERSION [testenv:py33] deps = From ded79e85f413f2869342d202412b2e7057bead00 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 15:54:00 -0700 Subject: [PATCH 113/250] Set __version__ string in kafka/version.py --- MANIFEST.in | 1 - VERSION | 1 - kafka/__init__.py | 4 +--- kafka/version.py | 1 + setup.py | 6 +++--- 5 files changed, 5 insertions(+), 8 deletions(-) delete mode 100644 VERSION create mode 100644 kafka/version.py diff --git a/MANIFEST.in b/MANIFEST.in index 68bd793..bdd6505 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -1,2 +1 @@ -include VERSION recursive-include kafka *.py diff --git a/VERSION b/VERSION deleted file mode 100644 index 8caff32..0000000 --- a/VERSION +++ /dev/null @@ -1 +0,0 @@ -0.9.4-dev diff --git a/kafka/__init__.py b/kafka/__init__.py index 3536084..396a8b8 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -1,7 +1,5 @@ __title__ = 'kafka' -# Use setuptools to get version from setup.py -import pkg_resources -__version__ = pkg_resources.require('kafka-python')[0].version +from .version import __version__ __author__ = 'David Arthur' __license__ = 'Apache License 2.0' __copyright__ = 'Copyright 2015, David Arthur under Apache License, v2.0' diff --git a/kafka/version.py b/kafka/version.py new file mode 100644 index 0000000..5b721ed --- /dev/null +++ b/kafka/version.py @@ -0,0 +1 @@ +__version__ = '0.9.4-dev' diff --git a/setup.py b/setup.py index f1c1954..8e4fb66 100644 --- a/setup.py +++ b/setup.py @@ -2,9 +2,9 @@ import sys import os from setuptools import setup, Command -with open('VERSION', 'r') as v: - __version__ = v.read().rstrip() - +# Pull version from source without importing +# since we can't import something we haven't built yet :) +exec(open('kafka/version.py').read()) class Tox(Command): From 6406747f37618a9409a4e9bbf805c7bcc8b9fe0e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Jun 2015 10:24:39 -0700 Subject: [PATCH 114/250] Fix import error in kafka/producer/base.py --- kafka/producer/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 2edeace..97565c3 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -15,7 +15,7 @@ from threading import Thread, Event import six from kafka.common import ( - ProduceRequest, TopicAndPartition, RetryOptions, + ProduceRequest, ProduceResponse, TopicAndPartition, RetryOptions, kafka_errors, UnsupportedCodecError, FailedPayloadsError, RequestTimedOutError, AsyncProducerQueueFull, UnknownError ) From 82aae4f00a94ebf5d2c34be5cc24e50fc3e16701 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 22:41:09 -0700 Subject: [PATCH 115/250] KafkaConsumer.task_done - warn and skip messages from unrecognized topic-partitions --- kafka/consumer/kafka.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 60f1a0b..7ba83cb 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -448,10 +448,16 @@ class KafkaConsumer(object): message (KafkaMessage): the message to mark as complete Returns: - Nothing - + True, unless the topic-partition for this message has not + been configured for the consumer. In normal operation, this + should not happen. But see github issue 364. """ topic_partition = (message.topic, message.partition) + if topic_partition not in self._topics: + logger.warning('Unrecognized topic/partition in task_done message: ' + '{0}:{1}'.format(*topic_partition)) + return False + offset = message.offset # Warn on non-contiguous offsets @@ -476,6 +482,8 @@ class KafkaConsumer(object): if self._should_auto_commit(): self.commit() + return True + def commit(self): """Store consumed message offsets (marked via task_done()) to kafka cluster for this consumer_group. From 5d9b174cad28ef09202151b12571b8811e781841 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 23:42:07 -0700 Subject: [PATCH 116/250] Use separate module loggers instead of a single 'kafka' logger --- kafka/client.py | 3 ++- kafka/conn.py | 3 ++- kafka/consumer/base.py | 3 ++- kafka/consumer/multiprocess.py | 4 +++- kafka/consumer/simple.py | 4 +++- kafka/producer/base.py | 2 +- kafka/producer/keyed.py | 2 +- kafka/producer/simple.py | 2 +- kafka/protocol.py | 3 ++- 9 files changed, 17 insertions(+), 9 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 6ef9d83..7ea09d7 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -16,7 +16,8 @@ from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SE from kafka.protocol import KafkaProtocol from kafka.util import kafka_bytestring -log = logging.getLogger("kafka") + +log = logging.getLogger(__name__) class KafkaClient(object): diff --git a/kafka/conn.py b/kafka/conn.py index ea55481..7a49d8c 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -9,7 +9,8 @@ import six from kafka.common import ConnectionError -log = logging.getLogger("kafka") + +log = logging.getLogger(__name__) DEFAULT_SOCKET_TIMEOUT_SECONDS = 120 DEFAULT_KAFKA_PORT = 9092 diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index f53217f..6365cfa 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -13,7 +13,8 @@ from kafka.common import ( from kafka.util import kafka_bytestring, ReentrantTimer -log = logging.getLogger("kafka") + +log = logging.getLogger('kafka.consumer') AUTO_COMMIT_MSG_COUNT = 100 AUTO_COMMIT_INTERVAL = 5000 diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index cfe0ef6..2ca76b6 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -18,9 +18,11 @@ from .base import ( ) from .simple import Consumer, SimpleConsumer + +log = logging.getLogger(__name__) + Events = namedtuple("Events", ["start", "pause", "exit"]) -log = logging.getLogger("kafka") def _mp_consume(client, group, topic, queue, size, events, **consumer_options): """ diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index ae00dab..2c2f820 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -34,7 +34,9 @@ from .base import ( NO_MESSAGES_WAIT_TIME_SECONDS ) -log = logging.getLogger("kafka") + +log = logging.getLogger(__name__) + class FetchContext(object): """ diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 97565c3..5cbc521 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -25,7 +25,7 @@ from kafka.common import ( from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set from kafka.util import kafka_bytestring -log = logging.getLogger("kafka") +log = logging.getLogger('kafka.producer') BATCH_SEND_DEFAULT_INTERVAL = 20 BATCH_SEND_MSG_COUNT = 20 diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 5252976..6bb2285 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -11,7 +11,7 @@ from .base import ( ASYNC_RETRY_LIMIT, ASYNC_RETRY_BACKOFF_MS, ASYNC_RETRY_ON_TIMEOUTS ) -log = logging.getLogger("kafka") +log = logging.getLogger(__name__) class KeyedProducer(Producer): diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index ded6eb6..78cc21c 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -14,7 +14,7 @@ from .base import ( ASYNC_RETRY_LIMIT, ASYNC_RETRY_BACKOFF_MS, ASYNC_RETRY_ON_TIMEOUTS ) -log = logging.getLogger("kafka") +log = logging.getLogger(__name__) class SimpleProducer(Producer): diff --git a/kafka/protocol.py b/kafka/protocol.py index b34a95d..f12e6a3 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -21,7 +21,8 @@ from kafka.util import ( write_short_string, write_int_string, group_by_topic_and_partition ) -log = logging.getLogger("kafka") + +log = logging.getLogger(__name__) ATTRIBUTE_CODEC_MASK = 0x03 CODEC_NONE = 0x00 From 3a4ceef3816e9ddfcb86099f691f72f1f60921cd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Jun 2015 16:10:19 -0700 Subject: [PATCH 117/250] Module loggers in test/fixtures and test/service --- test/fixtures.py | 42 +++++++++++++++++++++++------------------- test/service.py | 20 ++++++++++++-------- 2 files changed, 35 insertions(+), 27 deletions(-) diff --git a/test/fixtures.py b/test/fixtures.py index 3c496fd..90d01f1 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -11,6 +11,10 @@ from six.moves.urllib.parse import urlparse # pylint: disable-msg=E0611 from test.service import ExternalService, SpawnedService from test.testutil import get_open_port + +log = logging.getLogger(__name__) + + class Fixture(object): kafka_version = os.environ.get('KAFKA_VERSION', '0.8.0') scala_version = os.environ.get("SCALA_VERSION", '2.8.0') @@ -35,21 +39,21 @@ class Fixture(object): output_file = os.path.join(output_dir, distfile + '.tgz') if os.path.isfile(output_file): - logging.info("Found file already on disk: %s", output_file) + log.info("Found file already on disk: %s", output_file) return output_file # New tarballs are .tgz, older ones are sometimes .tar.gz try: url = url_base + distfile + '.tgz' - logging.info("Attempting to download %s", url) + log.info("Attempting to download %s", url) response = urllib.request.urlopen(url) except urllib.error.HTTPError: - logging.exception("HTTP Error") + log.exception("HTTP Error") url = url_base + distfile + '.tar.gz' - logging.info("Attempting to download %s", url) + log.info("Attempting to download %s", url) response = urllib.request.urlopen(url) - logging.info("Saving distribution file to %s", output_file) + log.info("Saving distribution file to %s", output_file) with open(output_file, 'w') as output_file_fd: output_file_fd.write(response.read()) @@ -101,14 +105,14 @@ class ZookeeperFixture(Fixture): self.child = None def out(self, message): - logging.info("*** Zookeeper [%s:%d]: %s", self.host, self.port, message) + log.info("*** Zookeeper [%s:%d]: %s", self.host, self.port, message) def open(self): self.tmp_dir = tempfile.mkdtemp() self.out("Running local instance...") - logging.info(" host = %s", self.host) - logging.info(" port = %s", self.port) - logging.info(" tmp_dir = %s", self.tmp_dir) + log.info(" host = %s", self.host) + log.info(" port = %s", self.port) + log.info(" tmp_dir = %s", self.tmp_dir) # Generate configs template = self.test_resource("zookeeper.properties") @@ -167,7 +171,7 @@ class KafkaFixture(Fixture): self.running = False def out(self, message): - logging.info("*** Kafka [%s:%d]: %s", self.host, self.port, message) + log.info("*** Kafka [%s:%d]: %s", self.host, self.port, message) def open(self): if self.running: @@ -176,15 +180,15 @@ class KafkaFixture(Fixture): self.tmp_dir = tempfile.mkdtemp() self.out("Running local instance...") - logging.info(" host = %s", self.host) - logging.info(" port = %s", self.port) - logging.info(" broker_id = %s", self.broker_id) - logging.info(" zk_host = %s", self.zk_host) - logging.info(" zk_port = %s", self.zk_port) - logging.info(" zk_chroot = %s", self.zk_chroot) - logging.info(" replicas = %s", self.replicas) - logging.info(" partitions = %s", self.partitions) - logging.info(" tmp_dir = %s", self.tmp_dir) + log.info(" host = %s", self.host) + log.info(" port = %s", self.port) + log.info(" broker_id = %s", self.broker_id) + log.info(" zk_host = %s", self.zk_host) + log.info(" zk_port = %s", self.zk_port) + log.info(" zk_chroot = %s", self.zk_chroot) + log.info(" replicas = %s", self.replicas) + log.info(" partitions = %s", self.partitions) + log.info(" tmp_dir = %s", self.tmp_dir) # Create directories os.mkdir(os.path.join(self.tmp_dir, "logs")) diff --git a/test/service.py b/test/service.py index dcd3e68..4048c7c 100644 --- a/test/service.py +++ b/test/service.py @@ -11,9 +11,13 @@ __all__ = [ ] + +log = logging.getLogger(__name__) + + class ExternalService(object): def __init__(self, host, port): - logging.info("Using already running service at %s:%d", host, port) + log.info("Using already running service at %s:%d", host, port) self.host = host self.port = port @@ -73,13 +77,13 @@ class SpawnedService(threading.Thread): raise RuntimeError("Subprocess has died. Aborting. (args=%s)" % ' '.join(str(x) for x in self.args)) def dump_logs(self): - logging.critical('stderr') + log.critical('stderr') for line in self.captured_stderr: - logging.critical(line.rstrip()) + log.critical(line.rstrip()) - logging.critical('stdout') + log.critical('stdout') for line in self.captured_stdout: - logging.critical(line.rstrip()) + log.critical(line.rstrip()) def wait_for(self, pattern, timeout=30): t1 = time.time() @@ -89,16 +93,16 @@ class SpawnedService(threading.Thread): try: self.child.kill() except: - logging.exception("Received exception when killing child process") + log.exception("Received exception when killing child process") self.dump_logs() raise RuntimeError("Waiting for %r timed out after %d seconds" % (pattern, timeout)) if re.search(pattern, '\n'.join(self.captured_stdout), re.IGNORECASE) is not None: - logging.info("Found pattern %r in %d seconds via stdout", pattern, (t2 - t1)) + log.info("Found pattern %r in %d seconds via stdout", pattern, (t2 - t1)) return if re.search(pattern, '\n'.join(self.captured_stderr), re.IGNORECASE) is not None: - logging.info("Found pattern %r in %d seconds via stderr", pattern, (t2 - t1)) + log.info("Found pattern %r in %d seconds via stderr", pattern, (t2 - t1)) return time.sleep(0.1) From 868115c703afc4403adc8d9481bf31d2c15064dd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Jun 2015 16:08:01 -0700 Subject: [PATCH 118/250] Raise an error if we attempt to group duplicate topic-partition payloads - previously this would simply drop one of the payloads --- kafka/client.py | 2 ++ kafka/util.py | 3 +++ test/test_util.py | 6 +++++- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index 7ea09d7..4302ce5 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -423,6 +423,8 @@ class KafkaClient(object): Arguments: payloads (list of ProduceRequest): produce requests to send to kafka + ProduceRequest payloads must not contain duplicates for any + topic-partition. acks (int, optional): how many acks the servers should receive from replica brokers before responding to the request. If it is 0, the server will not send any response. If it is 1, the server will wait diff --git a/kafka/util.py b/kafka/util.py index 78c3607..6d9d307 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -82,6 +82,9 @@ def relative_unpack(fmt, data, cur): def group_by_topic_and_partition(tuples): out = collections.defaultdict(dict) for t in tuples: + assert t.topic not in out or t.partition not in out[t.topic], \ + 'Duplicate {0}s for {1} {2}'.format(t.__class__.__name__, + t.topic, t.partition) out[t.topic][t.partition] = t return out diff --git a/test/test_util.py b/test/test_util.py index 6a8f45b..ea3783e 100644 --- a/test/test_util.py +++ b/test/test_util.py @@ -107,7 +107,6 @@ class UtilTest(unittest.TestCase): t = kafka.common.TopicAndPartition l = [ - t("a", 1), t("a", 1), t("a", 2), t("a", 3), @@ -124,3 +123,8 @@ class UtilTest(unittest.TestCase): 3: t("b", 3), } }) + + # should not be able to group duplicate topic-partitions + t1 = t("a", 1) + with self.assertRaises(AssertionError): + kafka.util.group_by_topic_and_partition([t1, t1]) From 207499b26d5ce6c7a72771250b8e4365944aa1bd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 22:10:21 -0700 Subject: [PATCH 119/250] random_string helper should return str not bytes --- test/test_codec.py | 12 ++++++------ test/test_consumer_integration.py | 2 +- test/test_failover_integration.py | 14 +++++++------- test/testutil.py | 5 ++--- 4 files changed, 16 insertions(+), 17 deletions(-) diff --git a/test/test_codec.py b/test/test_codec.py index 2d7670a..3416fdb 100644 --- a/test/test_codec.py +++ b/test/test_codec.py @@ -13,16 +13,16 @@ from test.testutil import random_string class TestCodec(unittest.TestCase): def test_gzip(self): for i in xrange(1000): - s1 = random_string(100) - s2 = gzip_decode(gzip_encode(s1)) - self.assertEqual(s1, s2) + b1 = random_string(100).encode('utf-8') + b2 = gzip_decode(gzip_encode(b1)) + self.assertEqual(b1, b2) @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.assertEqual(s1, s2) + b1 = random_string(100).encode('utf-8') + b2 = snappy_decode(snappy_encode(b1)) + self.assertEqual(b1, b2) @unittest.skipUnless(has_snappy(), "Snappy not available") def test_snappy_detect_xerial(self): diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index ddb54a7..3825f94 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -475,7 +475,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") def test_kafka_consumer__offset_commit_resume(self): - GROUP_ID = random_string(10) + GROUP_ID = random_string(10).encode('utf-8') self.send_messages(0, range(0, 100)) self.send_messages(1, range(100, 200)) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 3be0189..11e255d 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -133,8 +133,8 @@ class TestFailover(KafkaIntegrationTestCase): # Send 10 random messages for _ in range(10): - key = random_string(3) - msg = random_string(10) + key = random_string(3).encode('utf-8') + msg = random_string(10).encode('utf-8') producer.send_messages(topic, key, msg) # kill leader for partition 0 @@ -145,8 +145,8 @@ class TestFailover(KafkaIntegrationTestCase): timeout = 60 while not recovered and (time.time() - started) < timeout: try: - key = random_string(3) - msg = random_string(10) + key = random_string(3).encode('utf-8') + msg = random_string(10).encode('utf-8') producer.send_messages(topic, key, msg) if producer.partitioners[kafka_bytestring(topic)].partition(key) == 0: recovered = True @@ -159,15 +159,15 @@ class TestFailover(KafkaIntegrationTestCase): # send some more messages just to make sure no more exceptions for _ in range(10): - key = random_string(3) - msg = random_string(10) + key = random_string(3).encode('utf-8') + msg = random_string(10).encode('utf-8') producer.send_messages(topic, key, msg) def _send_random_messages(self, producer, topic, partition, n): for j in range(n): logging.debug('_send_random_message to %s:%d -- try %d', topic, partition, j) - resp = producer.send_messages(topic, partition, random_string(10)) + resp = producer.send_messages(topic, partition, random_string(10).encode('utf-8')) if len(resp) > 0: self.assertEqual(resp[0].error, 0) logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j) diff --git a/test/testutil.py b/test/testutil.py index e6947b4..1f1a1df 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -23,8 +23,7 @@ __all__ = [ ] def random_string(l): - s = "".join(random.choice(string.ascii_letters) for i in xrange(l)) - return s.encode('utf-8') + return "".join(random.choice(string.ascii_letters) for i in xrange(l)) def kafka_versions(*versions): def kafka_versions(func): @@ -60,7 +59,7 @@ class KafkaIntegrationTestCase(unittest.TestCase): return if not self.topic: - topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10).decode('utf-8')) + topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10)) self.topic = topic self.bytes_topic = topic.encode('utf-8') From e16541e3cbfb9501099cf02dc237237e7519d637 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:26:09 -0700 Subject: [PATCH 120/250] FailedPayloadsError now only has a single payload; use .payload attribute .failed_payloads attribute has been removed. --- kafka/common.py | 4 ++-- test/test_producer.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 8c13798..2fdf7d2 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -169,9 +169,9 @@ class KafkaTimeoutError(KafkaError): class FailedPayloadsError(KafkaError): - def __init__(self, failed_payloads, *args): + def __init__(self, payload, *args): super(FailedPayloadsError, self).__init__(*args) - self.failed_payloads = failed_payloads + self.payload = payload class ConnectionError(KafkaError): diff --git a/test/test_producer.py b/test/test_producer.py index 258b9c3..85a5a2e 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -143,7 +143,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def send_side_effect(reqs, *args, **kwargs): if self.client.is_first_time: self.client.is_first_time = False - return [FailedPayloadsError(reqs)] + return [FailedPayloadsError(req) for req in reqs] return [] self.client.send_produce_request.side_effect = send_side_effect @@ -165,7 +165,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): self.queue.put((TopicAndPartition("test", i), "msg %i" % i, "key %i" % i)) def send_side_effect(reqs, *args, **kwargs): - return [FailedPayloadsError(reqs)] + return [FailedPayloadsError(req) for req in reqs] self.client.send_produce_request.side_effect = send_side_effect From 9ac0f057b621c8706e8790b3c10295ef848121c3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:33:07 -0700 Subject: [PATCH 121/250] Reenable test_switch_leader_async --- test/test_failover_integration.py | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 11e255d..c23ab14 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -93,14 +93,14 @@ class TestFailover(KafkaIntegrationTestCase): self.assert_message_count(topic, 201, partitions=(partition,)) - #@kafka_versions("all") - @unittest.skip("async producer does not support reliable failover yet") + @kafka_versions("all") def test_switch_leader_async(self): topic = self.topic partition = 0 # Test the base class Producer -- send_messages to a specific partition - producer = Producer(self.client, async=True) + producer = Producer(self.client, async=True, + req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT) # Send 10 random messages self._send_random_messages(producer, topic, partition, 10) @@ -111,15 +111,21 @@ class TestFailover(KafkaIntegrationTestCase): logging.debug("attempting to send 'success' message after leader killed") # in async mode, this should return immediately - producer.send_messages(topic, partition, 'success') + producer.send_messages(topic, partition, b'success') # send to new leader self._send_random_messages(producer, topic, partition, 10) - # wait until producer queue is empty - while not producer.queue.empty(): - time.sleep(0.1) + # Stop the producer and wait for it to shutdown producer.stop() + started = time.time() + timeout = 60 + while (time.time() - started) < timeout: + if not producer.thread.is_alive(): + break + time.sleep(0.1) + else: + self.fail('timeout waiting for producer queue to empty') # count number of messages # Should be equal to 10 before + 1 recovery + 10 after From 26abd24dde1d50d79f5c4e39776de8e89f586e0b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 23:44:42 -0700 Subject: [PATCH 122/250] add option to check for at-least-once message delivery in failover tests --- test/test_failover_integration.py | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index c23ab14..631068a 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -92,7 +92,6 @@ class TestFailover(KafkaIntegrationTestCase): # Should be equal to 100 before + 1 recovery + 100 after self.assert_message_count(topic, 201, partitions=(partition,)) - @kafka_versions("all") def test_switch_leader_async(self): topic = self.topic @@ -129,7 +128,8 @@ class TestFailover(KafkaIntegrationTestCase): # count number of messages # Should be equal to 10 before + 1 recovery + 10 after - self.assert_message_count(topic, 21, partitions=(partition,)) + self.assert_message_count(topic, 21, partitions=(partition,), + at_least=True) @kafka_versions("all") def test_switch_leader_keyed_producer(self): @@ -184,12 +184,12 @@ class TestFailover(KafkaIntegrationTestCase): broker.close() return broker - def assert_message_count(self, topic, check_count, timeout=10, partitions=None): + def assert_message_count(self, topic, check_count, timeout=10, + partitions=None, at_least=False): hosts = ','.join(['%s:%d' % (broker.host, broker.port) for broker in self.brokers]) client = KafkaClient(hosts) - group = random_string(10) consumer = SimpleConsumer(client, None, topic, partitions=partitions, auto_commit=False, @@ -199,10 +199,16 @@ class TestFailover(KafkaIntegrationTestCase): pending = consumer.pending(partitions) # Keep checking if it isn't immediately correct, subject to timeout - while pending != check_count and (time.time() - started_at < timeout): + while pending < check_count and (time.time() - started_at < timeout): pending = consumer.pending(partitions) consumer.stop() client.close() - self.assertEqual(pending, check_count) + if pending < check_count: + self.fail('Too few pending messages: found %d, expected %d' % + (pending, check_count)) + elif pending > check_count and not at_least: + self.fail('Too many pending messages: found %d, expected %d' % + (pending, check_count)) + return True From 7b86964dd7c0cb71ba1a3ab150240d7b20e1f2be Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:33:53 -0700 Subject: [PATCH 123/250] Include message number in sent message value in _send_random_message --- test/test_failover_integration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 631068a..26da615 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -173,7 +173,7 @@ class TestFailover(KafkaIntegrationTestCase): def _send_random_messages(self, producer, topic, partition, n): for j in range(n): logging.debug('_send_random_message to %s:%d -- try %d', topic, partition, j) - resp = producer.send_messages(topic, partition, random_string(10).encode('utf-8')) + resp = producer.send_messages(topic, partition, 'msg {0}: {1}'.format(j, random_string(10))) if len(resp) > 0: self.assertEqual(resp[0].error, 0) logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j) From 522395f83711b2b8eca08135a6054de80c75d55e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 10:28:17 -0700 Subject: [PATCH 124/250] test_batched_simple_producer__triggers_by_message should wait for producer queue to drain before testing messages were sent --- test/test_producer_integration.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index e3f7767..329e4bc 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -251,6 +251,16 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # Batch mode is async. No ack self.assertEqual(len(resp), 0) + # Wait until producer has pulled all messages from internal queue + # this should signal that the first batch was sent, and the producer + # is now waiting for enough messages to batch again (or a timeout) + timeout = 5 + start = time.time() + while not producer.queue.empty(): + if time.time() - start > timeout: + self.fail('timeout waiting for producer queue to empty') + time.sleep(0.1) + # send messages groups all *msgs in a single call to the same partition # so we should see all messages from the first call in one partition self.assert_fetch_offset(partitions[0], start_offsets[0], [ From 1f23a4884553ec07ada9bfb67e475ab2520dee6e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:34:24 -0700 Subject: [PATCH 125/250] Sleep for 500ms before retrying consumer pending call in tests --- test/test_failover_integration.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 26da615..7cead86 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -201,6 +201,7 @@ class TestFailover(KafkaIntegrationTestCase): # Keep checking if it isn't immediately correct, subject to timeout while pending < check_count and (time.time() - started_at < timeout): pending = consumer.pending(partitions) + time.sleep(0.5) consumer.stop() client.close() From 679bf2f9271a064be6a083b80e6adfc15af1ca52 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 14:44:44 -0700 Subject: [PATCH 126/250] Small hanging indent style nits in some producer integration tests --- test/test_producer_integration.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 329e4bc..099b975 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -227,7 +227,8 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): random_start=False) # Send 4 messages -- should not trigger a batch - resp = producer.send_messages(self.topic, + resp = producer.send_messages( + self.topic, self.msg("one"), self.msg("two"), self.msg("three"), @@ -242,7 +243,8 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): self.assert_fetch_offset(partitions[1], start_offsets[1], []) # send 3 more messages -- should trigger batch on first 5 - resp = producer.send_messages(self.topic, + resp = producer.send_messages( + self.topic, self.msg("five"), self.msg("six"), self.msg("seven"), @@ -283,14 +285,16 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): start_offsets = [self.current_offset(self.topic, p) for p in partitions] batch_interval = 5 - producer = SimpleProducer(self.client, + producer = SimpleProducer( + self.client, batch_send=True, batch_send_every_n=100, batch_send_every_t=batch_interval, random_start=False) # Send 5 messages and do a fetch - resp = producer.send_messages(self.topic, + resp = producer.send_messages( + self.topic, self.msg("one"), self.msg("two"), self.msg("three"), From 3085faa5484123b5dd18aa96d0f331593228603d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Jun 2015 18:13:13 -0700 Subject: [PATCH 127/250] KafkaClient should try/except ConnnectionError when calling _get_conn --- kafka/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 4302ce5..63b33b3 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -169,19 +169,19 @@ class KafkaClient(object): responses_by_broker = collections.defaultdict(list) broker_failures = [] for broker, payloads in payloads_by_broker.items(): - conn = self._get_conn(broker.host.decode('utf-8'), broker.port) requestId = self._next_id() request = encoder_fn(client_id=self.client_id, correlation_id=requestId, payloads=payloads) # Send the request, recv the response try: + conn = self._get_conn(broker.host.decode('utf-8'), broker.port) conn.send(requestId, request) except ConnectionError as e: broker_failures.append(broker) log.warning("Could not send request [%s] to server %s: %s", - binascii.b2a_hex(request), conn, e) + binascii.b2a_hex(request), broker, e) for payload in payloads: responses_by_broker[broker].append(FailedPayloadsError(payload)) From 87bea900df341ddc2431569bdc329d069b6c5c61 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Jun 2015 19:00:01 -0700 Subject: [PATCH 128/250] Encode random_message strings to bytes in test_failover_integration --- test/test_failover_integration.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 7cead86..f260093 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -173,7 +173,8 @@ class TestFailover(KafkaIntegrationTestCase): def _send_random_messages(self, producer, topic, partition, n): for j in range(n): logging.debug('_send_random_message to %s:%d -- try %d', topic, partition, j) - resp = producer.send_messages(topic, partition, 'msg {0}: {1}'.format(j, random_string(10))) + msg = 'msg {0}: {1}'.format(j, random_string(10)) + resp = producer.send_messages(topic, partition, msg.encode('utf-8')) if len(resp) > 0: self.assertEqual(resp[0].error, 0) logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j) From 68f5506c20d936257f3c11aa12cb692c8c732ed0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:24:45 -0700 Subject: [PATCH 129/250] PR 331 fixup: Fix kafka.common imports from kafka.producer.base (one import block, add ProduceResponse) --- kafka/producer/base.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 5cbc521..2542df5 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -17,10 +17,9 @@ import six from kafka.common import ( ProduceRequest, ProduceResponse, TopicAndPartition, RetryOptions, kafka_errors, UnsupportedCodecError, FailedPayloadsError, - RequestTimedOutError, AsyncProducerQueueFull, UnknownError + RequestTimedOutError, AsyncProducerQueueFull, UnknownError, + RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES ) -from kafka.common import ( - RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES) from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set from kafka.util import kafka_bytestring From 6faa7c0e697b3096391453e50149c0dac59b05e0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:28:48 -0700 Subject: [PATCH 130/250] PR 331 fixup: fix _handle_error closure --- kafka/producer/base.py | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 2542df5..ef81a69 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -89,17 +89,18 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, continue reqs_to_retry, error_cls = [], None - do_backoff, do_refresh = False, False + retry_state = { + 'do_backoff': False, + 'do_refresh': False + } - def _handle_error(error_cls, reqs, all_retries): - if ((error_cls == RequestTimedOutError and - retry_options.retry_on_timeouts) or - error_cls in RETRY_ERROR_TYPES): - all_retries += reqs - if error_cls in RETRY_BACKOFF_ERROR_TYPES: - do_backoff = True - if error_cls in RETRY_REFRESH_ERROR_TYPES: - do_refresh = True + def _handle_error(error_cls, request): + if issubclass(error_cls, RETRY_ERROR_TYPES) or (retry_options.retry_on_timeouts and issubclass(error_cls, RequestTimedOutError)): + reqs_to_retry.append(request) + if issubclass(error_cls, RETRY_BACKOFF_ERROR_TYPES): + retry_state['do_backoff'] |= True + if issubclass(error_cls, RETRY_REFRESH_ERROR_TYPES): + retry_state['do_refresh'] |= True try: reply = client.send_produce_request(reqs.keys(), @@ -108,26 +109,26 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, fail_on_error=False) for i, response in enumerate(reply): if isinstance(response, FailedPayloadsError): - _handle_error(FailedPayloadsError, response.failed_payloads, reqs_to_retry) + _handle_error(FailedPayloadsError, response.payload) elif isinstance(response, ProduceResponse) and response.error: error_cls = kafka_errors.get(response.error, UnknownError) - _handle_error(error_cls, [reqs.keys()[i]], reqs_to_retry) + _handle_error(error_cls, reqs.keys()[i]) except Exception as ex: error_cls = kafka_errors.get(type(ex), UnknownError) - _handle_error(error_cls, reqs.keys(), reqs_to_retry) + _handle_error(error_cls, reqs.keys()) if not reqs_to_retry: reqs = {} continue # doing backoff before next retry - if do_backoff and retry_options.backoff_ms: + if retry_state['do_backoff'] and retry_options.backoff_ms: log.info("Doing backoff for %s(ms)." % retry_options.backoff_ms) time.sleep(float(retry_options.backoff_ms) / 1000) # refresh topic metadata before next retry - if do_refresh: + if retry_state['do_refresh']: client.load_metadata_for_topics() reqs = dict((key, count + 1) for (key, count) in reqs.items() From 3fdd7b78dcf2c29b72f301eec0bf71e74c1672b3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:29:46 -0700 Subject: [PATCH 131/250] PR 331 fixup: Support unlimited retries with async_retry_limit=None Async producer defaults are now retry everything always w/ 100ms backoff --- kafka/producer/base.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index ef81a69..05adb5e 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -32,10 +32,10 @@ BATCH_SEND_MSG_COUNT = 20 # unlimited ASYNC_QUEUE_MAXSIZE = 0 ASYNC_QUEUE_PUT_TIMEOUT = 0 -# no retries by default -ASYNC_RETRY_LIMIT = 0 -ASYNC_RETRY_BACKOFF_MS = 0 -ASYNC_RETRY_ON_TIMEOUTS = False +# unlimited retries by default +ASYNC_RETRY_LIMIT = None +ASYNC_RETRY_BACKOFF_MS = 100 +ASYNC_RETRY_ON_TIMEOUTS = True STOP_ASYNC_PRODUCER = -1 @@ -131,8 +131,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if retry_state['do_refresh']: client.load_metadata_for_topics() - reqs = dict((key, count + 1) for (key, count) in reqs.items() - if key in reqs_to_retry and count < retry_options.limit) + reqs = dict((key, count + 1) + for (key, count) in reqs.items() + if key in reqs_to_retry + and (retry_options.limit is None or (count < retry_options.limit))) class Producer(object): From 48e278941206815d680b4d4c81d7f1fd2637255c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 13:31:30 -0700 Subject: [PATCH 132/250] PR 331 fixup: log warnings on async producer backoff and metadata refresh --- kafka/producer/base.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 05adb5e..c1bc0c5 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -124,11 +124,12 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # doing backoff before next retry if retry_state['do_backoff'] and retry_options.backoff_ms: - log.info("Doing backoff for %s(ms)." % retry_options.backoff_ms) + log.warn('Async producer backoff for %s(ms) before retrying', retry_options.backoff_ms) time.sleep(float(retry_options.backoff_ms) / 1000) # refresh topic metadata before next retry if retry_state['do_refresh']: + log.warn('Async producer forcing metadata refresh metadata before retrying') client.load_metadata_for_topics() reqs = dict((key, count + 1) From d96a9b732ececb2f319e9e37ad4e040b366ce80b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 12:42:07 -0700 Subject: [PATCH 133/250] PR 331 fixup: Dont need try/except when calling send_produce_requests with fail_on_error=False --- kafka/producer/base.py | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index c1bc0c5..0bb0c81 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -102,21 +102,22 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if issubclass(error_cls, RETRY_REFRESH_ERROR_TYPES): retry_state['do_refresh'] |= True - try: - reply = client.send_produce_request(reqs.keys(), - acks=req_acks, - timeout=ack_timeout, - fail_on_error=False) - for i, response in enumerate(reply): - if isinstance(response, FailedPayloadsError): - _handle_error(FailedPayloadsError, response.payload) - elif isinstance(response, ProduceResponse) and response.error: - error_cls = kafka_errors.get(response.error, UnknownError) - _handle_error(error_cls, reqs.keys()[i]) + reply = client.send_produce_request(request_tries.keys(), + acks=req_acks, + timeout=ack_timeout, + fail_on_error=False) + for i, response in enumerate(reply): + error_cls = None + if isinstance(response, FailedPayloadsError): + error_cls = response.__class__ + orig_req = response.payload - except Exception as ex: - error_cls = kafka_errors.get(type(ex), UnknownError) - _handle_error(error_cls, reqs.keys()) + elif isinstance(response, ProduceResponse) and response.error: + error_cls = kafka_errors.get(response.error, UnknownError) + orig_req = request_tries.keys()[i] + + if error_cls: + _handle_error(error_cls, orig_req) if not reqs_to_retry: reqs = {} From 794ab5bba4807888839c2030d9b97422bddc3cc9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 12:44:39 -0700 Subject: [PATCH 134/250] PR 331 fixup: Rename reqs dict to request_tries --- kafka/producer/base.py | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 0bb0c81..15768be 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -47,7 +47,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, a specified timeout and send them upstream to the brokers in one request """ - reqs = {} + request_tries = {} client.reinit() while not stop_event.is_set(): @@ -55,7 +55,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # it's a simplification: we're comparing message sets and # messages: each set can contain [1..batch_size] messages - count = batch_size - len(reqs) + count = batch_size - len(request_tries) send_at = time.time() + timeout msgset = defaultdict(list) @@ -83,9 +83,9 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, req = ProduceRequest(topic_partition.topic, topic_partition.partition, tuple(messages)) - reqs[req] = 0 + request_tries[req] = 0 - if not reqs: + if not request_tries: continue reqs_to_retry, error_cls = [], None @@ -120,7 +120,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, _handle_error(error_cls, orig_req) if not reqs_to_retry: - reqs = {} + request_tries = {} continue # doing backoff before next retry @@ -133,10 +133,14 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, log.warn('Async producer forcing metadata refresh metadata before retrying') client.load_metadata_for_topics() - reqs = dict((key, count + 1) - for (key, count) in reqs.items() - if key in reqs_to_retry - and (retry_options.limit is None or (count < retry_options.limit))) + # Apply retry limit, dropping messages that are over + request_tries = dict( + (key, count + 1) + for (key, count) in request_tries.items() + if key in reqs_to_retry + and (retry_options.limit is None + or (count < retry_options.limit)) + ) class Producer(object): From 9712f613c9e7e4b0436f501b513249eab4edc4e9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Jun 2015 16:09:30 -0700 Subject: [PATCH 135/250] PR 331 fixup: do not attempt to get new messages if there are pending retries --- kafka/producer/base.py | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 15768be..2f47d87 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -52,13 +52,18 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, while not stop_event.is_set(): timeout = batch_time - - # it's a simplification: we're comparing message sets and - # messages: each set can contain [1..batch_size] messages - count = batch_size - len(request_tries) + count = batch_size send_at = time.time() + timeout msgset = defaultdict(list) + # Merging messages will require a bit more work to manage correctly + # for now, dont look for new batches if we have old ones to retry + if request_tries: + count = 0 + log.debug('Skipping new batch collection to handle retries') + else: + log.debug('Batching size: {0}, timeout: {1}'.format(count, timeout)) + # Keep fetching till we gather enough messages or a # timeout is reached while count > 0 and timeout >= 0: From 0f1579b047fc63c09596897cc1c83730bd0ddb94 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 12:50:43 -0700 Subject: [PATCH 136/250] Log retries and failed messages in async producer (configurable as full messages or hash()) --- kafka/producer/base.py | 36 ++++++++++++++++++++++++------------ 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 2f47d87..cd14ab6 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -36,12 +36,14 @@ ASYNC_QUEUE_PUT_TIMEOUT = 0 ASYNC_RETRY_LIMIT = None ASYNC_RETRY_BACKOFF_MS = 100 ASYNC_RETRY_ON_TIMEOUTS = True +ASYNC_LOG_MESSAGES_ON_ERROR = True STOP_ASYNC_PRODUCER = -1 def _send_upstream(queue, client, codec, batch_time, batch_size, - req_acks, ack_timeout, retry_options, stop_event): + req_acks, ack_timeout, retry_options, stop_event, + log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR): """ Listen on the queue for a specified number of messages or till a specified timeout and send them upstream to the brokers in one @@ -123,6 +125,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if error_cls: _handle_error(error_cls, orig_req) + log.error('Error sending ProduceRequest to %s:%d with msgs %s', + orig_req.topic, orig_req.partition, + orig_req.messages if log_messages_on_error + else hash(orig_req.messages)) if not reqs_to_retry: request_tries = {} @@ -147,6 +153,13 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, or (count < retry_options.limit)) ) + # Log messages we are going to retry + for orig_req in request_tries.keys(): + log.info('Retrying ProduceRequest to %s:%d with msgs %s', + orig_req.topic, orig_req.partition, + orig_req.messages if log_messages_on_error + else hash(orig_req.messages)) + class Producer(object): """ @@ -185,7 +198,8 @@ class Producer(object): async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS, async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, - async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): + async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT, + async_log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR): if batch_send: async = True @@ -218,16 +232,14 @@ class Producer(object): backoff_ms=async_retry_backoff_ms, retry_on_timeouts=async_retry_on_timeouts) self.thread_stop_event = Event() - self.thread = Thread(target=_send_upstream, - args=(self.queue, - self.client.copy(), - self.codec, - batch_send_every_t, - batch_send_every_n, - self.req_acks, - self.ack_timeout, - async_retry_options, - self.thread_stop_event)) + self.thread = Thread( + target=_send_upstream, + args=(self.queue, self.client.copy(), self.codec, + batch_send_every_t, batch_send_every_n, + self.req_acks, self.ack_timeout, + async_retry_options, self.thread_stop_event), + kwargs={'log_messages_on_error': async_log_messages_on_error} + ) # Thread will die if main thread exits self.thread.daemon = True From aa217e05448b4eced017b5ecdcb020a4411f863f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 14:12:14 -0700 Subject: [PATCH 137/250] Deprecate async producer batch_send kwarg -- use 'async' instead --- docs/usage.rst | 2 +- kafka/producer/base.py | 62 ++++++++++++++++++++----------- kafka/producer/keyed.py | 4 +- kafka/producer/simple.py | 4 +- test/test_producer.py | 17 --------- test/test_producer_integration.py | 4 +- 6 files changed, 47 insertions(+), 46 deletions(-) diff --git a/docs/usage.rst b/docs/usage.rst index 150d121..cdacfdc 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -47,7 +47,7 @@ SimpleProducer # Notes: # * If the producer dies before the messages are sent, there will be losses # * Call producer.stop() to send the messages and cleanup - producer = SimpleProducer(kafka, batch_send=True, + producer = SimpleProducer(kafka, async=True, batch_send_every_n=20, batch_send_every_t=60) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index cd14ab6..9f4942b 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -166,32 +166,54 @@ class Producer(object): Base class to be used by producers Arguments: - client: The Kafka client instance to use - async: If set to true, the messages are sent asynchronously via another - thread (process). We will not wait for a response to these - WARNING!!! current implementation of async producer does not - guarantee message delivery. Use at your own risk! Or help us - improve with a PR! - req_acks: A value indicating the acknowledgements that the server must - receive before responding to the request - ack_timeout: Value (in milliseconds) indicating a timeout for waiting - for an acknowledgement - batch_send: If True, messages are send in batches - batch_send_every_n: If set, messages are send in batches of this size - batch_send_every_t: If set, messages are send after this timeout - """ + client (KafkaClient): instance to use for broker communications. + codec (kafka.protocol.ALL_CODECS): compression codec to use. + req_acks (int, optional): A value indicating the acknowledgements that + the server must receive before responding to the request, + defaults to 1 (local ack). + ack_timeout (int, optional): millisecond timeout to wait for the + configured req_acks, defaults to 1000. + async (bool, optional): send message using a background thread, + defaults to False. + batch_send_every_n (int, optional): If async is True, messages are + sent in batches of this size, defaults to 20. + batch_send_every_t (int or float, optional): If async is True, + messages are sent immediately after this timeout in seconds, even + if there are fewer than batch_send_every_n, defaults to 20. + async_retry_limit (int, optional): number of retries for failed messages + or None for unlimited, defaults to None / unlimited. + async_retry_backoff_ms (int, optional): milliseconds to backoff on + failed messages, defaults to 100. + async_retry_on_timeouts (bool, optional): whether to retry on + RequestTimeoutError, defaults to True. + async_queue_maxsize (int, optional): limit to the size of the + internal message queue in number of messages (not size), defaults + to 0 (no limit). + async_queue_put_timeout (int or float, optional): timeout seconds + for queue.put in send_messages for async producers -- will only + apply if async_queue_maxsize > 0 and the queue is Full, + defaults to 0 (fail immediately on full queue). + async_log_messages_on_error (bool, optional): set to False and the + async producer will only log hash() contents on failed produce + requests, defaults to True (log full messages). Hash logging + will not allow you to identify the specific message that failed, + but it will allow you to match failures with retries. + Deprecated Arguments: + batch_send (bool, optional): If True, messages are sent by a background + thread in batches, defaults to False. Deprecated, use 'async' + """ ACK_NOT_REQUIRED = 0 # No ack is required ACK_AFTER_LOCAL_WRITE = 1 # Send response after it is written to log ACK_AFTER_CLUSTER_COMMIT = -1 # Send response after data is committed - DEFAULT_ACK_TIMEOUT = 1000 - def __init__(self, client, async=False, + def __init__(self, client, req_acks=ACK_AFTER_LOCAL_WRITE, ack_timeout=DEFAULT_ACK_TIMEOUT, codec=None, - batch_send=False, + async=False, + batch_send=False, # deprecated, use async batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, async_retry_limit=ASYNC_RETRY_LIMIT, @@ -201,14 +223,10 @@ class Producer(object): async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT, async_log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR): - if batch_send: - async = True + if async: assert batch_send_every_n > 0 assert batch_send_every_t > 0 assert async_queue_maxsize >= 0 - else: - batch_send_every_n = 1 - batch_send_every_t = 3600 self.client = client self.async = async diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 6bb2285..2de4dcc 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -49,8 +49,8 @@ class KeyedProducer(Producer): self.partitioner_class = partitioner self.partitioners = {} - super(KeyedProducer, self).__init__(client, async, req_acks, - ack_timeout, codec, batch_send, + super(KeyedProducer, self).__init__(client, req_acks, ack_timeout, + codec, async, batch_send, batch_send_every_n, batch_send_every_t, async_retry_limit, diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index 78cc21c..280a02e 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -54,8 +54,8 @@ class SimpleProducer(Producer): async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): self.partition_cycles = {} self.random_start = random_start - super(SimpleProducer, self).__init__(client, async, req_acks, - ack_timeout, codec, batch_send, + super(SimpleProducer, self).__init__(client, req_acks, ack_timeout, + codec, async, batch_send, batch_send_every_n, batch_send_every_t, async_retry_limit, diff --git a/test/test_producer.py b/test/test_producer.py index 85a5a2e..c12af02 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -56,23 +56,6 @@ class TestKafkaProducer(unittest.TestCase): producer.send_messages(topic, b'hi') assert client.send_produce_request.called - @patch('kafka.producer.base._send_upstream') - def test_producer_async_queue_overfilled_batch_send(self, mock): - queue_size = 2 - producer = Producer(MagicMock(), batch_send=True, - async_queue_maxsize=queue_size) - - topic = b'test-topic' - partition = 0 - message = b'test-message' - - with self.assertRaises(AsyncProducerQueueFull): - message_list = [message] * (queue_size + 1) - producer.send_messages(topic, partition, *message_list) - self.assertEqual(producer.queue.qsize(), queue_size) - for _ in xrange(producer.queue.qsize()): - producer.queue.get() - @patch('kafka.producer.base._send_upstream') def test_producer_async_queue_overfilled(self, mock): queue_size = 2 diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 099b975..3c414e1 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -221,7 +221,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): batch_interval = 5 producer = SimpleProducer( self.client, - batch_send=True, + async=True, batch_send_every_n=batch_messages, batch_send_every_t=batch_interval, random_start=False) @@ -287,7 +287,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): batch_interval = 5 producer = SimpleProducer( self.client, - batch_send=True, + async=True, batch_send_every_n=100, batch_send_every_t=batch_interval, random_start=False) From c76bc9dd179044811a5e0d3fde2e437f1ee6d46c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 14:45:44 -0700 Subject: [PATCH 138/250] Update Producer class docstring --- kafka/producer/base.py | 29 +++++++++++++++++++++++++---- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 9f4942b..a0bf47c 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -44,10 +44,31 @@ STOP_ASYNC_PRODUCER = -1 def _send_upstream(queue, client, codec, batch_time, batch_size, req_acks, ack_timeout, retry_options, stop_event, log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR): - """ - Listen on the queue for a specified number of messages or till - a specified timeout and send them upstream to the brokers in one - request + """Private method to manage producing messages asynchronously + + Listens on the queue for a specified number of messages or until + a specified timeout and then sends messages to the brokers in grouped + requests (one per broker). + + Messages placed on the queue should be tuples that conform to this format: + ((topic, partition), message, key) + + Currently does not mark messages with task_done. Do not attempt to join()! + + Arguments: + queue (threading.Queue): the queue from which to get messages + client (KafkaClient): instance to use for communicating with brokers + codec (kafka.protocol.ALL_CODECS): compression codec to use + batch_time (int): interval in seconds to send message batches + batch_size (int): count of messages that will trigger an immediate send + req_acks: required acks to use with ProduceRequests. see server protocol + ack_timeout: timeout to wait for required acks. see server protocol + retry_options (RetryOptions): settings for retry limits, backoff etc + stop_event (threading.Event): event to monitor for shutdown signal. + when this event is 'set', the producer will stop sending messages. + log_messages_on_error (bool, optional): log stringified message-contents + on any produce error, otherwise only log a hash() of the contents, + defaults to True. """ request_tries = {} client.reinit() From 2ba22bf4cebf5e25351816b38cd3cb70e2ea4cb8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 23:09:50 -0700 Subject: [PATCH 139/250] Dont stop async producer until all pending messages have been processed --- kafka/producer/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index a0bf47c..0fd742d 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -73,7 +73,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, request_tries = {} client.reinit() - while not stop_event.is_set(): + while not (stop_event.is_set() and queue.empty() and not request_tries): timeout = batch_time count = batch_size send_at = time.time() + timeout From 1d5f4b1f889737ef3ba04d8303a02a4957a2d183 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 5 Jun 2015 23:22:37 -0700 Subject: [PATCH 140/250] Add async_stop_timeout parameter to tune how long to let the producer keep trying to send messages before timing out. Log an error if async producer was stopped before all messages sent. --- kafka/producer/base.py | 29 ++++++++++++++++++++++++++--- 1 file changed, 26 insertions(+), 3 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 0fd742d..18af342 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -39,11 +39,13 @@ ASYNC_RETRY_ON_TIMEOUTS = True ASYNC_LOG_MESSAGES_ON_ERROR = True STOP_ASYNC_PRODUCER = -1 +ASYNC_STOP_TIMEOUT_SECS = 30 def _send_upstream(queue, client, codec, batch_time, batch_size, req_acks, ack_timeout, retry_options, stop_event, - log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR): + log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR, + stop_timeout=ASYNC_STOP_TIMEOUT_SECS): """Private method to manage producing messages asynchronously Listens on the queue for a specified number of messages or until @@ -69,11 +71,23 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, log_messages_on_error (bool, optional): log stringified message-contents on any produce error, otherwise only log a hash() of the contents, defaults to True. + stop_timeout (int or float, optional): number of seconds to continue + retrying messages after stop_event is set, defaults to 30. """ request_tries = {} client.reinit() + stop_at = None while not (stop_event.is_set() and queue.empty() and not request_tries): + + # Handle stop_timeout + if stop_event.is_set(): + if not stop_at: + stop_at = stop_timeout + time.time() + if time.time() > stop_at: + log.debug('Async producer stopping due to stop_timeout') + break + timeout = batch_time count = batch_size send_at = time.time() + timeout @@ -181,6 +195,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, orig_req.messages if log_messages_on_error else hash(orig_req.messages)) + if request_tries or not queue.empty(): + log.error('Stopped producer with {0} unsent messages' + .format(len(request_tries) + queue.qsize())) + class Producer(object): """ @@ -219,6 +237,9 @@ class Producer(object): requests, defaults to True (log full messages). Hash logging will not allow you to identify the specific message that failed, but it will allow you to match failures with retries. + async_stop_timeout (int or float, optional): seconds to continue + attempting to send queued messages after producer.stop(), + defaults to 30. Deprecated Arguments: batch_send (bool, optional): If True, messages are sent by a background @@ -242,7 +263,8 @@ class Producer(object): async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT, - async_log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR): + async_log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR, + async_stop_timeout=ASYNC_STOP_TIMEOUT_SECS): if async: assert batch_send_every_n > 0 @@ -277,7 +299,8 @@ class Producer(object): batch_send_every_t, batch_send_every_n, self.req_acks, self.ack_timeout, async_retry_options, self.thread_stop_event), - kwargs={'log_messages_on_error': async_log_messages_on_error} + kwargs={'log_messages_on_error': async_log_messages_on_error, + 'stop_timeout': async_stop_timeout} ) # Thread will die if main thread exits From b1aad92a2e7dfded5f57ebc497dccc5ad3c56781 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 7 Jun 2015 00:07:50 -0700 Subject: [PATCH 141/250] Respawn crashed services in test/service.py, rather than raise RuntimeError --- test/service.py | 38 ++++++++++++++++++++++++++------------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/test/service.py b/test/service.py index 4048c7c..7627322 100644 --- a/test/service.py +++ b/test/service.py @@ -40,19 +40,37 @@ class SpawnedService(threading.Thread): self.captured_stderr = [] self.should_die = threading.Event() + self.child = None + self.alive = False def run(self): self.run_with_handles() - def run_with_handles(self): + def _spawn(self): + if self.alive: return + if self.child and self.child.poll() is None: return + self.child = subprocess.Popen( self.args, env=self.env, bufsize=1, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - alive = True + self.alive = True + def _despawn(self): + self.child.terminate() + self.alive = False + for _ in range(50): + if self.child.poll() is not None: + self.child = None + break + time.sleep(0.1) + else: + self.child.kill() + + def run_with_handles(self): + self._spawn() while True: (rds, _, _) = select.select([self.child.stdout, self.child.stderr], [], [], 1) @@ -64,17 +82,13 @@ class SpawnedService(threading.Thread): line = self.child.stderr.readline() self.captured_stderr.append(line.decode('utf-8')) - if self.should_die.is_set(): - self.child.terminate() - alive = False + if self.child.poll() is not None: + self.dump_logs() + self._spawn() - poll_results = self.child.poll() - if poll_results is not None: - if not alive: - break - else: - self.dump_logs() - raise RuntimeError("Subprocess has died. Aborting. (args=%s)" % ' '.join(str(x) for x in self.args)) + if self.should_die.is_set(): + self._despawn() + break def dump_logs(self): log.critical('stderr') From 945fc048a8cc61e1a9390bd7a7fed371d2e23277 Mon Sep 17 00:00:00 2001 From: reAsOn2010 Date: Tue, 9 Jun 2015 04:20:16 +0800 Subject: [PATCH 142/250] try to fix uncaught FailedPayloadsError --- kafka/common.py | 2 ++ kafka/consumer/simple.py | 9 ++++++++- test/test_failover_integration.py | 8 ++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/kafka/common.py b/kafka/common.py index 2fdf7d2..66987ff 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -226,6 +226,8 @@ kafka_errors = dict([(x.errno, x) for x in _iter_broker_errors()]) def check_error(response): + if isinstance(response, Exception): + raise response if response.error: error_class = kafka_errors.get(response.error, UnknownError) raise error_class(response) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 2c2f820..88eeada 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -19,7 +19,7 @@ from kafka.common import ( FetchRequest, OffsetRequest, ConsumerFetchSizeTooSmall, ConsumerNoMoreData, UnknownTopicOrPartitionError, NotLeaderForPartitionError, - OffsetOutOfRangeError, check_error + OffsetOutOfRangeError, FailedPayloadsError, check_error ) from .base import ( Consumer, @@ -355,6 +355,13 @@ class SimpleConsumer(Consumer): # Retry this partition retry_partitions[resp.partition] = partitions[resp.partition] continue + except FailedPayloadsError as e: + log.warning("Failed payloads of %s" + "Resetting partition offset...", + e.payload) + # Retry this partition + retry_partitions[e.payload.partition] = partitions[e.payload.partition] + continue partition = resp.partition buffer_size = partitions[partition] diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index f260093..1d835e2 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -169,6 +169,14 @@ class TestFailover(KafkaIntegrationTestCase): msg = random_string(10).encode('utf-8') producer.send_messages(topic, key, msg) + @kafka_versions("all") + def test_switch_leader_simple_consumer(self): + producer = Producer(self.client, async=False) + consumer = SimpleConsumer(self.client, None, self.topic, partitions=None, auto_commit=False, iter_timeout=10) + self._send_random_messages(producer, self.topic, 0, 2) + consumer.get_messages() + self._kill_leader(self.topic, 0) + consumer.get_messages() def _send_random_messages(self, producer, topic, partition, n): for j in range(n): From c28c8a31c36696544d81495e0bf9e2c425ba3786 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 15:20:54 -0700 Subject: [PATCH 143/250] Add unit tests for SimpleConsumer error handling --- test/test_consumer.py | 75 ++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 74 insertions(+), 1 deletion(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index a3d09a8..08fd620 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -3,7 +3,12 @@ from mock import MagicMock, patch from . import unittest from kafka import SimpleConsumer, KafkaConsumer, MultiProcessConsumer -from kafka.common import KafkaConfigurationError +from kafka.common import ( + KafkaConfigurationError, FetchResponse, + FailedPayloadsError, OffsetAndMessage, + NotLeaderForPartitionError, UnknownTopicOrPartitionError +) + class TestKafkaConsumer(unittest.TestCase): def test_non_integer_partitions(self): @@ -14,6 +19,7 @@ class TestKafkaConsumer(unittest.TestCase): with self.assertRaises(KafkaConfigurationError): KafkaConsumer() + class TestMultiProcessConsumer(unittest.TestCase): def test_partition_list(self): client = MagicMock() @@ -22,3 +28,70 @@ class TestMultiProcessConsumer(unittest.TestCase): consumer = MultiProcessConsumer(client, 'testing-group', 'testing-topic', partitions=partitions) self.assertEqual(fetch_last_known_offsets.call_args[0], (partitions,) ) self.assertEqual(client.get_partition_ids_for_topic.call_count, 0) # pylint: disable=no-member + + def test_simple_consumer_failed_payloads(self): + client = MagicMock() + consumer = SimpleConsumer(client, group=None, + topic='topic', partitions=[0, 1], + auto_commit=False) + + def failed_payloads(payload): + return FailedPayloadsError(payload) + + client.send_fetch_request.side_effect = self.fail_requests_factory(failed_payloads) + + # This should not raise an exception + consumer.get_messages(5) + + def test_simple_consumer_leader_change(self): + client = MagicMock() + consumer = SimpleConsumer(client, group=None, + topic='topic', partitions=[0, 1], + auto_commit=False) + + # Mock so that only the first request gets a valid response + def not_leader(request): + return FetchResponse(request.topic, request.partition, + NotLeaderForPartitionError.errno, -1, ()) + + client.send_fetch_request.side_effect = self.fail_requests_factory(not_leader) + + # This should not raise an exception + consumer.get_messages(20) + + # client should have updated metadata + self.assertGreaterEqual(client.reset_topic_metadata.call_count, 1) + self.assertGreaterEqual(client.load_metadata_for_topics.call_count, 1) + + def test_simple_consumer_unknown_topic_partition(self): + client = MagicMock() + consumer = SimpleConsumer(client, group=None, + topic='topic', partitions=[0, 1], + auto_commit=False) + + # Mock so that only the first request gets a valid response + def unknown_topic_partition(request): + return FetchResponse(request.topic, request.partition, + UnknownTopicOrPartitionError.errno, -1, ()) + + client.send_fetch_request.side_effect = self.fail_requests_factory(unknown_topic_partition) + + # This should not raise an exception + with self.assertRaises(UnknownTopicOrPartitionError): + consumer.get_messages(20) + + @staticmethod + def fail_requests_factory(error_factory): + # Mock so that only the first request gets a valid response + def fail_requests(payloads, **kwargs): + responses = [ + FetchResponse(payloads[0].topic, payloads[0].partition, 0, 0, + (OffsetAndMessage( + payloads[0].offset + i, + "msg %d" % (payloads[0].offset + i)) + for i in range(10))), + ] + for failure in payloads[1:]: + responses.append(error_factory(failure)) + return responses + return fail_requests From b235ce89733e17997ad7192e10064d47da9ccfb4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 15:23:05 -0700 Subject: [PATCH 144/250] SimpleConsumer should refresh metadata and retry on leadership changes; only raise on UnknownTopicOrPartition --- kafka/consumer/simple.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 88eeada..384fa8e 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -344,9 +344,12 @@ class SimpleConsumer(Consumer): try: check_error(resp) - except (UnknownTopicOrPartitionError, NotLeaderForPartitionError): + except UnknownTopicOrPartitionError: self.client.reset_topic_metadata(resp.topic) raise + except NotLeaderForPartitionError: + self.client.reset_topic_metadata(resp.topic) + continue except OffsetOutOfRangeError: log.warning("OffsetOutOfRangeError for %s - %d. " "Resetting partition offset...", From a9ddf155e7f9c94be3c4b7508946158aa1900466 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 17:21:28 -0700 Subject: [PATCH 145/250] Cleanup imports in kafka/client and kafka/consumer --- kafka/client.py | 2 +- kafka/consumer/multiprocess.py | 17 ++++++++--------- kafka/consumer/simple.py | 27 +++++++++++++-------------- 3 files changed, 22 insertions(+), 24 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 63b33b3..63c9073 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -4,8 +4,8 @@ import copy import functools import logging import time -import kafka.common +import kafka.common from kafka.common import (TopicAndPartition, BrokerMetadata, ConnectionError, FailedPayloadsError, KafkaTimeoutError, KafkaUnavailableError, diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 8cec92d..d03eb95 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -1,22 +1,21 @@ from __future__ import absolute_import +from collections import namedtuple import logging +from multiprocessing import Process, Manager as MPManager +try: + from Queue import Empty, Full # python 3 +except ImportError: + from queue import Empty, Full # python 2 import time -from collections import namedtuple -from multiprocessing import Process, Manager as MPManager - -try: - from Queue import Empty, Full -except ImportError: # python 2 - from queue import Empty, Full - from .base import ( + Consumer, AUTO_COMMIT_MSG_COUNT, AUTO_COMMIT_INTERVAL, NO_MESSAGES_WAIT_TIME_SECONDS, FULL_QUEUE_WAIT_TIME_SECONDS ) -from .simple import Consumer, SimpleConsumer +from .simple import SimpleConsumer log = logging.getLogger(__name__) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 384fa8e..e4233ff 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -2,25 +2,18 @@ from __future__ import absolute_import try: from itertools import zip_longest as izip_longest, repeat # pylint: disable-msg=E0611 -except ImportError: # python 2 - from itertools import izip_longest as izip_longest, repeat +except ImportError: + from itertools import izip_longest as izip_longest, repeat # python 2 import logging +try: + from Queue import Empty, Queue # python 3 +except ImportError: + from queue import Empty, Queue # python 2 +import sys import time import six -import sys -try: - from Queue import Empty, Queue -except ImportError: # python 2 - from queue import Empty, Queue - -from kafka.common import ( - FetchRequest, OffsetRequest, - ConsumerFetchSizeTooSmall, ConsumerNoMoreData, - UnknownTopicOrPartitionError, NotLeaderForPartitionError, - OffsetOutOfRangeError, FailedPayloadsError, check_error -) from .base import ( Consumer, FETCH_DEFAULT_BLOCK_TIMEOUT, @@ -33,6 +26,12 @@ from .base import ( ITER_TIMEOUT_SECONDS, NO_MESSAGES_WAIT_TIME_SECONDS ) +from ..common import ( + FetchRequest, OffsetRequest, + ConsumerFetchSizeTooSmall, ConsumerNoMoreData, + UnknownTopicOrPartitionError, NotLeaderForPartitionError, + OffsetOutOfRangeError, FailedPayloadsError, check_error +) log = logging.getLogger(__name__) From c0fb0de7c4ced45dae4e09cdc660ebc741e11af3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 17:23:56 -0700 Subject: [PATCH 146/250] Dont maintain all producer args / kwargs in subclass __init__ and docstrings -- just refer to super class (Producer) --- kafka/producer/keyed.py | 53 +++++++----------------------------- kafka/producer/simple.py | 58 +++++++++------------------------------- 2 files changed, 22 insertions(+), 89 deletions(-) diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index 5fe9b12..a5a26c9 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -3,14 +3,10 @@ from __future__ import absolute_import import logging import warnings -from kafka.partitioner import HashedPartitioner -from kafka.util import kafka_bytestring +from .base import Producer +from ..partitioner import HashedPartitioner +from ..util import kafka_bytestring -from .base import ( - Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT, - ASYNC_RETRY_LIMIT, ASYNC_RETRY_BACKOFF_MS, ASYNC_RETRY_ON_TIMEOUTS -) log = logging.getLogger(__name__) @@ -19,46 +15,17 @@ class KeyedProducer(Producer): """ A producer which distributes messages to partitions based on the key - Arguments: - client: The kafka client instance + See Producer class for Arguments - Keyword Arguments: + Additional Arguments: partitioner: A partitioner class that will be used to get the partition - to send the message to. Must be derived from Partitioner - async: If True, the messages are sent asynchronously via another - thread (process). We will not wait for a response to these - ack_timeout: Value (in milliseconds) indicating a timeout for waiting - for an acknowledgement - batch_send: If True, messages are send in batches - batch_send_every_n: If set, messages are send in batches of this size - batch_send_every_t: If set, messages are send after this timeout + to send the message to. Must be derived from Partitioner. + Defaults to HashedPartitioner. """ - def __init__(self, client, partitioner=None, async=False, - req_acks=Producer.ACK_AFTER_LOCAL_WRITE, - ack_timeout=Producer.DEFAULT_ACK_TIMEOUT, - codec=None, - batch_send=False, - batch_send_every_n=BATCH_SEND_MSG_COUNT, - batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - async_retry_limit=ASYNC_RETRY_LIMIT, - async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS, - async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, - async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, - async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): - if not partitioner: - partitioner = HashedPartitioner - self.partitioner_class = partitioner + def __init__(self, *args, **kwargs): + self.partitioner_class = kwargs.pop('partitioner', HashedPartitioner) self.partitioners = {} - - super(KeyedProducer, self).__init__(client, req_acks, ack_timeout, - codec, async, batch_send, - batch_send_every_n, - batch_send_every_t, - async_retry_limit, - async_retry_backoff_ms, - async_retry_on_timeouts, - async_queue_maxsize, - async_queue_put_timeout) + super(KeyedProducer, self).__init__(*args, **kwargs) def _next_partition(self, topic, key): if topic not in self.partitioners: diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index 280a02e..13e60d9 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -1,68 +1,34 @@ from __future__ import absolute_import +from itertools import cycle import logging import random import six -from itertools import cycle - from six.moves import xrange -from .base import ( - Producer, BATCH_SEND_DEFAULT_INTERVAL, - BATCH_SEND_MSG_COUNT, ASYNC_QUEUE_MAXSIZE, ASYNC_QUEUE_PUT_TIMEOUT, - ASYNC_RETRY_LIMIT, ASYNC_RETRY_BACKOFF_MS, ASYNC_RETRY_ON_TIMEOUTS -) +from .base import Producer + log = logging.getLogger(__name__) class SimpleProducer(Producer): - """ - A simple, round-robin producer. Each message goes to exactly one partition + """A simple, round-robin producer. - Arguments: - client: The Kafka client instance to use + See Producer class for Base Arguments - Keyword Arguments: - async: If True, the messages are sent asynchronously via another - thread (process). We will not wait for a response to these - req_acks: A value indicating the acknowledgements that the server must - receive before responding to the request - ack_timeout: Value (in milliseconds) indicating a timeout for waiting - for an acknowledgement - batch_send: If True, messages are send in batches - batch_send_every_n: If set, messages are send in batches of this size - batch_send_every_t: If set, messages are send after this timeout - random_start: If true, randomize the initial partition which the + Additional Arguments: + random_start (bool, optional): randomize the initial partition which the first message block will be published to, otherwise if false, the first message block will always publish - to partition 0 before cycling through each partition + to partition 0 before cycling through each partition, + defaults to True. """ - def __init__(self, client, async=False, - req_acks=Producer.ACK_AFTER_LOCAL_WRITE, - ack_timeout=Producer.DEFAULT_ACK_TIMEOUT, - codec=None, - batch_send=False, - batch_send_every_n=BATCH_SEND_MSG_COUNT, - batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL, - random_start=True, - async_retry_limit=ASYNC_RETRY_LIMIT, - async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS, - async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS, - async_queue_maxsize=ASYNC_QUEUE_MAXSIZE, - async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT): + def __init__(self, *args, **kwargs): self.partition_cycles = {} - self.random_start = random_start - super(SimpleProducer, self).__init__(client, req_acks, ack_timeout, - codec, async, batch_send, - batch_send_every_n, - batch_send_every_t, - async_retry_limit, - async_retry_backoff_ms, - async_retry_on_timeouts, - async_queue_maxsize, - async_queue_put_timeout) + self.random_start = kwargs.pop('random_start', True) + super(SimpleProducer, self).__init__(*args, **kwargs) def _next_partition(self, topic): if topic not in self.partition_cycles: From 0dc6663d24f6b9386ac2119a4a11836391e5da65 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 17:25:45 -0700 Subject: [PATCH 147/250] Add a few extra docstring comments about thread-safe clients/connections --- kafka/client.py | 7 +++++-- kafka/conn.py | 8 +++++--- kafka/producer/base.py | 2 ++ 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 63c9073..20e20f2 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -254,8 +254,11 @@ class KafkaClient(object): def copy(self): """ - Create an inactive copy of the client object - A reinit() has to be done on the copy before it can be used again + Create an inactive copy of the client object, suitable for passing + to a separate thread. + + Note that the copied connections are not initialized, so reinit() must + be called on the returned copy. """ c = copy.deepcopy(self) for key in c.conns: diff --git a/kafka/conn.py b/kafka/conn.py index 7a49d8c..432e10b 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -161,9 +161,11 @@ class KafkaConnection(local): def copy(self): """ - Create an inactive copy of the connection object - A reinit() has to be done on the copy before it can be used again - return a new KafkaConnection object + Create an inactive copy of the connection object, suitable for + passing to a background thread. + + The returned copy is not connected; you must call reinit() before + using. """ c = copy.deepcopy(self) # Python 3 doesn't copy custom attributes of the threadlocal subclass diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 18af342..e0c086b 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -206,6 +206,8 @@ class Producer(object): Arguments: client (KafkaClient): instance to use for broker communications. + If async=True, the background thread will use client.copy(), + which is expected to return a thread-safe object. codec (kafka.protocol.ALL_CODECS): compression codec to use. req_acks (int, optional): A value indicating the acknowledgements that the server must receive before responding to the request, From fa997e2ee105cbdacc146fd03e6cac8a5c6cef72 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 17:33:26 -0700 Subject: [PATCH 148/250] Prefer single quotes for strings --- kafka/client.py | 26 ++++++++++++-------------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 20e20f2..18327ee 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -22,7 +22,7 @@ log = logging.getLogger(__name__) class KafkaClient(object): - CLIENT_ID = b"kafka-python" + CLIENT_ID = b'kafka-python' # NOTE: The timeout given to the client should always be greater than the # one passed to SimpleConsumer.get_message(), otherwise you can get a @@ -50,7 +50,7 @@ class KafkaClient(object): ################## def _get_conn(self, host, port): - "Get or create a connection to a broker using host and port" + """Get or create a connection to a broker using host and port""" host_key = (host, port) if host_key not in self.conns: self.conns[host_key] = KafkaConnection( @@ -122,10 +122,10 @@ class KafkaClient(object): return decoder_fn(response) except Exception: - log.exception("Could not send request [%r] to server %s:%i, " - "trying next server" % (requestId, host, port)) + log.exception('Error sending request [%s] to server %s:%s, ' + 'trying next server', requestId, host, port) - raise KafkaUnavailableError("All servers failed to process request") + raise KafkaUnavailableError('All servers failed to process request') def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): """ @@ -180,7 +180,7 @@ class KafkaClient(object): except ConnectionError as e: broker_failures.append(broker) - log.warning("Could not send request [%s] to server %s: %s", + log.warning('Could not send request [%s] to server %s: %s', binascii.b2a_hex(request), broker, e) for payload in payloads: @@ -201,15 +201,14 @@ class KafkaClient(object): response = conn.recv(requestId) except ConnectionError as e: broker_failures.append(broker) - log.warning("Could not receive response to request [%s] " - "from server %s: %s", + log.warning('Could not receive response to request [%s] ' + 'from server %s: %s', binascii.b2a_hex(request), conn, e) for payload in payloads: responses_by_broker[broker].append(FailedPayloadsError(payload)) else: - for payload_response in decoder_fn(response): responses_by_broker[broker].append(payload_response) @@ -300,7 +299,7 @@ class KafkaClient(object): while not self.has_metadata_for_topic(topic): if time.time() > start_time + timeout: - raise KafkaTimeoutError("Unable to create topic {0}".format(topic)) + raise KafkaTimeoutError('Unable to create topic {0}'.format(topic)) try: self.load_metadata_for_topics(topic) except LeaderNotAvailableError: @@ -348,8 +347,8 @@ class KafkaClient(object): resp = self.send_metadata_request(topics) - log.debug("Received new broker metadata: %s", resp.brokers) - log.debug("Received new topic metadata: %s", resp.topics) + log.debug('Received new broker metadata: %s', resp.brokers) + log.debug('Received new topic metadata: %s', resp.topics) self.brokers = dict([(broker.nodeId, broker) for broker in resp.brokers]) @@ -368,7 +367,7 @@ class KafkaClient(object): raise # Otherwise, just log a warning - log.error("Error loading topic metadata for %s: %s", topic, type(e)) + log.error('Error loading topic metadata for %s: %s', topic, type(e)) continue self.topic_partitions[topic] = {} @@ -409,7 +408,6 @@ class KafkaClient(object): def send_metadata_request(self, payloads=[], fail_on_error=True, callback=None): - encoder = KafkaProtocol.encode_metadata_request decoder = KafkaProtocol.decode_metadata_response From 7889d605c41d6d0b0fcdd7359464f016df78771f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 17:39:43 -0700 Subject: [PATCH 149/250] Improve kafka client debug request/response logging --- kafka/client.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 18327ee..2ef22b3 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -111,6 +111,7 @@ class KafkaClient(object): """ for (host, port) in self.hosts: requestId = self._next_id() + log.debug('Request %s: %s', requestId, payloads) try: conn = self._get_conn(host, port) request = encoder_fn(client_id=self.client_id, @@ -119,7 +120,9 @@ class KafkaClient(object): conn.send(requestId, request) response = conn.recv(requestId) - return decoder_fn(response) + decoded = decoder_fn(response) + log.debug('Response %s: %s', requestId, decoded) + return decoded except Exception: log.exception('Error sending request [%s] to server %s:%s, ' @@ -150,9 +153,6 @@ class KafkaClient(object): List of response objects in the same order as the supplied payloads """ - - log.debug("Sending Payloads: %s" % payloads) - # Group the requests by topic+partition brokers_for_payloads = [] payloads_by_broker = collections.defaultdict(list) @@ -170,6 +170,7 @@ class KafkaClient(object): broker_failures = [] for broker, payloads in payloads_by_broker.items(): requestId = self._next_id() + log.debug('Request %s to %s: %s', requestId, broker, payloads) request = encoder_fn(client_id=self.client_id, correlation_id=requestId, payloads=payloads) @@ -222,7 +223,6 @@ class KafkaClient(object): # Return responses in the same order as provided responses_by_payload = [responses_by_broker[broker].pop(0) for broker in brokers_for_payloads] - log.debug('Responses: %s' % responses_by_payload) return responses_by_payload def __repr__(self): From 81abf094dcdfbbe78e55ee519b35658fefa410ef Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 18:53:44 -0700 Subject: [PATCH 150/250] Fixup for loop vars in kafka/protocol.py --- kafka/protocol.py | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/kafka/protocol.py b/kafka/protocol.py index f12e6a3..d5adf89 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -232,12 +232,12 @@ class KafkaProtocol(object): """ ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - for i in range(num_topics): + for _ in range(num_topics): ((strlen,), cur) = relative_unpack('>h', data, cur) topic = data[cur:cur + strlen] cur += strlen ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): + for _ in range(num_partitions): ((partition, error, offset), cur) = relative_unpack('>ihq', data, cur) @@ -289,11 +289,11 @@ class KafkaProtocol(object): """ ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - for i in range(num_topics): + for _ in range(num_topics): (topic, cur) = read_short_string(data, cur) ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): + for j in range(num_partitions): ((partition, error, highwater_mark_offset), cur) = \ relative_unpack('>ihq', data, cur) @@ -337,16 +337,16 @@ class KafkaProtocol(object): """ ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - for i in range(num_topics): + for _ in range(num_topics): (topic, cur) = read_short_string(data, cur) ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): + for _ in range(num_partitions): ((partition, error, num_offsets,), cur) = \ relative_unpack('>ihi', data, cur) offsets = [] - for j in range(num_offsets): + for k in range(num_offsets): ((offset,), cur) = relative_unpack('>q', data, cur) offsets.append(offset) @@ -392,7 +392,7 @@ class KafkaProtocol(object): # Broker info brokers = [] - for i in range(numbrokers): + for _ in range(numbrokers): ((nodeId, ), cur) = relative_unpack('>i', data, cur) (host, cur) = read_short_string(data, cur) ((port,), cur) = relative_unpack('>i', data, cur) @@ -402,13 +402,13 @@ class KafkaProtocol(object): ((num_topics,), cur) = relative_unpack('>i', data, cur) topic_metadata = [] - for i in range(num_topics): + for _ in range(num_topics): ((topic_error,), cur) = relative_unpack('>h', data, cur) (topic_name, cur) = read_short_string(data, cur) ((num_partitions,), cur) = relative_unpack('>i', data, cur) partition_metadata = [] - for j in range(num_partitions): + for _ in range(num_partitions): ((partition_error_code, partition, leader, numReplicas), cur) = \ relative_unpack('>hiii', data, cur) @@ -471,11 +471,11 @@ class KafkaProtocol(object): ((correlation_id,), cur) = relative_unpack('>i', data, 0) ((num_topics,), cur) = relative_unpack('>i', data, cur) - for i in xrange(num_topics): + for _ in xrange(num_topics): (topic, cur) = read_short_string(data, cur) ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in xrange(num_partitions): + for _ in xrange(num_partitions): ((partition, error), cur) = relative_unpack('>ih', data, cur) yield OffsetCommitResponse(topic, partition, error) @@ -522,11 +522,11 @@ class KafkaProtocol(object): ((correlation_id,), cur) = relative_unpack('>i', data, 0) ((num_topics,), cur) = relative_unpack('>i', data, cur) - for i in range(num_topics): + for _ in range(num_topics): (topic, cur) = read_short_string(data, cur) ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): + for _ in range(num_partitions): ((partition, offset), cur) = relative_unpack('>iq', data, cur) (metadata, cur) = read_short_string(data, cur) ((error,), cur) = relative_unpack('>h', data, cur) From d262107aa870a72a15f7da097c116a8c6dcea0cd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 21:05:40 -0700 Subject: [PATCH 151/250] Retry with shorter 5sec timeout when trying to open() fixtures - this is intended to reduce flapping tests caused by intermittent - fixture startup issues on travis-ci --- test/fixtures.py | 14 ++++++++++---- test/service.py | 7 ++++--- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/test/fixtures.py b/test/fixtures.py index 90d01f1..4231452 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -126,8 +126,11 @@ class ZookeeperFixture(Fixture): # Party! self.out("Starting...") - self.child.start() - self.child.wait_for(r"binding to port") + while True: + self.child.start() + if self.child.wait_for(r"binding to port", timeout=5): + break + self.child.stop() self.out("Done!") def close(self): @@ -222,8 +225,11 @@ class KafkaFixture(Fixture): self.out("Done!") self.out("Starting...") - self.child.start() - self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id) + while True: + self.child.start() + if self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id, timeout=5): + break + self.child.stop() self.out("Done!") self.running = True diff --git a/test/service.py b/test/service.py index 7627322..9368b85 100644 --- a/test/service.py +++ b/test/service.py @@ -110,14 +110,15 @@ class SpawnedService(threading.Thread): log.exception("Received exception when killing child process") self.dump_logs() - raise RuntimeError("Waiting for %r timed out after %d seconds" % (pattern, timeout)) + log.error("Waiting for %r timed out after %d seconds", pattern, timeout) + return False if re.search(pattern, '\n'.join(self.captured_stdout), re.IGNORECASE) is not None: log.info("Found pattern %r in %d seconds via stdout", pattern, (t2 - t1)) - return + return True if re.search(pattern, '\n'.join(self.captured_stderr), re.IGNORECASE) is not None: log.info("Found pattern %r in %d seconds via stderr", pattern, (t2 - t1)) - return + return True time.sleep(0.1) def start(self): From ac1ef729594b192931c56e1ab97d76fca7e08a24 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 21:06:48 -0700 Subject: [PATCH 152/250] Dump fixture logs on OffsetResponse error during producer integration tests. This is intended to help debug an intermittent failure that requires server logs. --- test/testutil.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/test/testutil.py b/test/testutil.py index 1f1a1df..b9827a8 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -79,9 +79,15 @@ class KafkaIntegrationTestCase(unittest.TestCase): self.client.close() def current_offset(self, topic, partition): - offsets, = self.client.send_offset_request([ OffsetRequest(kafka_bytestring(topic), - partition, -1, 1) ]) - return offsets.offsets[0] + try: + offsets, = self.client.send_offset_request([ OffsetRequest(kafka_bytestring(topic), partition, -1, 1) ]) + except: + # XXX: We've seen some UnknownErrors here and cant debug w/o server logs + self.zk.child.dump_logs() + self.server.child.dump_logs() + raise + else: + return offsets.offsets[0] def msgs(self, iterable): return [ self.msg(x) for x in iterable ] From ffcb49d71a5ef1871a2631216db5198bbf8a7b41 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 21:26:31 -0700 Subject: [PATCH 153/250] Add zk to KafkaIntegrationTestCase attributes to fix lint error --- test/testutil.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test/testutil.py b/test/testutil.py index b9827a8..99d8d01 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -51,6 +51,7 @@ class KafkaIntegrationTestCase(unittest.TestCase): create_client = True topic = None bytes_topic = None + zk = None server = None def setUp(self): From 712377ab15ae7c1c0b031df310d60235b7c57cae Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 21:26:58 -0700 Subject: [PATCH 154/250] Import style nits in integration tests --- test/test_client_integration.py | 5 ++--- test/test_consumer_integration.py | 8 ++++++-- test/test_failover_integration.py | 3 +-- test/test_producer_integration.py | 1 + 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/test/test_client_integration.py b/test/test_client_integration.py index d963d85..585123b 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -6,9 +6,8 @@ from kafka.common import ( ) from test.fixtures import ZookeeperFixture, KafkaFixture -from test.testutil import ( - KafkaIntegrationTestCase, kafka_versions -) +from test.testutil import KafkaIntegrationTestCase, kafka_versions + class TestKafkaClientIntegration(KafkaIntegrationTestCase): @classmethod diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 3825f94..c202c5c 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -3,9 +3,12 @@ import os from six.moves import xrange -from kafka import SimpleConsumer, MultiProcessConsumer, KafkaConsumer, create_message +from kafka import ( + KafkaConsumer, MultiProcessConsumer, SimpleConsumer, create_message +) from kafka.common import ( - ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout, OffsetOutOfRangeError + ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout, + OffsetOutOfRangeError ) from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES @@ -14,6 +17,7 @@ from test.testutil import ( KafkaIntegrationTestCase, kafka_versions, random_string, Timer ) + class TestConsumerIntegration(KafkaIntegrationTestCase): @classmethod def setUpClass(cls): diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 1d835e2..5082d7c 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -4,10 +4,9 @@ import time from . import unittest -from kafka import KafkaClient, SimpleConsumer +from kafka import KafkaClient, SimpleConsumer, KeyedProducer from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer -from kafka.producer import KeyedProducer from kafka.util import kafka_bytestring from test.fixtures import ZookeeperFixture, KafkaFixture diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 3439f4e..abf34c3 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -19,6 +19,7 @@ from kafka.producer.base import Producer from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import KafkaIntegrationTestCase, kafka_versions + class TestKafkaProducerIntegration(KafkaIntegrationTestCase): @classmethod From b998fc7376272fc16ea4c3242d4f009f234ef85b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 22:20:16 -0700 Subject: [PATCH 155/250] Update KafkaConsumer to handle request-specific FailedPayloadsErrors --- kafka/consumer/kafka.py | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 7ba83cb..b141a98 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -323,19 +323,21 @@ class KafkaConsumer(object): max_bytes) for (topic, partition) in self._topics] - # 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 + # send_fetch_request will batch topic/partition requests by leader + 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: + + if isinstance(resp, FailedPayloadsError): + logger.warning('FailedPayloadsError attempting to fetch data') + self._refresh_metadata_on_error() + continue + topic = kafka_bytestring(resp.topic) partition = resp.partition try: From 25d5a523570cc3e286439e6296755e8746fa3982 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 22:27:31 -0700 Subject: [PATCH 156/250] KafkaConsumer style nits --- kafka/consumer/kafka.py | 72 +++++++++++++++++++++++++++-------------- 1 file changed, 47 insertions(+), 25 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index b141a98..11c4221 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -120,7 +120,10 @@ class KafkaConsumer(object): 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)') + raise KafkaConfigurationError( + 'KafkaConsumer configured to auto-commit ' + 'without required consumer group (group_id)' + ) # Check auto-commit configuration if self._config['auto_commit_enable']: @@ -128,12 +131,15 @@ class KafkaConsumer(object): self._reset_auto_commit() if not self._config['bootstrap_servers']: - raise KafkaConfigurationError('bootstrap_servers required to ' - 'configure KafkaConsumer') + raise KafkaConfigurationError( + 'bootstrap_servers required to configure KafkaConsumer' + ) - self._client = KafkaClient(self._config['bootstrap_servers'], - client_id=self._config['client_id'], - timeout=(self._config['socket_timeout_ms'] / 1000.0)) + self._client = KafkaClient( + self._config['bootstrap_servers'], + client_id=self._config['client_id'], + timeout=(self._config['socket_timeout_ms'] / 1000.0) + ) def set_topic_partitions(self, *topics): """ @@ -163,12 +169,12 @@ class KafkaConsumer(object): # 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 + # Consume topic1-0 starting at offset 12, and topic2-1 at offset 45 # using tuples -- - kafka.set_topic_partitions(("topic1", 0, 123), ("topic2", 1, 456)) + kafka.set_topic_partitions(("topic1", 0, 12), ("topic2", 1, 45)) # using dict -- - kafka.set_topic_partitions({ ("topic1", 0): 123, ("topic2", 1): 456 }) + kafka.set_topic_partitions({ ("topic1", 0): 12, ("topic2", 1): 45 }) """ self._topics = [] @@ -216,8 +222,10 @@ class KafkaConsumer(object): for partition in value: self._consume_topic_partition(topic, partition) else: - raise KafkaConfigurationError('Unknown topic type (dict key must be ' - 'int or list/tuple of ints)') + raise KafkaConfigurationError( + 'Unknown topic type ' + '(dict key must be int or list/tuple of ints)' + ) # (topic, partition): offset elif isinstance(key, tuple): @@ -316,7 +324,9 @@ class KafkaConsumer(object): raise KafkaConfigurationError('No topics or partitions configured') if not self._offsets.fetch: - raise KafkaConfigurationError('No fetch offsets found when calling fetch_messages') + raise KafkaConfigurationError( + 'No fetch offsets found when calling fetch_messages' + ) fetches = [FetchRequest(topic, partition, self._offsets.fetch[(topic, partition)], @@ -383,7 +393,8 @@ class KafkaConsumer(object): logger.debug('message offset less than fetched offset ' 'skipping: %s', msg) continue - # Only increment fetch offset if we safely got the message and deserialized + # Only increment fetch offset + # if we safely got the message and deserialized self._offsets.fetch[(topic, partition)] = offset + 1 # Then yield to user @@ -396,10 +407,12 @@ class KafkaConsumer(object): topic (str): topic for offset request partition (int): partition for offset request 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. + 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. max_num_offsets (int): Maximum offsets to include in the OffsetResponse Returns: @@ -499,7 +512,10 @@ class KafkaConsumer(object): """ 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)') + 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 @@ -523,13 +539,17 @@ class KafkaConsumer(object): if commit_offset == self._offsets.commit[topic_partition]: continue - commits.append(OffsetCommitRequest(topic_partition[0], topic_partition[1], 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']) - resps = self._client.send_offset_commit_request(kafka_bytestring(self._config['group_id']), - commits, - fail_on_error=False) + resps = self._client.send_offset_commit_request( + kafka_bytestring(self._config['group_id']), commits, + fail_on_error=False + ) for r in resps: check_error(r) @@ -726,9 +746,11 @@ class KafkaConsumer(object): # def __repr__(self): - return '' % ', '.join(["%s-%d" % topic_partition - for topic_partition in - self._topics]) + return '<{0} topics=({1})>'.format( + self.__class__.__name__, + '|'.join(["%s-%d" % topic_partition + for topic_partition in self._topics]) + ) # # other private methods From b22ac148aa75e8e07075404bbfe98193641c1827 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 23:04:50 -0700 Subject: [PATCH 157/250] Fail fast in nosetests --- tox.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tox.ini b/tox.ini index e3e8568..fcb8908 100644 --- a/tox.ini +++ b/tox.ini @@ -11,7 +11,7 @@ deps = mock python-snappy commands = - nosetests {posargs:-v --with-id --id-file={envdir}/.noseids --with-timer --timer-top-n 10 --with-coverage --cover-erase --cover-package kafka} + nosetests {posargs:-v -x --with-id --id-file={envdir}/.noseids --with-timer --timer-top-n 10 --with-coverage --cover-erase --cover-package kafka} setenv = PROJECT_ROOT = {toxinidir} passenv = KAFKA_VERSION From 432590550d745eb1eda49ac12d2f8a3dca01111d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 23:05:55 -0700 Subject: [PATCH 158/250] Reduce log chatter in tests (only debug kafka.conn in test_conn) --- test/test_conn.py | 5 +++++ test/testutil.py | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/test/test_conn.py b/test/test_conn.py index c4f219b..6e47cc8 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -1,3 +1,4 @@ +import logging import socket import struct from threading import Thread @@ -10,6 +11,10 @@ from kafka.conn import KafkaConnection, collect_hosts, DEFAULT_SOCKET_TIMEOUT_SE class ConnTest(unittest.TestCase): def setUp(self): + + # kafka.conn debug logging is verbose, so only enable in conn tests + logging.getLogger('kafka.conn').setLevel(logging.DEBUG) + self.config = { 'host': 'localhost', 'port': 9090, diff --git a/test/testutil.py b/test/testutil.py index 99d8d01..3a1d2ba 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -113,3 +113,8 @@ class Timer(object): self.interval = self.end - self.start logging.basicConfig(level=logging.DEBUG) +logging.getLogger('test.fixtures').setLevel(logging.ERROR) +logging.getLogger('test.service').setLevel(logging.ERROR) + +# kafka.conn debug logging is verbose, disable in tests by default +logging.getLogger('kafka.conn').setLevel(logging.INFO) From ddb536d87e7c6514d33a8b783cd955af05ed9b2f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 23:12:32 -0700 Subject: [PATCH 159/250] Reduce blocking times in consumer integration tests --- test/test_consumer_integration.py | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index c202c5c..8911e3e 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -170,11 +170,11 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def test_simple_consumer_blocking(self): consumer = self.consumer() - # Ask for 5 messages, nothing in queue, block 5 seconds + # Ask for 5 messages, nothing in queue, block 1 second with Timer() as t: - messages = consumer.get_messages(block=True, timeout=5) + messages = consumer.get_messages(block=True, timeout=1) self.assert_message_count(messages, 0) - self.assertGreaterEqual(t.interval, 5) + self.assertGreaterEqual(t.interval, 1) self.send_messages(0, range(0, 10)) @@ -184,11 +184,11 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assert_message_count(messages, 5) self.assertLessEqual(t.interval, 1) - # Ask for 10 messages, get 5 back, block 5 seconds + # Ask for 10 messages, get 5 back, block 1 second with Timer() as t: - messages = consumer.get_messages(count=10, block=True, timeout=5) + messages = consumer.get_messages(count=10, block=True, timeout=1) self.assert_message_count(messages, 5) - self.assertGreaterEqual(t.interval, 5) + self.assertGreaterEqual(t.interval, 1) consumer.stop() @@ -236,12 +236,12 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def test_multi_process_consumer_blocking(self): consumer = self.consumer(consumer = MultiProcessConsumer) - # Ask for 5 messages, No messages in queue, block 5 seconds + # Ask for 5 messages, No messages in queue, block 1 second with Timer() as t: - messages = consumer.get_messages(block=True, timeout=5) + messages = consumer.get_messages(block=True, timeout=1) self.assert_message_count(messages, 0) - self.assertGreaterEqual(t.interval, 5) + self.assertGreaterEqual(t.interval, 1) # Send 10 messages self.send_messages(0, range(0, 10)) @@ -252,11 +252,11 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assert_message_count(messages, 5) self.assertLessEqual(t.interval, 1) - # Ask for 10 messages, 5 in queue, block 5 seconds + # Ask for 10 messages, 5 in queue, block 1 second with Timer() as t: - messages = consumer.get_messages(count=10, block=True, timeout=5) + messages = consumer.get_messages(count=10, block=True, timeout=1) self.assert_message_count(messages, 5) - self.assertGreaterEqual(t.interval, 4.95) + self.assertGreaterEqual(t.interval, 1) consumer.stop() @@ -450,7 +450,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer = self.kafka_consumer(auto_offset_reset='smallest', consumer_timeout_ms=TIMEOUT_MS) - # Ask for 5 messages, nothing in queue, block 5 seconds + # Ask for 5 messages, nothing in queue, block 500ms with Timer() as t: with self.assertRaises(ConsumerTimeout): msg = consumer.next() @@ -467,7 +467,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assertEqual(len(messages), 5) self.assertLess(t.interval, TIMEOUT_MS / 1000.0 ) - # Ask for 10 messages, get 5 back, block 5 seconds + # Ask for 10 messages, get 5 back, block 500ms messages = set() with Timer() as t: with self.assertRaises(ConsumerTimeout): From 53d8251a18d9c033269e105854a7c4cc9730930a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 23:13:46 -0700 Subject: [PATCH 160/250] Produce messages to both partitions in async producer leader switch test --- test/test_failover_integration.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 5082d7c..91e22cf 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -98,10 +98,14 @@ class TestFailover(KafkaIntegrationTestCase): # Test the base class Producer -- send_messages to a specific partition producer = Producer(self.client, async=True, - req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT) + batch_send_every_n=15, + batch_send_every_t=3, + req_acks=Producer.ACK_AFTER_CLUSTER_COMMIT, + async_log_messages_on_error=False) # Send 10 random messages self._send_random_messages(producer, topic, partition, 10) + self._send_random_messages(producer, topic, partition + 1, 10) # kill leader for partition self._kill_leader(topic, partition) @@ -110,9 +114,11 @@ class TestFailover(KafkaIntegrationTestCase): # in async mode, this should return immediately producer.send_messages(topic, partition, b'success') + producer.send_messages(topic, partition + 1, b'success') # send to new leader self._send_random_messages(producer, topic, partition, 10) + self._send_random_messages(producer, topic, partition + 1, 10) # Stop the producer and wait for it to shutdown producer.stop() @@ -129,6 +135,8 @@ class TestFailover(KafkaIntegrationTestCase): # Should be equal to 10 before + 1 recovery + 10 after self.assert_message_count(topic, 21, partitions=(partition,), at_least=True) + self.assert_message_count(topic, 21, partitions=(partition + 1,), + at_least=True) @kafka_versions("all") def test_switch_leader_keyed_producer(self): From 0e416d528767a1246c72ae1efa917e401cedd79f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 23:26:51 -0700 Subject: [PATCH 161/250] Dont try to terminate a child that has already exited in test/service _despawn --- test/service.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/service.py b/test/service.py index 9368b85..b986a71 100644 --- a/test/service.py +++ b/test/service.py @@ -59,7 +59,8 @@ class SpawnedService(threading.Thread): self.alive = True def _despawn(self): - self.child.terminate() + if self.child.poll() is None: + self.child.terminate() self.alive = False for _ in range(50): if self.child.poll() is not None: From c75b84eb7a7e81947e4d785dc871fee05e350476 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 09:56:38 -0700 Subject: [PATCH 162/250] Add Unit test for async producer leader change; return ProduceResponse on success --- test/test_producer.py | 66 ++++++++++++++++++++++++++++++++++++------- 1 file changed, 56 insertions(+), 10 deletions(-) diff --git a/test/test_producer.py b/test/test_producer.py index c12af02..c7bdfdb 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -1,15 +1,17 @@ # -*- coding: utf-8 -*- -import time +import collections import logging +import time from mock import MagicMock, patch from . import unittest -from kafka.common import TopicAndPartition, FailedPayloadsError, RetryOptions -from kafka.common import AsyncProducerQueueFull -from kafka.producer.base import Producer -from kafka.producer.base import _send_upstream +from kafka.common import ( + AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError, + ProduceResponse, RetryOptions, TopicAndPartition +) +from kafka.producer.base import Producer, _send_upstream from kafka.protocol import CODEC_NONE import threading @@ -122,12 +124,21 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): for i in range(10): self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + # Mock offsets counter for closure + offsets = collections.defaultdict(lambda: collections.defaultdict(lambda: 0)) self.client.is_first_time = True def send_side_effect(reqs, *args, **kwargs): if self.client.is_first_time: self.client.is_first_time = False return [FailedPayloadsError(req) for req in reqs] - return [] + responses = [] + for req in reqs: + offset = offsets[req.topic][req.partition] + offsets[req.topic][req.partition] += len(req.messages) + responses.append( + ProduceResponse(req.topic, req.partition, 0, offset) + ) + return responses self.client.send_produce_request.side_effect = send_side_effect @@ -136,8 +147,8 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # the queue should be void at the end of the test self.assertEqual(self.queue.empty(), True) - # there should be 5 non-void cals: 1st failed batch of 3 msgs - # + 3 batches of 3 msgs each + 1 batch of 1 msg = 1 + 3 + 1 = 5 + # there should be 5 non-void calls: 1st failed batch of 3 msgs + # plus 3 batches of 3 msgs each + 1 batch of 1 message self.assertEqual(self.client.send_produce_request.call_count, 5) def test_with_limited_retries(self): @@ -157,11 +168,46 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # the queue should be void at the end of the test self.assertEqual(self.queue.empty(), True) - # there should be 16 non-void cals: + # there should be 16 non-void calls: # 3 initial batches of 3 msgs each + 1 initial batch of 1 msg + - # 3 retries of the batches above = 4 + 3 * 4 = 16, all failed + # 3 retries of the batches above = (1 + 3 retries) * 4 batches = 16 self.assertEqual(self.client.send_produce_request.call_count, 16) + def test_async_producer_not_leader(self): + + for i in range(10): + self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + + # Mock offsets counter for closure + offsets = collections.defaultdict(lambda: collections.defaultdict(lambda: 0)) + self.client.is_first_time = True + def send_side_effect(reqs, *args, **kwargs): + if self.client.is_first_time: + self.client.is_first_time = False + return [ProduceResponse(req.topic, req.partition, + NotLeaderForPartitionError.errno, -1) + for req in reqs] + + responses = [] + for req in reqs: + offset = offsets[req.topic][req.partition] + offsets[req.topic][req.partition] += len(req.messages) + responses.append( + ProduceResponse(req.topic, req.partition, 0, offset) + ) + return responses + + self.client.send_produce_request.side_effect = send_side_effect + + self._run_process(2) + + # the queue should be void at the end of the test + self.assertEqual(self.queue.empty(), True) + + # there should be 5 non-void calls: 1st failed batch of 3 msgs + # + 3 batches of 3 msgs each + 1 batch of 1 msg = 1 + 3 + 1 = 5 + self.assertEqual(self.client.send_produce_request.call_count, 5) + def tearDown(self): for _ in xrange(self.queue.qsize()): self.queue.get() From 63ecdbd32edc5e0cc989722d9cdd02d384beffb0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 11:29:14 -0700 Subject: [PATCH 163/250] Use a list, not request_tries.keys(), to track requests / responses in async producer --- kafka/producer/base.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index e0c086b..498539d 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -144,10 +144,12 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if issubclass(error_cls, RETRY_REFRESH_ERROR_TYPES): retry_state['do_refresh'] |= True - reply = client.send_produce_request(request_tries.keys(), + requests = list(request_tries.keys()) + reply = client.send_produce_request(requests, acks=req_acks, timeout=ack_timeout, fail_on_error=False) + for i, response in enumerate(reply): error_cls = None if isinstance(response, FailedPayloadsError): @@ -156,7 +158,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, elif isinstance(response, ProduceResponse) and response.error: error_cls = kafka_errors.get(response.error, UnknownError) - orig_req = request_tries.keys()[i] + orig_req = requests[i] if error_cls: _handle_error(error_cls, orig_req) From f6be28372cffc6ddc675905f67bf3aa1f1716bf9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 16:37:40 -0700 Subject: [PATCH 164/250] Create a new child thread on each open fixture try --- test/fixtures.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/test/fixtures.py b/test/fixtures.py index 4231452..d4d03ee 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -122,11 +122,11 @@ class ZookeeperFixture(Fixture): # Configure Zookeeper child process args = self.kafka_run_class_args("org.apache.zookeeper.server.quorum.QuorumPeerMain", properties) env = self.kafka_run_class_env() - self.child = SpawnedService(args, env) # Party! self.out("Starting...") while True: + self.child = SpawnedService(args, env) self.child.start() if self.child.wait_for(r"binding to port", timeout=5): break @@ -202,11 +202,6 @@ class KafkaFixture(Fixture): properties = os.path.join(self.tmp_dir, "kafka.properties") self.render_template(template, properties, vars(self)) - # Configure Kafka child process - args = self.kafka_run_class_args("kafka.Kafka", properties) - env = self.kafka_run_class_env() - self.child = SpawnedService(args, env) - # Party! self.out("Creating Zookeeper chroot node...") args = self.kafka_run_class_args("org.apache.zookeeper.ZooKeeperMain", @@ -225,7 +220,13 @@ class KafkaFixture(Fixture): self.out("Done!") self.out("Starting...") + + # Configure Kafka child process + args = self.kafka_run_class_args("kafka.Kafka", properties) + env = self.kafka_run_class_env() + while True: + self.child = SpawnedService(args, env) self.child.start() if self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id, timeout=5): break From 99bcf078c23fbc1e17add1620b34f3177861846b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 18:07:57 -0700 Subject: [PATCH 165/250] Support sync_fail_on_error kwarg in Producer --- kafka/producer/base.py | 14 ++++++++++++-- test/test_producer.py | 20 ++++++++++++++++++-- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 498539d..824ef5d 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -41,6 +41,8 @@ ASYNC_LOG_MESSAGES_ON_ERROR = True STOP_ASYNC_PRODUCER = -1 ASYNC_STOP_TIMEOUT_SECS = 30 +SYNC_FAIL_ON_ERROR_DEFAULT = True + def _send_upstream(queue, client, codec, batch_time, batch_size, req_acks, ack_timeout, retry_options, stop_event, @@ -216,6 +218,9 @@ class Producer(object): defaults to 1 (local ack). ack_timeout (int, optional): millisecond timeout to wait for the configured req_acks, defaults to 1000. + sync_fail_on_error (bool, optional): whether sync producer should + raise exceptions (True), or just return errors (False), + defaults to True. async (bool, optional): send message using a background thread, defaults to False. batch_send_every_n (int, optional): If async is True, messages are @@ -258,6 +263,7 @@ class Producer(object): req_acks=ACK_AFTER_LOCAL_WRITE, ack_timeout=DEFAULT_ACK_TIMEOUT, codec=None, + sync_fail_on_error=SYNC_FAIL_ON_ERROR_DEFAULT, async=False, batch_send=False, # deprecated, use async batch_send_every_n=BATCH_SEND_MSG_COUNT, @@ -316,6 +322,8 @@ class Producer(object): obj.stop() self._cleanup_func = cleanup atexit.register(cleanup, self) + else: + self.sync_fail_on_error = sync_fail_on_error def send_messages(self, topic, partition, *msg): """ @@ -373,8 +381,10 @@ class Producer(object): messages = create_message_set([(m, key) for m in msg], self.codec, key) req = ProduceRequest(topic, partition, messages) try: - resp = self.client.send_produce_request([req], acks=self.req_acks, - timeout=self.ack_timeout) + resp = self.client.send_produce_request( + [req], acks=self.req_acks, timeout=self.ack_timeout, + fail_on_error=self.sync_fail_on_error + ) except Exception: log.exception("Unable to send messages") raise diff --git a/test/test_producer.py b/test/test_producer.py index c7bdfdb..27272f6 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -7,6 +7,7 @@ import time from mock import MagicMock, patch from . import unittest +from kafka import KafkaClient, SimpleProducer from kafka.common import ( AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError, ProduceResponse, RetryOptions, TopicAndPartition @@ -44,8 +45,6 @@ class TestKafkaProducer(unittest.TestCase): producer.send_messages(topic, partition, m) def test_topic_message_types(self): - from kafka.producer.simple import SimpleProducer - client = MagicMock() def partitions(topic): @@ -75,6 +74,23 @@ class TestKafkaProducer(unittest.TestCase): for _ in xrange(producer.queue.qsize()): producer.queue.get() + def test_producer_sync_fail_on_error(self): + error = FailedPayloadsError('failure') + with patch.object(KafkaClient, 'load_metadata_for_topics'): + with patch.object(KafkaClient, 'get_partition_ids_for_topic', return_value=[0, 1]): + with patch.object(KafkaClient, '_send_broker_aware_request', return_value = [error]): + + client = KafkaClient(MagicMock()) + producer = SimpleProducer(client, async=False, sync_fail_on_error=False) + + # This should not raise + (response,) = producer.send_messages('foobar', b'test message') + self.assertEqual(response, error) + + producer = SimpleProducer(client, async=False, sync_fail_on_error=True) + with self.assertRaises(FailedPayloadsError): + producer.send_messages('foobar', b'test message') + class TestKafkaProducerSendUpstream(unittest.TestCase): From fc30855953f3dfd46a80e65e17283102085392e5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 20:25:13 -0700 Subject: [PATCH 166/250] Reset kafka.conn logger level in tearDown --- test/test_conn.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/test/test_conn.py b/test/test_conn.py index 6e47cc8..2b70344 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -50,6 +50,11 @@ class ConnTest(unittest.TestCase): # Reset any mock counts caused by __init__ self.MockCreateConn.reset_mock() + def tearDown(self): + # Return connection logging to INFO + logging.getLogger('kafka.conn').setLevel(logging.INFO) + + def test_collect_hosts__happy_path(self): hosts = "localhost:1234,localhost" results = collect_hosts(hosts) @@ -172,6 +177,14 @@ class ConnTest(unittest.TestCase): class TestKafkaConnection(unittest.TestCase): + def setUp(self): + # kafka.conn debug logging is verbose, so only enable in conn tests + logging.getLogger('kafka.conn').setLevel(logging.DEBUG) + + def tearDown(self): + # Return connection logging to INFO + logging.getLogger('kafka.conn').setLevel(logging.INFO) + @mock.patch('socket.create_connection') def test_copy(self, socket): """KafkaConnection copies work as expected""" From 0a048a3422977cdfabb79c0f702bf090d65c73f5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 20:25:45 -0700 Subject: [PATCH 167/250] Add thread id to logging format to assist with debugging test failures --- tox.ini | 1 + 1 file changed, 1 insertion(+) diff --git a/tox.ini b/tox.ini index fcb8908..a69dc99 100644 --- a/tox.ini +++ b/tox.ini @@ -13,6 +13,7 @@ deps = commands = nosetests {posargs:-v -x --with-id --id-file={envdir}/.noseids --with-timer --timer-top-n 10 --with-coverage --cover-erase --cover-package kafka} setenv = + NOSE_LOGFORMAT = %(asctime)s - %(thread)d - %(name)s - %(levelname)s - %(message)s PROJECT_ROOT = {toxinidir} passenv = KAFKA_VERSION From bedaafa8046ed172b64f8ea4d444fcd8dbb8ae92 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 21:56:35 -0700 Subject: [PATCH 168/250] KafkaClient: log requestId on ConnectionError instead of hexdump of request contents --- kafka/client.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 2ef22b3..6b07acd 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -1,4 +1,3 @@ -import binascii import collections import copy import functools @@ -181,8 +180,8 @@ class KafkaClient(object): except ConnectionError as e: broker_failures.append(broker) - log.warning('Could not send request [%s] to server %s: %s', - binascii.b2a_hex(request), broker, e) + log.warning('ConnectionError attempting to send request %s ' + 'to server %s: %s', requestId, broker, e) for payload in payloads: responses_by_broker[broker].append(FailedPayloadsError(payload)) @@ -202,9 +201,9 @@ class KafkaClient(object): response = conn.recv(requestId) except ConnectionError as e: broker_failures.append(broker) - log.warning('Could not receive response to request [%s] ' - 'from server %s: %s', - binascii.b2a_hex(request), conn, e) + log.warning('ConnectionError attempting to receive a ' + 'response to request %s from server %s: %s', + requestId, broker, e) for payload in payloads: responses_by_broker[broker].append(FailedPayloadsError(payload)) From d6585483e0851e85a639b954b59807606792f516 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 21:57:26 -0700 Subject: [PATCH 169/250] KafkaClient: log responses by requestId for debugging --- kafka/client.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index 6b07acd..5c869b9 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -189,10 +189,12 @@ class KafkaClient(object): # No exception, try to get response else: - # decoder_fn=None signal that the server is expected to not + # decoder_fn=None signal that the server is expected to not # send a response. This probably only applies to # ProduceRequest w/ acks = 0 if decoder_fn is None: + log.debug('Request %s does not expect a response ' + '(skipping conn.recv)', requestId) for payload in payloads: responses_by_broker[broker].append(None) continue @@ -211,6 +213,7 @@ class KafkaClient(object): else: for payload_response in decoder_fn(response): responses_by_broker[broker].append(payload_response) + log.debug('Response %s: %s', requestId, responses_by_broker[broker]) # Connection errors generally mean stale metadata # although sometimes it means incorrect api request From 7b2f98f5ec2fa606b8c29e902d84a8e8dc486681 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 21:58:19 -0700 Subject: [PATCH 170/250] KafkaClient log new broker and topic metadata received as INFO --- kafka/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 5c869b9..da86175 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -349,8 +349,8 @@ class KafkaClient(object): resp = self.send_metadata_request(topics) - log.debug('Received new broker metadata: %s', resp.brokers) - log.debug('Received new topic metadata: %s', resp.topics) + log.info('Updating broker metadata: %s', resp.brokers) + log.info('Updating topic metadata: %s', resp.topics) self.brokers = dict([(broker.nodeId, broker) for broker in resp.brokers]) From 8b0a598045c010756e79d059736de06423b728c5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 22:28:54 -0700 Subject: [PATCH 171/250] Add send/receive debug logging to async producer --- kafka/producer/base.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 824ef5d..49090bd 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -101,7 +101,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, count = 0 log.debug('Skipping new batch collection to handle retries') else: - log.debug('Batching size: {0}, timeout: {1}'.format(count, timeout)) + log.debug('Batching size: %s, timeout: %s', count, timeout) # Keep fetching till we gather enough messages or a # timeout is reached @@ -147,12 +147,14 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, retry_state['do_refresh'] |= True requests = list(request_tries.keys()) - reply = client.send_produce_request(requests, - acks=req_acks, - timeout=ack_timeout, - fail_on_error=False) + log.debug('Sending: %s', requests) + responses = client.send_produce_request(requests, + acks=req_acks, + timeout=ack_timeout, + fail_on_error=False) - for i, response in enumerate(reply): + log.debug('Received: %s', responses) + for i, response in enumerate(responses): error_cls = None if isinstance(response, FailedPayloadsError): error_cls = response.__class__ @@ -164,7 +166,8 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if error_cls: _handle_error(error_cls, orig_req) - log.error('Error sending ProduceRequest to %s:%d with msgs %s', + log.error('Error sending ProduceRequest (#%d of %d) to %s:%d ' + 'with msgs %s', i + 1, len(requests), orig_req.topic, orig_req.partition, orig_req.messages if log_messages_on_error else hash(orig_req.messages)) From 57b9c75df61d1ec6edb5430e75a7e3a80cae76b5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 22:59:52 -0700 Subject: [PATCH 172/250] Test failover integration with 3-brokers / replicas / partitions --- test/test_failover_integration.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 91e22cf..f156152 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -23,10 +23,10 @@ class TestFailover(KafkaIntegrationTestCase): return zk_chroot = random_string(10) - replicas = 2 - partitions = 2 + replicas = 3 + partitions = 3 - # mini zookeeper, 2 kafka brokers + # mini zookeeper, 3 kafka brokers self.zk = ZookeeperFixture.instance() kk_args = [self.zk.host, self.zk.port, zk_chroot, replicas, partitions] self.brokers = [KafkaFixture.instance(i, *kk_args) for i in range(replicas)] From 4c9a3c6b9dac952154cdab2e11892bff240f9c91 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 00:56:19 -0700 Subject: [PATCH 173/250] Reduce replica timeouts to 1sec for failover integration testing --- servers/0.8.0/resources/kafka.properties | 4 ++++ servers/0.8.1.1/resources/kafka.properties | 4 ++++ servers/0.8.1/resources/kafka.properties | 4 ++++ servers/0.8.2.0/resources/kafka.properties | 4 ++++ 4 files changed, 16 insertions(+) diff --git a/servers/0.8.0/resources/kafka.properties b/servers/0.8.0/resources/kafka.properties index c9fd552..1aafd46 100644 --- a/servers/0.8.0/resources/kafka.properties +++ b/servers/0.8.0/resources/kafka.properties @@ -35,6 +35,10 @@ log.dirs={tmp_dir}/data num.partitions={partitions} default.replication.factor={replicas} +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + ############################# Log Flush Policy ############################# log.flush.interval.messages=10000 diff --git a/servers/0.8.1.1/resources/kafka.properties b/servers/0.8.1.1/resources/kafka.properties index a638f39..a59ef60 100644 --- a/servers/0.8.1.1/resources/kafka.properties +++ b/servers/0.8.1.1/resources/kafka.properties @@ -63,6 +63,10 @@ log.dirs={tmp_dir}/data num.partitions={partitions} default.replication.factor={replicas} +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + ############################# Log Flush Policy ############################# # Messages are immediately written to the filesystem but by default we only fsync() to sync diff --git a/servers/0.8.1/resources/kafka.properties b/servers/0.8.1/resources/kafka.properties index 5d47520..af67a91 100644 --- a/servers/0.8.1/resources/kafka.properties +++ b/servers/0.8.1/resources/kafka.properties @@ -35,6 +35,10 @@ log.dirs={tmp_dir}/data num.partitions={partitions} default.replication.factor={replicas} +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + ############################# Log Flush Policy ############################# log.flush.interval.messages=10000 diff --git a/servers/0.8.2.0/resources/kafka.properties b/servers/0.8.2.0/resources/kafka.properties index a638f39..a59ef60 100644 --- a/servers/0.8.2.0/resources/kafka.properties +++ b/servers/0.8.2.0/resources/kafka.properties @@ -63,6 +63,10 @@ log.dirs={tmp_dir}/data num.partitions={partitions} default.replication.factor={replicas} +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + ############################# Log Flush Policy ############################# # Messages are immediately written to the filesystem but by default we only fsync() to sync From 61172651082365a8d3dc244d531d0c02e888a138 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 00:48:19 -0700 Subject: [PATCH 174/250] Add KafkaClient test to show that request / response ordering is broken --- test/test_client_integration.py | 32 +++++++++++++++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/test/test_client_integration.py b/test/test_client_integration.py index 585123b..a6ea8f7 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -2,8 +2,9 @@ import os from kafka.common import ( FetchRequest, OffsetCommitRequest, OffsetFetchRequest, - KafkaTimeoutError + KafkaTimeoutError, ProduceRequest ) +from kafka.protocol import create_message from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import KafkaIntegrationTestCase, kafka_versions @@ -49,6 +50,35 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): with self.assertRaises(KafkaTimeoutError): self.client.ensure_topic_exists(b"this_topic_doesnt_exist", timeout=0) + @kafka_versions('all') + def test_send_produce_request_maintains_request_response_order(self): + + self.client.ensure_topic_exists(b'foo', timeout=1) + self.client.ensure_topic_exists(b'bar', timeout=1) + + requests = [ + ProduceRequest( + b'foo', 0, + [create_message(b'a'), create_message(b'b')]), + ProduceRequest( + b'bar', 1, + [create_message(b'a'), create_message(b'b')]), + ProduceRequest( + b'foo', 1, + [create_message(b'a'), create_message(b'b')]), + ProduceRequest( + b'bar', 0, + [create_message(b'a'), create_message(b'b')]), + ] + + responses = self.client.send_produce_request(requests) + while len(responses): + request = requests.pop() + response = responses.pop() + self.assertEqual(request.topic, response.topic) + self.assertEqual(request.partition, response.partition) + + #################### # Offset Tests # #################### From 66b6b4aa6ee7c4461a4e43b2512e76ba3f04230f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 00:48:36 -0700 Subject: [PATCH 175/250] Fix KafkaClient request / response ordering --- kafka/client.py | 30 ++++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index da86175..1bd8587 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -138,7 +138,8 @@ class KafkaClient(object): Arguments: payloads: list of object-like entities with a topic (str) and - partition (int) attribute + partition (int) attribute; payloads with duplicate topic-partitions + are not supported. encode_fn: a method to encode the list of payloads to a request body, must accept client_id, correlation_id, and payloads as @@ -152,6 +153,10 @@ class KafkaClient(object): List of response objects in the same order as the supplied payloads """ + # encoders / decoders do not maintain ordering currently + # so we need to keep this so we can rebuild order before returning + original_ordering = [(p.topic, p.partition) for p in payloads] + # Group the requests by topic+partition brokers_for_payloads = [] payloads_by_broker = collections.defaultdict(list) @@ -165,7 +170,7 @@ class KafkaClient(object): # For each broker, send the list of request payloads # and collect the responses and errors - responses_by_broker = collections.defaultdict(list) + responses = {} broker_failures = [] for broker, payloads in payloads_by_broker.items(): requestId = self._next_id() @@ -184,7 +189,8 @@ class KafkaClient(object): 'to server %s: %s', requestId, broker, e) for payload in payloads: - responses_by_broker[broker].append(FailedPayloadsError(payload)) + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = FailedPayloadsError(payload) # No exception, try to get response else: @@ -196,7 +202,8 @@ class KafkaClient(object): log.debug('Request %s does not expect a response ' '(skipping conn.recv)', requestId) for payload in payloads: - responses_by_broker[broker].append(None) + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = None continue try: @@ -208,12 +215,17 @@ class KafkaClient(object): requestId, broker, e) for payload in payloads: - responses_by_broker[broker].append(FailedPayloadsError(payload)) + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = FailedPayloadsError(payload) else: + _resps = [] for payload_response in decoder_fn(response): - responses_by_broker[broker].append(payload_response) - log.debug('Response %s: %s', requestId, responses_by_broker[broker]) + topic_partition = (payload_response.topic, + payload_response.partition) + responses[topic_partition] = payload_response + _resps.append(payload_response) + log.debug('Response %s: %s', requestId, _resps) # Connection errors generally mean stale metadata # although sometimes it means incorrect api request @@ -223,9 +235,7 @@ class KafkaClient(object): self.reset_all_metadata() # Return responses in the same order as provided - responses_by_payload = [responses_by_broker[broker].pop(0) - for broker in brokers_for_payloads] - return responses_by_payload + return [responses[tp] for tp in original_ordering] def __repr__(self): return '' % (self.client_id) From ded2ac7d321bab02c2b9fb3d8b03a60d6b9a5f84 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 07:50:36 -0700 Subject: [PATCH 176/250] Remove 1 sec timeout on topic creation in client response ordering test --- test/test_client_integration.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/test_client_integration.py b/test/test_client_integration.py index a6ea8f7..baaf79e 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -53,8 +53,8 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions('all') def test_send_produce_request_maintains_request_response_order(self): - self.client.ensure_topic_exists(b'foo', timeout=1) - self.client.ensure_topic_exists(b'bar', timeout=1) + self.client.ensure_topic_exists(b'foo') + self.client.ensure_topic_exists(b'bar') requests = [ ProduceRequest( From 8984141aefecedd260485047760c70fdc171c74f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 23:18:03 -0700 Subject: [PATCH 177/250] Log response error type in async producer --- kafka/producer/base.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 49090bd..3c826cd 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -166,8 +166,9 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, if error_cls: _handle_error(error_cls, orig_req) - log.error('Error sending ProduceRequest (#%d of %d) to %s:%d ' - 'with msgs %s', i + 1, len(requests), + log.error('%s sending ProduceRequest (#%d of %d) ' + 'to %s:%d with msgs %s', + error_cls.__name__, (i + 1), len(requests), orig_req.topic, orig_req.partition, orig_req.messages if log_messages_on_error else hash(orig_req.messages)) From 043e9bb04f1149ab8f9f8942ee591227f659128d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 10:01:30 -0700 Subject: [PATCH 178/250] Retry failed messages in failover integration tests; use module logger --- test/test_failover_integration.py | 29 +++++++++++++++++------------ 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index f156152..e27f12b 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -2,8 +2,6 @@ import logging import os import time -from . import unittest - from kafka import KafkaClient, SimpleConsumer, KeyedProducer from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer @@ -15,6 +13,9 @@ from test.testutil import ( ) +log = logging.getLogger(__name__) + + class TestFailover(KafkaIntegrationTestCase): create_client = False @@ -73,12 +74,12 @@ class TestFailover(KafkaIntegrationTestCase): timeout = 60 while not recovered and (time.time() - started) < timeout: try: - logging.debug("attempting to send 'success' message after leader killed") + log.debug("attempting to send 'success' message after leader killed") producer.send_messages(topic, partition, b'success') - logging.debug("success!") + log.debug("success!") recovered = True except (FailedPayloadsError, ConnectionError): - logging.debug("caught exception sending message -- will retry") + log.debug("caught exception sending message -- will retry") continue # Verify we successfully sent the message @@ -110,7 +111,7 @@ class TestFailover(KafkaIntegrationTestCase): # kill leader for partition self._kill_leader(topic, partition) - logging.debug("attempting to send 'success' message after leader killed") + log.debug("attempting to send 'success' message after leader killed") # in async mode, this should return immediately producer.send_messages(topic, partition, b'success') @@ -164,7 +165,7 @@ class TestFailover(KafkaIntegrationTestCase): if producer.partitioners[kafka_bytestring(topic)].partition(key) == 0: recovered = True except (FailedPayloadsError, ConnectionError): - logging.debug("caught exception sending message -- will retry") + log.debug("caught exception sending message -- will retry") continue # Verify we successfully sent the message @@ -187,12 +188,16 @@ class TestFailover(KafkaIntegrationTestCase): def _send_random_messages(self, producer, topic, partition, n): for j in range(n): - logging.debug('_send_random_message to %s:%d -- try %d', topic, partition, j) msg = 'msg {0}: {1}'.format(j, random_string(10)) - resp = producer.send_messages(topic, partition, msg.encode('utf-8')) - if len(resp) > 0: - self.assertEqual(resp[0].error, 0) - logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j) + log.debug('_send_random_message %s to %s:%d', msg, topic, partition) + while True: + try: + producer.send_messages(topic, partition, msg.encode('utf-8')) + except: + log.exception('failure in _send_random_messages - retrying') + continue + else: + break def _kill_leader(self, topic, partition): leader = self.client.topics_to_brokers[TopicAndPartition(kafka_bytestring(topic), partition)] From e60de943175c45d64602c0388b502b9763b34f27 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 09:38:24 -0700 Subject: [PATCH 179/250] Split consumer test class between Simple and MultiProcess --- test/test_consumer.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index 08fd620..bbced02 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -25,10 +25,11 @@ class TestMultiProcessConsumer(unittest.TestCase): client = MagicMock() partitions = (0,) with patch.object(MultiProcessConsumer, 'fetch_last_known_offsets') as fetch_last_known_offsets: - consumer = MultiProcessConsumer(client, 'testing-group', 'testing-topic', partitions=partitions) + MultiProcessConsumer(client, 'testing-group', 'testing-topic', partitions=partitions) self.assertEqual(fetch_last_known_offsets.call_args[0], (partitions,) ) self.assertEqual(client.get_partition_ids_for_topic.call_count, 0) # pylint: disable=no-member +class TestSimpleConsumer(unittest.TestCase): def test_simple_consumer_failed_payloads(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, From ecdcdf531d232a923f4869f421d3a908dd735d4a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 09:43:47 -0700 Subject: [PATCH 180/250] (test) Consumer commit() should log errors and return True/False, not raise exceptions --- test/test_consumer.py | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/test/test_consumer.py b/test/test_consumer.py index bbced02..57c69a2 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -81,6 +81,30 @@ class TestSimpleConsumer(unittest.TestCase): with self.assertRaises(UnknownTopicOrPartitionError): consumer.get_messages(20) + def test_simple_consumer_commit_does_not_raise(self): + client = MagicMock() + client.get_partition_ids_for_topic.return_value = [0, 1] + + def mock_offset_fetch_request(group, payloads, **kwargs): + return [OffsetFetchResponse(p.topic, p.partition, 0, b'', 0) for p in payloads] + + client.send_offset_fetch_request.side_effect = mock_offset_fetch_request + + def mock_offset_commit_request(group, payloads, **kwargs): + raise FailedPayloadsError(payloads[0]) + + client.send_offset_commit_request.side_effect = mock_offset_commit_request + + consumer = SimpleConsumer(client, group='foobar', + topic='topic', partitions=[0, 1], + auto_commit=False) + + # Mock internal commit check + consumer.count_since_commit = 10 + + # This should not raise an exception + self.assertFalse(consumer.commit(partitions=[0, 1])) + @staticmethod def fail_requests_factory(error_factory): # Mock so that only the first request gets a valid response From f021609911d25dbb7ef20410890483c1439edd58 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 09:48:19 -0700 Subject: [PATCH 181/250] Change Consumer commit() to return True/False and log error; dont raise client exceptions --- kafka/consumer/base.py | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 6365cfa..b5383a3 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -8,7 +8,7 @@ from threading import Lock import kafka.common from kafka.common import ( OffsetRequest, OffsetCommitRequest, OffsetFetchRequest, - UnknownTopicOrPartitionError, check_error + UnknownTopicOrPartitionError, check_error, KafkaError ) from kafka.util import kafka_bytestring, ReentrantTimer @@ -114,12 +114,13 @@ class Consumer(object): self.offsets[resp.partition] = resp.offset def commit(self, partitions=None): - """ - Commit offsets for this consumer + """Commit stored offsets to Kafka via OffsetCommitRequest (v0) Keyword Arguments: partitions (list): list of partitions to commit, default is to commit all of them + + Returns: True on success, False on failure """ # short circuit if nothing happened. This check is kept outside @@ -135,22 +136,27 @@ class Consumer(object): reqs = [] if partitions is None: # commit all partitions - partitions = self.offsets.keys() + partitions = list(self.offsets.keys()) + log.info('Committing new offsets for %s, partitions %s', + self.topic, partitions) for partition in partitions: offset = self.offsets[partition] - log.debug("Commit offset %d in SimpleConsumer: " - "group=%s, topic=%s, partition=%s" % - (offset, self.group, self.topic, partition)) + log.debug('Commit offset %d in SimpleConsumer: ' + 'group=%s, topic=%s, partition=%s', + offset, self.group, self.topic, partition) reqs.append(OffsetCommitRequest(self.topic, partition, offset, None)) - resps = self.client.send_offset_commit_request(self.group, reqs) - for resp in resps: - kafka.common.check_error(resp) - - self.count_since_commit = 0 + try: + self.client.send_offset_commit_request(self.group, reqs) + except KafkaError as e: + log.error('%s saving offsets: %s', e.__class__.__name__, e) + return False + else: + self.count_since_commit = 0 + return True def _auto_commit(self): """ From 680a8dc3376badccccf0aab27a2307adc0b4cb0d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 09:46:33 -0700 Subject: [PATCH 182/250] (test) SimpleConsumer.reset_partition_offset should not raise exception on failure --- test/test_consumer.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index 57c69a2..df15115 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -4,7 +4,7 @@ from . import unittest from kafka import SimpleConsumer, KafkaConsumer, MultiProcessConsumer from kafka.common import ( - KafkaConfigurationError, FetchResponse, + KafkaConfigurationError, FetchResponse, OffsetFetchResponse, FailedPayloadsError, OffsetAndMessage, NotLeaderForPartitionError, UnknownTopicOrPartitionError ) @@ -105,6 +105,21 @@ class TestSimpleConsumer(unittest.TestCase): # This should not raise an exception self.assertFalse(consumer.commit(partitions=[0, 1])) + def test_simple_consumer_reset_partition_offset(self): + client = MagicMock() + + def mock_offset_request(payloads, **kwargs): + raise FailedPayloadsError(payloads[0]) + + client.send_offset_request.side_effect = mock_offset_request + + consumer = SimpleConsumer(client, group='foobar', + topic='topic', partitions=[0, 1], + auto_commit=False) + + # This should not raise an exception + self.assertEqual(consumer.reset_partition_offset(0), None) + @staticmethod def fail_requests_factory(error_factory): # Mock so that only the first request gets a valid response From ed42d7899117e4bba8ef47afe825c13185cbdfc7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 09:50:06 -0700 Subject: [PATCH 183/250] Change SimpleConsumer.reset_partition_offset to return offset / None on failure (dont raise exception) --- kafka/consumer/simple.py | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index e4233ff..c75e78b 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -27,7 +27,7 @@ from .base import ( NO_MESSAGES_WAIT_TIME_SECONDS ) from ..common import ( - FetchRequest, OffsetRequest, + FetchRequest, KafkaError, OffsetRequest, ConsumerFetchSizeTooSmall, ConsumerNoMoreData, UnknownTopicOrPartitionError, NotLeaderForPartitionError, OffsetOutOfRangeError, FailedPayloadsError, check_error @@ -144,6 +144,13 @@ class SimpleConsumer(Consumer): (self.group, self.topic, str(self.offsets.keys())) def reset_partition_offset(self, partition): + """Update offsets using auto_offset_reset policy (smallest|largest) + + Arguments: + partition (int): the partition for which offsets should be updated + + Returns: Updated offset on success, None on failure + """ LATEST = -1 EARLIEST = -2 if self.auto_offset_reset == 'largest': @@ -163,10 +170,17 @@ class SimpleConsumer(Consumer): raise # send_offset_request - (resp, ) = self.client.send_offset_request(reqs) - check_error(resp) - self.offsets[partition] = resp.offsets[0] - self.fetch_offsets[partition] = resp.offsets[0] + log.info('Resetting topic-partition offset to %s for %s:%d', + self.auto_offset_reset, self.topic, partition) + try: + (resp, ) = self.client.send_offset_request(reqs) + except KafkaError as e: + log.error('%s sending offset request for %s:%d', + e.__class__.__name__, self.topic, partition) + else: + self.offsets[partition] = resp.offsets[0] + self.fetch_offsets[partition] = resp.offsets[0] + return resp.offsets[0] def provide_partition_info(self): """ From 6a3e813199f352bf14355ba375d86fc3f1024c9a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 11:06:26 -0700 Subject: [PATCH 184/250] Use at_least=True when counting messages in leader failover tests --- test/test_failover_integration.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index e27f12b..91779d7 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -90,7 +90,9 @@ class TestFailover(KafkaIntegrationTestCase): # count number of messages # Should be equal to 100 before + 1 recovery + 100 after - self.assert_message_count(topic, 201, partitions=(partition,)) + # at_least=True because exactly once delivery isn't really a thing + self.assert_message_count(topic, 201, partitions=(partition,), + at_least=True) @kafka_versions("all") def test_switch_leader_async(self): @@ -134,6 +136,7 @@ class TestFailover(KafkaIntegrationTestCase): # count number of messages # Should be equal to 10 before + 1 recovery + 10 after + # at_least=True because exactly once delivery isn't really a thing self.assert_message_count(topic, 21, partitions=(partition,), at_least=True) self.assert_message_count(topic, 21, partitions=(partition + 1,), From 04cf4b36686131af392f401a32be7bf5567fb7c2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 11:21:21 -0700 Subject: [PATCH 185/250] Use 500ms zookeeper session timeout for integration tests - no network here, and our restarts are fast --- servers/0.8.0/resources/kafka.properties | 4 ++++ servers/0.8.1.1/resources/kafka.properties | 2 ++ servers/0.8.1/resources/kafka.properties | 4 ++++ servers/0.8.2.0/resources/kafka.properties | 2 ++ 4 files changed, 12 insertions(+) diff --git a/servers/0.8.0/resources/kafka.properties b/servers/0.8.0/resources/kafka.properties index 1aafd46..b9f5c49 100644 --- a/servers/0.8.0/resources/kafka.properties +++ b/servers/0.8.0/resources/kafka.properties @@ -53,7 +53,11 @@ log.cleanup.interval.mins=1 ############################# Zookeeper ############################# zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 kafka.metrics.polling.interval.secs=5 kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/servers/0.8.1.1/resources/kafka.properties b/servers/0.8.1.1/resources/kafka.properties index a59ef60..685aed1 100644 --- a/servers/0.8.1.1/resources/kafka.properties +++ b/servers/0.8.1.1/resources/kafka.properties @@ -120,3 +120,5 @@ zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} # Timeout in ms for connecting to zookeeper zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/0.8.1/resources/kafka.properties b/servers/0.8.1/resources/kafka.properties index af67a91..76b0cb4 100644 --- a/servers/0.8.1/resources/kafka.properties +++ b/servers/0.8.1/resources/kafka.properties @@ -60,4 +60,8 @@ log.cleaner.enable=false # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/0.8.2.0/resources/kafka.properties b/servers/0.8.2.0/resources/kafka.properties index a59ef60..685aed1 100644 --- a/servers/0.8.2.0/resources/kafka.properties +++ b/servers/0.8.2.0/resources/kafka.properties @@ -120,3 +120,5 @@ zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} # Timeout in ms for connecting to zookeeper zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 From 942c693c23996a3430c82fff8714fecb821a2b9d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 12:08:02 -0700 Subject: [PATCH 186/250] Add error type to SimpleConsumer error logging --- kafka/consumer/simple.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index e4233ff..aa49fca 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -344,23 +344,26 @@ class SimpleConsumer(Consumer): try: check_error(resp) except UnknownTopicOrPartitionError: + log.error('UnknownTopicOrPartitionError for %s:%d', + resp.topic, resp.partition) self.client.reset_topic_metadata(resp.topic) raise except NotLeaderForPartitionError: + log.error('NotLeaderForPartitionError for %s:%d', + resp.topic, resp.partition) self.client.reset_topic_metadata(resp.topic) continue except OffsetOutOfRangeError: - log.warning("OffsetOutOfRangeError for %s - %d. " - "Resetting partition offset...", + log.warning('OffsetOutOfRangeError for %s:%d. ' + 'Resetting partition offset...', resp.topic, resp.partition) self.reset_partition_offset(resp.partition) # Retry this partition retry_partitions[resp.partition] = partitions[resp.partition] continue except FailedPayloadsError as e: - log.warning("Failed payloads of %s" - "Resetting partition offset...", - e.payload) + log.warning('FailedPayloadsError for %s:%d', + e.payload.topic, e.payload.partition) # Retry this partition retry_partitions[e.payload.partition] = partitions[e.payload.partition] continue From 6fde9236d40f1f8df30b84d93ee1dbc1038f5ea7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 4 Jun 2015 23:50:10 -0700 Subject: [PATCH 187/250] Add kafka 0.8.2.1 to integration tests, remove 0.8.2.0 --- .travis.yml | 2 +- build_integration.sh | 2 +- servers/0.8.2.1/resources/kafka.properties | 124 ++++++++++++++++++ servers/0.8.2.1/resources/log4j.properties | 24 ++++ .../0.8.2.1/resources/zookeeper.properties | 21 +++ test/test_client_integration.py | 2 +- test/test_consumer_integration.py | 10 +- 7 files changed, 177 insertions(+), 8 deletions(-) create mode 100644 servers/0.8.2.1/resources/kafka.properties create mode 100644 servers/0.8.2.1/resources/log4j.properties create mode 100644 servers/0.8.2.1/resources/zookeeper.properties diff --git a/.travis.yml b/.travis.yml index 7184bc8..136c19f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -12,7 +12,7 @@ env: - KAFKA_VERSION=0.8.0 - KAFKA_VERSION=0.8.1 - KAFKA_VERSION=0.8.1.1 - - KAFKA_VERSION=0.8.2.0 + - KAFKA_VERSION=0.8.2.1 before_install: - sudo apt-get install libsnappy-dev diff --git a/build_integration.sh b/build_integration.sh index 2b81745..5395bb8 100755 --- a/build_integration.sh +++ b/build_integration.sh @@ -1,7 +1,7 @@ #!/bin/bash # Versions available for testing via binary distributions -OFFICIAL_RELEASES="0.8.0 0.8.1 0.8.1.1 0.8.2.0" +OFFICIAL_RELEASES="0.8.0 0.8.1 0.8.1.1 0.8.2.1" # Useful configuration vars, with sensible defaults if [ -z "$SCALA_VERSION" ]; then diff --git a/servers/0.8.2.1/resources/kafka.properties b/servers/0.8.2.1/resources/kafka.properties new file mode 100644 index 0000000..685aed1 --- /dev/null +++ b/servers/0.8.2.1/resources/kafka.properties @@ -0,0 +1,124 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id={broker_id} + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port={port} + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +host.name={host} + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs={tmp_dir}/data + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions={partitions} +default.replication.factor={replicas} + +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=536870912 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=60000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/0.8.2.1/resources/log4j.properties b/servers/0.8.2.1/resources/log4j.properties new file mode 100644 index 0000000..f863b3b --- /dev/null +++ b/servers/0.8.2.1/resources/log4j.properties @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.logger.kafka=DEBUG, stdout +log4j.logger.org.I0Itec.zkclient.ZkClient=INFO, stdout +log4j.logger.org.apache.zookeeper=INFO, stdout diff --git a/servers/0.8.2.1/resources/zookeeper.properties b/servers/0.8.2.1/resources/zookeeper.properties new file mode 100644 index 0000000..e3fd097 --- /dev/null +++ b/servers/0.8.2.1/resources/zookeeper.properties @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir={tmp_dir} +# the port at which the clients will connect +clientPort={port} +clientPortAddress={host} +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 diff --git a/test/test_client_integration.py b/test/test_client_integration.py index baaf79e..8853350 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -83,7 +83,7 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): # Offset Tests # #################### - @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_commit_fetch_offsets(self): req = OffsetCommitRequest(self.bytes_topic, 0, 42, b"metadata") (resp,) = self.client.send_offset_commit_request(b"group", [req]) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 8911e3e..df2eeea 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -132,7 +132,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): with self.assertRaises(OffsetOutOfRangeError): consumer.get_message() - @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_simple_consumer_load_initial_offsets(self): self.send_messages(0, range(0, 100)) self.send_messages(1, range(100, 200)) @@ -276,7 +276,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer.stop() - @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_multi_process_consumer_load_initial_offsets(self): self.send_messages(0, range(0, 10)) self.send_messages(1, range(10, 20)) @@ -342,7 +342,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): big_consumer.stop() - @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_offset_behavior__resuming_behavior(self): self.send_messages(0, range(0, 100)) self.send_messages(1, range(100, 200)) @@ -369,7 +369,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer1.stop() consumer2.stop() - @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_multi_process_offset_behavior__resuming_behavior(self): self.send_messages(0, range(0, 100)) self.send_messages(1, range(100, 200)) @@ -477,7 +477,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assertEqual(len(messages), 5) self.assertGreaterEqual(t.interval, TIMEOUT_MS / 1000.0 ) - @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_kafka_consumer__offset_commit_resume(self): GROUP_ID = random_string(10).encode('utf-8') From 92da259539cc31bf5bba859286a8b098ea242b80 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 8 Jun 2015 21:49:25 -0700 Subject: [PATCH 188/250] Update to 0.8.2.1 compatibility in docs --- README.rst | 4 ++-- docs/index.rst | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/README.rst b/README.rst index 5405f92..866c570 100644 --- a/README.rst +++ b/README.rst @@ -36,7 +36,7 @@ The current stable version of this package is `0.9.3`_ and is compatible with: Kafka broker versions -- 0.8.2.0 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] +- 0.8.2.1 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] - 0.8.1.1 - 0.8.1 - 0.8.0 @@ -47,7 +47,7 @@ Python versions - 2.7 (tested on 2.7.9) - 3.3 (tested on 3.3.5) - 3.4 (tested on 3.4.2) -- pypy (tested on pypy 2.4.0 / python 2.7.8) +- pypy (tested on pypy 2.5.0 / python 2.7.8) .. _Full documentation available on ReadTheDocs: http://kafka-python.readthedocs.org/en/latest/ .. _0.9.3: https://github.com/mumrah/kafka-python/releases/tag/v0.9.3 diff --git a/docs/index.rst b/docs/index.rst index e4a9ac7..30436a0 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -20,7 +20,7 @@ The current stable version of this package is `0.9.3 Date: Mon, 8 Jun 2015 23:44:29 -0700 Subject: [PATCH 189/250] Refactor SimpleConsumer get_messages and _fetch - attempt to make these easier to read - add marginal retry logic to internal _fetch loop --- kafka/consumer/simple.py | 62 +++++++++++++++++++++------------------- 1 file changed, 33 insertions(+), 29 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 82a1fe2..b08255b 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -126,8 +126,8 @@ class SimpleConsumer(Consumer): auto_commit_every_t=auto_commit_every_t) if max_buffer_size is not None and buffer_size > max_buffer_size: - raise ValueError("buffer_size (%d) is greater than " - "max_buffer_size (%d)" % + raise ValueError('buffer_size (%d) is greater than ' + 'max_buffer_size (%d)' % (buffer_size, max_buffer_size)) self.buffer_size = buffer_size self.max_buffer_size = max_buffer_size @@ -227,7 +227,7 @@ class SimpleConsumer(Consumer): self.offsets[resp.partition] = \ resp.offsets[0] + deltas[resp.partition] else: - raise ValueError("Unexpected value for `whence`, %d" % whence) + raise ValueError('Unexpected value for `whence`, %d' % whence) # Reset queue and fetch offsets since they are invalid self.fetch_offsets = self.offsets.copy() @@ -250,35 +250,32 @@ class SimpleConsumer(Consumer): """ messages = [] if timeout is not None: - max_time = time.time() + timeout + timeout += time.time() new_offsets = {} - while count > 0 and (timeout is None or timeout > 0): - result = self._get_message(block, timeout, get_partition_info=True, + log.debug('getting %d messages', count) + while len(messages) < count: + block_time = timeout - time.time() + log.debug('calling _get_message block=%s timeout=%s', block, block_time) + result = self._get_message(block, block_time, + get_partition_info=True, update_offset=False) - if result: - partition, message = result - if self.partition_info: - messages.append(result) - else: - messages.append(message) - new_offsets[partition] = message.offset + 1 - count -= 1 - else: - # Ran out of messages for the last request. - if not block: - # If we're not blocking, break. - break + log.debug('got %s from _get_messages', result) + if not result: + if block and (timeout is None or time.time() <= timeout): + continue + break - # If we have a timeout, reduce it to the - # appropriate value - if timeout is not None: - timeout = max_time - time.time() + partition, message = result + _msg = (partition, message) if self.partition_info else message + messages.append(_msg) + new_offsets[partition] = message.offset + 1 # Update and commit offsets if necessary self.offsets.update(new_offsets) self.count_since_commit += len(messages) self._auto_commit() + log.debug('got %d messages: %s', len(messages), messages) return messages def get_message(self, block=True, timeout=0.1, get_partition_info=None): @@ -292,10 +289,16 @@ class SimpleConsumer(Consumer): If get_partition_info is True, returns (partition, message) If get_partition_info is False, returns message """ - if self.queue.empty(): + start_at = time.time() + while self.queue.empty(): # We're out of messages, go grab some more. + log.debug('internal queue empty, fetching more messages') with FetchContext(self, block, timeout): self._fetch() + + if not block or time.time() > (start_at + timeout): + break + try: partition, message = self.queue.get_nowait() @@ -314,6 +317,7 @@ class SimpleConsumer(Consumer): else: return message except Empty: + log.debug('internal queue empty after fetch - returning None') return None def __iter__(self): @@ -396,7 +400,7 @@ class SimpleConsumer(Consumer): except ConsumerFetchSizeTooSmall: if (self.max_buffer_size is not None and buffer_size == self.max_buffer_size): - log.error("Max fetch size %d too small", + log.error('Max fetch size %d too small', self.max_buffer_size) raise if self.max_buffer_size is None: @@ -404,12 +408,12 @@ class SimpleConsumer(Consumer): else: buffer_size = min(buffer_size * 2, self.max_buffer_size) - log.warn("Fetch size too small, increase to %d (2x) " - "and retry", buffer_size) + log.warning('Fetch size too small, increase to %d (2x) ' + 'and retry', buffer_size) retry_partitions[partition] = buffer_size except ConsumerNoMoreData as e: - log.debug("Iteration was ended by %r", e) + log.debug('Iteration was ended by %r', e) except StopIteration: # Stop iterating through this partition - log.debug("Done iterating over partition %s" % partition) + log.debug('Done iterating over partition %s', partition) partitions = retry_partitions From 4e339a746d84650f75fb401e3bd9cfe6e3a65e00 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 17 May 2015 12:28:28 -0700 Subject: [PATCH 190/250] Use MurmurHash2 for key partition hashing --- kafka/partitioner/__init__.py | 5 +- kafka/partitioner/hashed.py | 96 ++++++++++++++++++++++++++++++++++- 2 files changed, 98 insertions(+), 3 deletions(-) diff --git a/kafka/partitioner/__init__.py b/kafka/partitioner/__init__.py index fdb19bb..5b6ac2d 100644 --- a/kafka/partitioner/__init__.py +++ b/kafka/partitioner/__init__.py @@ -1,6 +1,7 @@ from .roundrobin import RoundRobinPartitioner -from .hashed import HashedPartitioner +from .hashed import HashedPartitioner, Murmur2Partitioner, LegacyPartitioner __all__ = [ - 'RoundRobinPartitioner', 'HashedPartitioner' + 'RoundRobinPartitioner', 'HashedPartitioner', 'Murmur2Partitioner', + 'LegacyPartitioner' ] diff --git a/kafka/partitioner/hashed.py b/kafka/partitioner/hashed.py index fb5e598..6393ce2 100644 --- a/kafka/partitioner/hashed.py +++ b/kafka/partitioner/hashed.py @@ -1,7 +1,25 @@ from .base import Partitioner -class HashedPartitioner(Partitioner): + +class Murmur2Partitioner(Partitioner): """ + Implements a partitioner which selects the target partition based on + the hash of the key. Attempts to apply the same hashing + function as mainline java client. + """ + def partition(self, key, partitions=None): + if not partitions: + partitions = self.partitions + + # https://github.com/apache/kafka/blob/0.8.2/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java#L69 + idx = (murmur2(key) & 0x7fffffff) % len(partitions) + + return partitions[idx] + + +class LegacyPartitioner(Partitioner): + """DEPRECATED -- See Issue 374 + Implements a partitioner which selects the target partition based on the hash of the key """ @@ -12,3 +30,79 @@ class HashedPartitioner(Partitioner): idx = hash(key) % size return partitions[idx] + + +# Default will change to Murmur2 in 0.10 release +HashedPartitioner = LegacyPartitioner + + +# https://github.com/apache/kafka/blob/0.8.2/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L244 +def murmur2(key): + """Pure-python Murmur2 implementation. + + Based on java client, see org.apache.kafka.common.utils.Utils.murmur2 + + Args: + key: if not a bytearray, converted via bytearray(str(key)) + + Returns: MurmurHash2 of key bytearray + """ + + # Convert key to a bytearray + if not isinstance(key, bytearray): + data = bytearray(str(key)) + + length = len(data) + seed = 0x9747b28c + # 'm' and 'r' are mixing constants generated offline. + # They're not really 'magic', they just happen to work well. + m = 0x5bd1e995 + r = 24 + + # Initialize the hash to a random value + h = seed ^ length + length4 = length / 4 + + for i in range(length4): + i4 = i * 4 + k = ((data[i4 + 0] & 0xff) + + ((data[i4 + 1] & 0xff) << 8) + + ((data[i4 + 2] & 0xff) << 16) + + ((data[i4 + 3] & 0xff) << 24)) + k &= 0xffffffff + k *= m + k &= 0xffffffff + k ^= (k % 0x100000000) >> r # k ^= k >>> r + k &= 0xffffffff + k *= m + k &= 0xffffffff + + h *= m + h &= 0xffffffff + h ^= k + h &= 0xffffffff + + # Handle the last few bytes of the input array + extra_bytes = length % 4 + if extra_bytes == 3: + h ^= (data[(length & ~3) + 2] & 0xff) << 16 + h &= 0xffffffff + + if extra_bytes == 2: + h ^= (data[(length & ~3) + 1] & 0xff) << 8 + h &= 0xffffffff + + if extra_bytes == 1: + h ^= (data[length & ~3] & 0xff) + h &= 0xffffffff + h *= m + h &= 0xffffffff + + h ^= (h % 0x100000000) >> 13 # h >>> 13; + h &= 0xffffffff + h *= m + h &= 0xffffffff + h ^= (h % 0x100000000) >> 15 # h >>> 15; + h &= 0xffffffff + + return h From 2451dcc55201a311c8fb5e53261cffd3235e6c77 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Jun 2015 10:38:41 -0700 Subject: [PATCH 191/250] Increase timeout and backoff when retrying spawn of test fixtures --- test/fixtures.py | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/test/fixtures.py b/test/fixtures.py index d4d03ee..164d0d7 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -4,6 +4,7 @@ import os.path import shutil import subprocess import tempfile +import time from six.moves import urllib import uuid @@ -125,12 +126,18 @@ class ZookeeperFixture(Fixture): # Party! self.out("Starting...") + timeout = 5 + max_timeout = 30 + backoff = 1 while True: self.child = SpawnedService(args, env) self.child.start() - if self.child.wait_for(r"binding to port", timeout=5): + timeout = min(timeout, max_timeout) + if self.child.wait_for(r"binding to port", timeout=timeout): break self.child.stop() + timeout *= 2 + time.sleep(backoff) self.out("Done!") def close(self): @@ -225,12 +232,19 @@ class KafkaFixture(Fixture): args = self.kafka_run_class_args("kafka.Kafka", properties) env = self.kafka_run_class_env() + timeout = 5 + max_timeout = 30 + backoff = 1 while True: self.child = SpawnedService(args, env) self.child.start() - if self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id, timeout=5): + timeout = min(timeout, max_timeout) + if self.child.wait_for(r"\[Kafka Server %d\], Started" % + self.broker_id, timeout=timeout): break self.child.stop() + timeout *= 2 + time.sleep(backoff) self.out("Done!") self.running = True From 4c76da9ad7e19954487e0bed51c367d5fc963be8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Jun 2015 22:52:15 -0700 Subject: [PATCH 192/250] Fix reading version in docs/conf.py --- docs/conf.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/conf.py b/docs/conf.py index ea223c2..2979560 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -56,11 +56,10 @@ copyright = u'2015, David Arthur' # built documents. # # The short X.Y version. -with open('../VERSION') as version_file: - version = version_file.read() +exec(open('../kafka/version.py').read()) # The full version, including alpha/beta/rc tags. -release = version +release = __version__ # The language for content autogenerated by Sphinx. Refer to documentation # for a list of supported languages. From 47bc6a70953980cdf40caa2a4a127ae426d39bb1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Jun 2015 22:52:50 -0700 Subject: [PATCH 193/250] Some minor updates to usage docs --- docs/usage.rst | 75 +++++++++++++++++++++++++++----------------------- 1 file changed, 40 insertions(+), 35 deletions(-) diff --git a/docs/usage.rst b/docs/usage.rst index ca326d4..6417cd8 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -9,21 +9,24 @@ SimpleProducer from kafka import SimpleProducer, KafkaClient # To send messages synchronously - kafka = KafkaClient("localhost:9092") + kafka = KafkaClient('localhost:9092') producer = SimpleProducer(kafka) - # Note that the application is responsible for encoding messages to type str - producer.send_messages("my-topic", "some message") - producer.send_messages("my-topic", "this method", "is variadic") + # Note that the application is responsible for encoding messages to type bytes + producer.send_messages(b'my-topic', b'some message') + producer.send_messages(b'my-topic', b'this method', b'is variadic') # Send unicode message - producer.send_messages("my-topic", u'你怎么样?'.encode('utf-8')) + producer.send_messages(b'my-topic', u'你怎么样?'.encode('utf-8')) + +Asynchronous Mode +----------------- + +.. code:: python # To send messages asynchronously - # WARNING: current implementation does not guarantee message delivery on failure! - # messages can get dropped! Use at your own risk! Or help us improve with a PR! producer = SimpleProducer(kafka, async=True) - producer.send_messages("my-topic", "async message") + producer.send_messages(b'my-topic', b'async message') # To wait for acknowledgements # ACK_AFTER_LOCAL_WRITE : server will wait till the data is written to @@ -32,13 +35,12 @@ SimpleProducer # by all in sync replicas before sending a response producer = SimpleProducer(kafka, async=False, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE, - ack_timeout=2000) + ack_timeout=2000, + sync_fail_on_error=False) - response = producer.send_messages("my-topic", "another message") - - if response: - print(response[0].error) - print(response[0].offset) + responses = producer.send_messages(b'my-topic', b'another message') + for r in responses: + logging.info(r.offset) # To send messages in batch. You can use any of the available # producers for doing this. The following producer will collect @@ -56,16 +58,21 @@ Keyed messages .. code:: python - from kafka import (KafkaClient, KeyedProducer, HashedPartitioner, - RoundRobinPartitioner) + from kafka import ( + KafkaClient, KeyedProducer, + Murmur2Partitioner, RoundRobinPartitioner) - kafka = KafkaClient("localhost:9092") + kafka = KafkaClient('localhost:9092') - # HashedPartitioner is default + # HashedPartitioner is default (currently uses python hash()) producer = KeyedProducer(kafka) - producer.send_messages("my-topic", "key1", "some message") - producer.send_messages("my-topic", "key2", "this methode") + producer.send_messages(b'my-topic', b'key1', b'some message') + producer.send_messages(b'my-topic', b'key2', b'this methode') + # Murmur2Partitioner attempts to mirror the java client hashing + producer = KeyedProducer(kafka, partitioner=Murmur2Partitioner) + + # Or just produce round-robin (or just use SimpleProducer) producer = KeyedProducer(kafka, partitioner=RoundRobinPartitioner) @@ -78,9 +85,9 @@ KafkaConsumer from kafka import KafkaConsumer # To consume messages - consumer = KafkaConsumer("my-topic", - group_id="my_group", - bootstrap_servers=["localhost:9092"]) + consumer = KafkaConsumer('my-topic', + group_id='my_group', + bootstrap_servers=['localhost:9092']) for message in consumer: # message value is raw byte string -- decode if necessary! # e.g., for unicode: `message.value.decode('utf-8')` @@ -88,8 +95,6 @@ KafkaConsumer message.offset, message.key, message.value)) - kafka.close() - messages (m) are namedtuples with attributes: @@ -121,16 +126,16 @@ messages (m) are namedtuples with attributes: # so it can be included in the next commit # # **messages that are not marked w/ task_done currently do not commit! - kafka.task_done(m) + consumer.task_done(m) # If auto_commit_enable is False, remember to commit() periodically - kafka.commit() + consumer.commit() # Batch process interface while True: for m in kafka.fetch_messages(): process_message(m) - kafka.task_done(m) + consumer.task_done(m) Configuration settings can be passed to constructor, @@ -162,13 +167,13 @@ Multiprocess consumer from kafka import KafkaClient, MultiProcessConsumer - kafka = KafkaClient("localhost:9092") + kafka = KafkaClient('localhost:9092') # This will split the number of partitions among two processes - consumer = MultiProcessConsumer(kafka, "my-group", "my-topic", num_procs=2) + consumer = MultiProcessConsumer(kafka, b'my-group', b'my-topic', num_procs=2) # This will spawn processes such that each handles 2 partitions max - consumer = MultiProcessConsumer(kafka, "my-group", "my-topic", + consumer = MultiProcessConsumer(kafka, b'my-group', b'my-topic', partitions_per_proc=2) for message in consumer: @@ -186,14 +191,14 @@ Low level from kafka.protocol import KafkaProtocol from kafka.common import ProduceRequest - kafka = KafkaClient("localhost:9092") + kafka = KafkaClient('localhost:9092') - req = ProduceRequest(topic="my-topic", partition=1, - messages=[create_message("some message")]) + req = ProduceRequest(topic=b'my-topic', partition=1, + messages=[create_message(b'some message')]) resps = kafka.send_produce_request(payloads=[req], fail_on_error=True) kafka.close() - resps[0].topic # "my-topic" + resps[0].topic # b'my-topic' resps[0].partition # 1 resps[0].error # 0 (hopefully) resps[0].offset # offset of the first message sent in this request From 513f210ba347adb083108b12a32dca5a4931fe21 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 9 Jun 2015 09:39:29 -0700 Subject: [PATCH 194/250] Add KafkaClient.topics property to get list of known topics --- kafka/client.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/kafka/client.py b/kafka/client.py index 1bd8587..cf29eda 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -306,6 +306,10 @@ class KafkaClient(object): return sorted(list(self.topic_partitions[topic])) + @property + def topics(self): + return list(self.topic_partitions.keys()) + def ensure_topic_exists(self, topic, timeout = 30): start_time = time.time() From 605402116eda6f781fbadce0a0f482cd81a4f7f5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Jun 2015 23:02:09 -0700 Subject: [PATCH 195/250] Update contributors list -- great community! --- AUTHORS.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/AUTHORS.md b/AUTHORS.md index 67e3789..d9ce2ed 100644 --- a/AUTHORS.md +++ b/AUTHORS.md @@ -1,16 +1,19 @@ # Contributors -Top 10 contributors, listed by contribution. See https://github.com/mumrah/kafka-python/graphs/contributors for the full list +Top contributors, listed by contribution. See https://github.com/mumrah/kafka-python/graphs/contributors for the full list * David Arthur, [@mumrah](https://github.com/mumrah) * Dana Powers, [@dpkp](https://github.com/dpkp) * Mahendra M, [@mahendra](https://github.com/mahendra) * Mark Roberts, [@wizzat](https://github.com/wizzat) * Omar, [@rdiomar](https://github.com/rdiomar) - RIP, Omar. 2014 +* Viktor Shlapakov, [@vshlapakov](https://github.com/vshlapakov) * Bruno Renié, [@brutasse](https://github.com/brutasse) * Marc Labbé, [@mrtheb](https://github.com/mrtheb) +* John Anderson, [@sontek](https://github.com/sontek) * Ivan Pouzyrevsky, [@sandello](https://github.com/sandello) * Thomas Dimson, [@cosbynator](https://github.com/cosbynator) -* Zack Dever, [@zever](https://github.com/zever) +* Alex Couture-Beil, [@alexcb](https://github.com/alexcb) +* Zack Dever, [@zackdever](https://github.com/zackdever) Thanks to all who have contributed! From a6ab3fa4827a8479a2a8192a307ebd9038138569 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 10 Jun 2015 02:39:59 -0700 Subject: [PATCH 196/250] Update CHANGELOG for 0.9.4 --- CHANGES.md | 69 ++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 5704afa..c94cbd5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,72 @@ +# 0.9.4 (June 11, 2015) + +Consumers +* Refactor SimpleConsumer internal fetch handling (dpkp PR 399) +* Handle exceptions in SimpleConsumer commit() and reset_partition_offset() (dpkp PR 404) +* Improve FailedPayloadsError handling in KafkaConsumer (dpkp PR 398) +* KafkaConsumer: avoid raising KeyError in task_done (dpkp PR 389) +* MultiProcessConsumer -- support configured partitions list (dpkp PR 380) +* Fix SimpleConsumer leadership change handling (dpkp PR 393) +* Fix SimpleConsumer connection error handling (reAsOn2010 PR 392) +* Improve Consumer handling of 'falsy' partition values (wting PR 342) +* Fix _offsets call error in KafkaConsumer (hellais PR 376) +* Fix str/bytes bug in KafkaConsumer (dpkp PR 365) +* Register atexit handlers for consumer and producer thread/multiprocess cleanup (dpkp PR 360) +* Always fetch commit offsets in base consumer unless group is None (dpkp PR 356) +* Stop consumer threads on delete (dpkp PR 357) +* Deprecate metadata_broker_list in favor of bootstrap_servers in KafkaConsumer (dpkp PR 340) +* Support pass-through parameters in multiprocess consumer (scrapinghub PR 336) +* Enable offset commit on SimpleConsumer.seek (ecanzonieri PR 350) +* Improve multiprocess consumer partition distribution (scrapinghub PR 335) +* Ignore messages with offset less than requested (wkiser PR 328) +* Handle OffsetOutOfRange in SimpleConsumer (ecanzonieri PR 296) + +Producers +* Add Murmur2Partitioner (dpkp PR 378) +* Log error types in SimpleProducer and SimpleConsumer (dpkp PR 405) +* SimpleProducer support configuration of fail_on_error (dpkp PR 396) +* Deprecate KeyedProducer.send() (dpkp PR 379) +* Further improvements to async producer code (dpkp PR 388) +* Add more configuration parameters for async producer (dpkp) +* Deprecate SimpleProducer batch_send=True in favor of async (dpkp) +* Improve async producer error handling and retry logic (vshlapakov PR 331) +* Support message keys in async producer (vshlapakov PR 329) +* Use threading instead of multiprocessing for Async Producer (vshlapakov PR 330) +* Stop threads on __del__ (chmduquesne PR 324) +* Fix leadership failover handling in KeyedProducer (dpkp PR 314) + +KafkaClient +* Add .topics property for list of known topics (dpkp) +* Fix request / response order guarantee bug in KafkaClient (dpkp PR 403) +* Improve KafkaClient handling of connection failures in _get_conn (dpkp) +* Client clears local metadata cache before updating from server (dpkp PR 367) +* KafkaClient should return a response or error for each request - enable better retry handling (dpkp PR 366) +* Improve str/bytes conversion in KafkaClient and KafkaConsumer (dpkp PR 332) +* Always return sorted partition ids in client.get_partition_ids_for_topic() (dpkp PR 315) + +Documentation +* Cleanup Usage Documentation +* Improve KafkaConsumer documentation (dpkp PR 341) +* Update consumer documentation (sontek PR 317) +* Add doc configuration for tox (sontek PR 316) +* Switch to .rst doc format (sontek PR 321) +* Fixup google groups link in README (sontek PR 320) +* Automate documentation at kafka-python.readthedocs.org + +Internals +* Switch integration testing from 0.8.2.0 to 0.8.2.1 (dpkp PR 402) +* Fix most flaky tests, improve debug logging, improve fixture handling (dpkp) +* General style cleanups (dpkp PR 394) +* Raise error on duplicate topic-partition payloads in protocol grouping (dpkp) +* Use module-level loggers instead of simply 'kafka' (dpkp) +* Remove pkg_resources check for __version__ at runtime (dpkp PR 387) +* Make external API consistently support python3 strings for topic (kecaps PR 361) +* Fix correlation id overflow (dpkp PR 355) +* Cleanup kafka/common structs (dpkp PR 338) +* Use context managers in gzip_encode / gzip_decode (dpkp PR 337) +* Save failed request as FailedPayloadsError attribute (jobevers PR 302) +* Remove unused kafka.queue (mumrah) + # 0.9.3 (Feb 3, 2015) * Add coveralls.io support (sontek PR 307) From 4955582be1443b75c23f700268b7abbef0fde0ad Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 11 Jun 2015 23:27:48 -0700 Subject: [PATCH 197/250] Release 0.9.4 --- README.rst | 4 ++-- kafka/version.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.rst b/README.rst index 866c570..e957ee3 100644 --- a/README.rst +++ b/README.rst @@ -32,7 +32,7 @@ Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE` Status ---------- -The current stable version of this package is `0.9.3`_ and is compatible with: +The current stable version of this package is `0.9.4`_ and is compatible with: Kafka broker versions @@ -50,4 +50,4 @@ Python versions - pypy (tested on pypy 2.5.0 / python 2.7.8) .. _Full documentation available on ReadTheDocs: http://kafka-python.readthedocs.org/en/latest/ -.. _0.9.3: https://github.com/mumrah/kafka-python/releases/tag/v0.9.3 +.. _0.9.4: https://github.com/mumrah/kafka-python/releases/tag/v0.9.4 diff --git a/kafka/version.py b/kafka/version.py index 5b721ed..4033697 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '0.9.4-dev' +__version__ = '0.9.4' From 176d0167ec50fb428d1c822e3e79f60bf505389d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 12 Jun 2015 11:21:48 -0700 Subject: [PATCH 198/250] Update docs to 0.9.4 as latest --- docs/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/index.rst b/docs/index.rst index 30436a0..c499d4c 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -16,7 +16,7 @@ see https://groups.google.com/forum/m/#!forum/kafka-clients Status ------ -The current stable version of this package is `0.9.3 `_ and is compatible with: +The current stable version of this package is `0.9.4 `_ and is compatible with: Kafka broker versions From 915a9ce0d1c565fcc16621f5abb46fda310c4552 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 12 Jun 2015 11:23:52 -0700 Subject: [PATCH 199/250] Set development version until next release --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 4033697..cd64b48 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '0.9.4' +__version__ = '0.9.5-dev' From 99998af7ed66b3ddb8cb134fc047524dbe074698 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Tue, 16 Jun 2015 12:09:03 +0300 Subject: [PATCH 200/250] Allow null payload for deletion feature --- kafka/producer/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 49090bd..00e1d13 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -355,7 +355,7 @@ class Producer(object): raise TypeError("msg is not a list or tuple!") # Raise TypeError if any message is not encoded as bytes - if any(not isinstance(m, six.binary_type) for m in msg): + if any(m is not None and not isinstance(m, six.binary_type) for m in msg): raise TypeError("all produce message payloads must be type bytes") # Raise TypeError if topic is not encoded as bytes From a943eb3bedc94366dff34bfe0fcc8c6484adb2d1 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 17 Jun 2015 14:05:05 +0300 Subject: [PATCH 201/250] Exclude (null,null) pair for producer --- kafka/producer/base.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 00e1d13..bee1888 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -354,9 +354,15 @@ class Producer(object): if not isinstance(msg, (list, tuple)): raise TypeError("msg is not a list or tuple!") - # Raise TypeError if any message is not encoded as bytes - if any(m is not None and not isinstance(m, six.binary_type) for m in msg): - raise TypeError("all produce message payloads must be type bytes") + for m in msg: + # The protocol allows to have key & payload with null values both, + # (https://goo.gl/o694yN) but having (null,null) pair doesn't make sense. + if m is None: + if key is None: + raise TypeError("key and payload can't be null in one") + # Raise TypeError if any non-null message is not encoded as bytes + elif not isinstance(m, six.binary_type): + raise TypeError("all produce message payloads must be null or type bytes") # Raise TypeError if topic is not encoded as bytes if not isinstance(topic, six.binary_type): From 86772332474500ec251ff9bd5e4be761563e9aac Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 17 Jun 2015 14:05:44 +0300 Subject: [PATCH 202/250] Add basic keyed producer tests --- test/test_producer.py | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/test/test_producer.py b/test/test_producer.py index 27272f6..297b265 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -7,7 +7,7 @@ import time from mock import MagicMock, patch from . import unittest -from kafka import KafkaClient, SimpleProducer +from kafka import KafkaClient, SimpleProducer, KeyedProducer from kafka.common import ( AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError, ProduceResponse, RetryOptions, TopicAndPartition @@ -33,7 +33,8 @@ class TestKafkaProducer(unittest.TestCase): topic = b"test-topic" partition = 0 - bad_data_types = (u'你怎么样?', 12, ['a', 'list'], ('a', 'tuple'), {'a': 'dict'}) + bad_data_types = (u'你怎么样?', 12, ['a', 'list'], + ('a', 'tuple'), {'a': 'dict'}, None,) for m in bad_data_types: with self.assertRaises(TypeError): logging.debug("attempting to send message of type %s", type(m)) @@ -44,6 +45,25 @@ class TestKafkaProducer(unittest.TestCase): # This should not raise an exception producer.send_messages(topic, partition, m) + def test_keyedproducer_message_types(self): + client = MagicMock() + client.get_partition_ids_for_topic.return_value = [0, 1] + producer = KeyedProducer(client) + topic = b"test-topic" + key = 'testkey' + + bad_data_types = (u'你怎么样?', 12, ['a', 'list'], + ('a', 'tuple'), {'a': 'dict'},) + for m in bad_data_types: + with self.assertRaises(TypeError): + logging.debug("attempting to send message of type %s", type(m)) + producer.send_messages(topic, key, m) + + good_data_types = (b'a string!', None,) + for m in good_data_types: + # This should not raise an exception + producer.send_messages(topic, key, m) + def test_topic_message_types(self): client = MagicMock() From 4df78c293f162c5b99f18c860af10ddafc0c8861 Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 17 Jun 2015 16:11:37 +0300 Subject: [PATCH 203/250] Add KeyedProducer test with null payloads --- test/test_producer_integration.py | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index abf34c3..46b6851 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -341,6 +341,28 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # KeyedProducer Tests # ############################ + @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0") + def test_keyedproducer_null_payload(self): + partitions = self.client.get_partition_ids_for_topic(self.topic) + start_offsets = [self.current_offset(self.topic, p) for p in partitions] + + producer = KeyedProducer(self.client, partitioner=RoundRobinPartitioner) + key = "test" + + resp = producer.send_messages(self.topic, self.key("key1"), self.msg("one")) + self.assert_produce_response(resp, start_offsets[0]) + resp = producer.send_messages(self.topic, self.key("key2"), None) + self.assert_produce_response(resp, start_offsets[1]) + resp = producer.send_messages(self.topic, self.key("key3"), None) + self.assert_produce_response(resp, start_offsets[0]+1) + resp = producer.send_messages(self.topic, self.key("key4"), self.msg("four")) + self.assert_produce_response(resp, start_offsets[1]+1) + + self.assert_fetch_offset(partitions[0], start_offsets[0], [ self.msg("one"), None ]) + self.assert_fetch_offset(partitions[1], start_offsets[1], [ None, self.msg("four") ]) + + producer.stop() + @kafka_versions("all") def test_round_robin_partitioner(self): partitions = self.client.get_partition_ids_for_topic(self.topic) From 3fbd1cc80965a0f49bfd6c505a1c021af95354fb Mon Sep 17 00:00:00 2001 From: Viktor Shlapakov Date: Wed, 17 Jun 2015 17:16:38 +0300 Subject: [PATCH 204/250] Producer test fix for python3 --- test/test_producer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_producer.py b/test/test_producer.py index 297b265..e681e43 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -50,7 +50,7 @@ class TestKafkaProducer(unittest.TestCase): client.get_partition_ids_for_topic.return_value = [0, 1] producer = KeyedProducer(client) topic = b"test-topic" - key = 'testkey' + key = b"testkey" bad_data_types = (u'你怎么样?', 12, ['a', 'list'], ('a', 'tuple'), {'a': 'dict'},) From f74d712adfda21f6c24949b1d00791c0de7bb049 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Fri, 19 Jun 2015 10:17:49 -0700 Subject: [PATCH 205/250] Lower logging level on replica not available and commit --- kafka/client.py | 2 +- kafka/consumer/base.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index cf29eda..817c621 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -409,7 +409,7 @@ class KafkaClient(object): # this error code is provided for admin purposes only # we never talk to replicas, only the leader except ReplicaNotAvailableError: - log.warning('Some (non-leader) replicas not available for topic %s partition %d', topic, partition) + log.debug('Some (non-leader) replicas not available for topic %s partition %d', topic, partition) # If Known Broker, topic_partition -> BrokerMetadata if leader in self.brokers: diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index b5383a3..0800327 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -138,7 +138,7 @@ class Consumer(object): if partitions is None: # commit all partitions partitions = list(self.offsets.keys()) - log.info('Committing new offsets for %s, partitions %s', + log.debug('Committing new offsets for %s, partitions %s', self.topic, partitions) for partition in partitions: offset = self.offsets[partition] From 2028a232098abeb89a8125e26abc4f4a379ef1b9 Mon Sep 17 00:00:00 2001 From: haosdent Date: Sat, 20 Jun 2015 20:04:57 +0800 Subject: [PATCH 206/250] fix #410 SimpleConsumer cannot seek to an absolute offset. --- kafka/consumer/simple.py | 63 ++++++++++++++++++++++--------- test/test_consumer_integration.py | 14 +++++++ 2 files changed, 60 insertions(+), 17 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index b08255b..733baa8 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -188,33 +188,62 @@ class SimpleConsumer(Consumer): """ self.partition_info = True - def seek(self, offset, whence): + def seek(self, offset, whence=None, partition=None): """ Alter the current offset in the consumer, similar to fseek Arguments: offset: how much to modify the offset - whence: where to modify it from + whence: where to modify it from, default is None - * 0 is relative to the earliest available offset (head) - * 1 is relative to the current offset - * 2 is relative to the latest known offset (tail) + * None is an absolute offset + * 0 is relative to the earliest available offset (head) + * 1 is relative to the current offset + * 2 is relative to the latest known offset (tail) + + partition: modify which partition, default is None. + If partition is None, would modify all partitions. """ - if whence == 1: # relative to current position - for partition, _offset in self.offsets.items(): - self.offsets[partition] = _offset + offset + if whence is None: # set an absolute offset + if partition is None: + for tmp_partition in self.offsets: + self.offsets[tmp_partition] = offset + else: + self.offsets[partition] = offset + elif whence == 1: # relative to current position + if partition is None: + for tmp_partition, _offset in self.offsets.items(): + self.offsets[tmp_partition] = _offset + offset + else: + self.offsets[partition] += offset elif whence in (0, 2): # relative to beginning or end - # divide the request offset by number of partitions, - # distribute the remained evenly - (delta, rem) = divmod(offset, len(self.offsets)) - deltas = {} - for partition, r in izip_longest(self.offsets.keys(), - repeat(1, rem), fillvalue=0): - deltas[partition] = delta + r - reqs = [] - for partition in self.offsets.keys(): + deltas = {} + if partition is None: + # divide the request offset by number of partitions, + # distribute the remained evenly + (delta, rem) = divmod(offset, len(self.offsets)) + for tmp_partition, r in izip_longest(self.offsets.keys(), + repeat(1, rem), + fillvalue=0): + deltas[tmp_partition] = delta + r + + for tmp_partition in self.offsets.keys(): + if whence == 0: + reqs.append(OffsetRequest(self.topic, + tmp_partition, + -2, + 1)) + elif whence == 2: + reqs.append(OffsetRequest(self.topic, + tmp_partition, + -1, + 1)) + else: + pass + else: + deltas[partition] = offset if whence == 0: reqs.append(OffsetRequest(self.topic, partition, -2, 1)) elif whence == 2: diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index df2eeea..52b3e85 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -164,6 +164,20 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer.seek(-13, 2) self.assert_message_count([ message for message in consumer ], 13) + # Set absolute offset + consumer.seek(100) + self.assert_message_count([ message for message in consumer ], 0) + consumer.seek(100, partition=0) + self.assert_message_count([ message for message in consumer ], 0) + consumer.seek(101, partition=1) + self.assert_message_count([ message for message in consumer ], 0) + consumer.seek(90, partition=0) + self.assert_message_count([ message for message in consumer ], 10) + consumer.seek(20, partition=1) + self.assert_message_count([ message for message in consumer ], 80) + consumer.seek(0, partition=1) + self.assert_message_count([ message for message in consumer ], 100) + consumer.stop() @kafka_versions("all") From 43164ad096cf3dc9f252d0c6e4377ff025bb01f0 Mon Sep 17 00:00:00 2001 From: Martin Olveyra Date: Mon, 29 Jun 2015 13:28:12 -0200 Subject: [PATCH 207/250] allow to retrieve partition info in mp consumer --- kafka/consumer/base.py | 8 ++++++++ kafka/consumer/multiprocess.py | 3 ++- kafka/consumer/simple.py | 7 ------- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 0800327..25c01a1 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -83,6 +83,14 @@ class Consumer(object): self._cleanup_func = cleanup atexit.register(cleanup, self) + self.partition_info = False # Do not return partition info in msgs + + def provide_partition_info(self): + """ + Indicates that partition info must be returned by the consumer + """ + self.partition_info = True + def fetch_last_known_offsets(self, partitions=None): if self.group is None: raise ValueError('KafkaClient.group must not be None') diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index d03eb95..bd784cf 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -257,7 +257,8 @@ class MultiProcessConsumer(Consumer): except Empty: break - messages.append(message) + _msg = (partition, message) if self.partition_info else message + messages.append(_msg) new_offsets[partition] = message.offset + 1 count -= 1 if timeout is not None: diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 733baa8..9b85f8c 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -131,7 +131,6 @@ class SimpleConsumer(Consumer): (buffer_size, max_buffer_size)) self.buffer_size = buffer_size self.max_buffer_size = max_buffer_size - self.partition_info = False # Do not return partition info in msgs self.fetch_max_wait_time = FETCH_MAX_WAIT_TIME self.fetch_min_bytes = fetch_size_bytes self.fetch_offsets = self.offsets.copy() @@ -182,12 +181,6 @@ class SimpleConsumer(Consumer): self.fetch_offsets[partition] = resp.offsets[0] return resp.offsets[0] - def provide_partition_info(self): - """ - Indicates that partition info must be returned by the consumer - """ - self.partition_info = True - def seek(self, offset, whence=None, partition=None): """ Alter the current offset in the consumer, similar to fseek From c01963682e1ec16dbde9b55cda4ec95ad2346476 Mon Sep 17 00:00:00 2001 From: tpalino Date: Wed, 1 Jul 2015 22:29:56 -0700 Subject: [PATCH 208/250] Support consumer metadata requests Support added for ConsumerMetadataRequest and ConsumerMetadataResponse Added consumer-aware request routine for supporting the consumer coordinator Added separate client method for fetching Kafka-committed offsets from the coordinator --- kafka/client.py | 128 ++++++++++++++++++++++++++++++++++++++++++ kafka/common.py | 22 ++++++++ kafka/protocol.py | 53 +++++++++++++++-- test/test_protocol.py | 28 +++++++++ 4 files changed, 225 insertions(+), 6 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 817c621..cf180b4 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -97,6 +97,26 @@ class KafkaClient(object): # Otherwise return the BrokerMetadata return self.brokers[meta.leader] + def _get_coordinator_for_group(self, group): + """ + Returns the coordinator broker for a consumer group. + + ConsumerCoordinatorNotAvailableCode will be raised if the coordinator + does not currently exist for the group. + + OffsetsLoadInProgressCode is raised if the coordinator is available + but is still loading offsets from the internal topic + """ + + resp = self.send_consumer_metadata_request(group) + + # If there's a problem with finding the coordinator, raise the + # provided error + kafka.common.check_error(resp) + + # Otherwise return the BrokerMetadata + return BrokerMetadata(resp.nodeId, resp.host, resp.port) + def _next_id(self): """Generate a new correlation id""" # modulo to keep w/i int32 @@ -237,6 +257,96 @@ class KafkaClient(object): # Return responses in the same order as provided return [responses[tp] for tp in original_ordering] + def _send_consumer_aware_request(self, group, payloads, encoder_fn, decoder_fn): + """ + Send a list of requests to the consumer coordinator for the group + specified using the supplied encode/decode functions. As the payloads + that use consumer-aware requests do not contain the group (e.g. + OffsetFetchRequest), all payloads must be for a single group. + + Arguments: + + group: the name of the consumer group (str) the payloads are for + payloads: list of object-like entities with topic (str) and + partition (int) attributes; payloads with duplicate + topic+partition are not supported. + + encode_fn: a method to encode the list of payloads to a request body, + must accept client_id, correlation_id, and payloads as + keyword arguments + + decode_fn: a method to decode a response body into response objects. + The response objects must be object-like and have topic + and partition attributes + + Returns: + + List of response objects in the same order as the supplied payloads + """ + # encoders / decoders do not maintain ordering currently + # so we need to keep this so we can rebuild order before returning + original_ordering = [(p.topic, p.partition) for p in payloads] + + broker = self._get_coordinator_for_group(group) + + # Send the list of request payloads and collect the responses and + # errors + responses = {} + requestId = self._next_id() + log.debug('Request %s to %s: %s', requestId, broker, payloads) + request = encoder_fn(client_id=self.client_id, + correlation_id=requestId, payloads=payloads) + + # Send the request, recv the response + try: + conn = self._get_conn(broker.host.decode('utf-8'), broker.port) + conn.send(requestId, request) + + except ConnectionError as e: + log.warning('ConnectionError attempting to send request %s ' + 'to server %s: %s', requestId, broker, e) + + for payload in payloads: + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = FailedPayloadsError(payload) + + # No exception, try to get response + else: + + # decoder_fn=None signal that the server is expected to not + # send a response. This probably only applies to + # ProduceRequest w/ acks = 0 + if decoder_fn is None: + log.debug('Request %s does not expect a response ' + '(skipping conn.recv)', requestId) + for payload in payloads: + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = None + return [] + + try: + response = conn.recv(requestId) + except ConnectionError as e: + log.warning('ConnectionError attempting to receive a ' + 'response to request %s from server %s: %s', + requestId, broker, e) + + for payload in payloads: + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = FailedPayloadsError(payload) + + else: + _resps = [] + for payload_response in decoder_fn(response): + topic_partition = (payload_response.topic, + payload_response.partition) + responses[topic_partition] = payload_response + _resps.append(payload_response) + log.debug('Response %s: %s', requestId, _resps) + + # Return responses in the same order as provided + return [responses[tp] for tp in original_ordering] + def __repr__(self): return '' % (self.client_id) @@ -429,6 +539,13 @@ class KafkaClient(object): return self._send_broker_unaware_request(payloads, encoder, decoder) + def send_consumer_metadata_request(self, payloads=[], fail_on_error=True, + callback=None): + encoder = KafkaProtocol.encode_consumer_metadata_request + decoder = KafkaProtocol.decode_consumer_metadata_response + + return self._send_broker_unaware_request(payloads, encoder, decoder) + def send_produce_request(self, payloads=[], acks=1, timeout=1000, fail_on_error=True, callback=None): """ @@ -529,3 +646,14 @@ class KafkaClient(object): return [resp if not callback else callback(resp) for resp in resps if not fail_on_error or not self._raise_on_response_error(resp)] + + def send_offset_fetch_request_kafka(self, group, payloads=[], + fail_on_error=True, callback=None): + + encoder = functools.partial(KafkaProtocol.encode_offset_fetch_request, + group=group, from_kafka=True) + decoder = KafkaProtocol.decode_offset_fetch_response + resps = self._send_consumer_aware_request(group, payloads, encoder, decoder) + + return [resp if not callback else callback(resp) for resp in resps + if not fail_on_error or not self._raise_on_response_error(resp)] diff --git a/kafka/common.py b/kafka/common.py index 66987ff..a7d8164 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -13,6 +13,13 @@ MetadataRequest = namedtuple("MetadataRequest", MetadataResponse = namedtuple("MetadataResponse", ["brokers", "topics"]) +# https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ConsumerMetadataRequest +ConsumerMetadataRequest = namedtuple("ConsumerMetadataRequest", + ["groups"]) + +ConsumerMetadataResponse = namedtuple("ConsumerMetadataResponse", + ["error", "nodeId", "host", "port"]) + # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ProduceAPI ProduceRequest = namedtuple("ProduceRequest", ["topic", "partition", "messages"]) @@ -160,6 +167,21 @@ class StaleLeaderEpochCodeError(BrokerResponseError): message = 'STALE_LEADER_EPOCH_CODE' +class OffsetsLoadInProgressCode(BrokerResponseError): + errno = 14 + message = 'OFFSETS_LOAD_IN_PROGRESS_CODE' + + +class ConsumerCoordinatorNotAvailableCode(BrokerResponseError): + errno = 15 + message = 'CONSUMER_COORDINATOR_NOT_AVAILABLE_CODE' + + +class NotCoordinatorForConsumerCode(BrokerResponseError): + errno = 16 + message = 'NOT_COORDINATOR_FOR_CONSUMER_CODE' + + class KafkaUnavailableError(KafkaError): pass diff --git a/kafka/protocol.py b/kafka/protocol.py index d5adf89..1f3ea2f 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -14,7 +14,8 @@ from kafka.common import ( MetadataResponse, ProduceResponse, FetchResponse, OffsetResponse, OffsetCommitResponse, OffsetFetchResponse, ProtocolError, BufferUnderflowError, ChecksumError, - ConsumerFetchSizeTooSmall, UnsupportedCodecError + ConsumerFetchSizeTooSmall, UnsupportedCodecError, + ConsumerMetadataResponse ) from kafka.util import ( crc32, read_short_string, read_int_string, relative_unpack, @@ -43,19 +44,21 @@ class KafkaProtocol(object): METADATA_KEY = 3 OFFSET_COMMIT_KEY = 8 OFFSET_FETCH_KEY = 9 + CONSUMER_METADATA_KEY = 10 ################### # Private API # ################### @classmethod - def _encode_message_header(cls, client_id, correlation_id, request_key): + def _encode_message_header(cls, client_id, correlation_id, request_key, + version=0): """ Encode the common request envelope """ return struct.pack('>hhih%ds' % len(client_id), request_key, # ApiKey - 0, # ApiVersion + version, # ApiVersion correlation_id, # CorrelationId len(client_id), # ClientId size client_id) # ClientId @@ -429,6 +432,38 @@ class KafkaProtocol(object): return MetadataResponse(brokers, topic_metadata) + @classmethod + def encode_consumer_metadata_request(cls, client_id, correlation_id, payloads): + """ + Encode a ConsumerMetadataRequest + + Arguments: + client_id: string + correlation_id: int + payloads: string (consumer group) + """ + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.CONSUMER_METADATA_KEY)) + message.append(struct.pack('>h%ds' % len(payloads), len(payloads), payloads)) + + msg = b''.join(message) + return write_int_string(msg) + + @classmethod + def decode_consumer_metadata_response(cls, data): + """ + Decode bytes to a ConsumerMetadataResponse + + Arguments: + data: bytes to decode + """ + ((correlation_id, error, nodeId), cur) = relative_unpack('>ihi', data, 0) + (host, cur) = read_short_string(data, cur) + ((port,), cur) = relative_unpack('>i', data, cur) + + return ConsumerMetadataResponse(error, nodeId, host, port) + @classmethod def encode_offset_commit_request(cls, client_id, correlation_id, group, payloads): @@ -481,21 +516,27 @@ class KafkaProtocol(object): @classmethod def encode_offset_fetch_request(cls, client_id, correlation_id, - group, payloads): + group, payloads, from_kafka=False): """ - Encode some OffsetFetchRequest structs + Encode some OffsetFetchRequest structs. The request is encoded using + version 0 if from_kafka is false, indicating a request for Zookeeper + offsets. It is encoded using version 1 otherwise, indicating a request + for Kafka offsets. Arguments: client_id: string correlation_id: int group: string, the consumer group you are fetching offsets for payloads: list of OffsetFetchRequest + from_kafka: bool, default False, set True for Kafka-committed offsets """ grouped_payloads = group_by_topic_and_partition(payloads) message = [] + reqver = 1 if from_kafka else 0 message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_FETCH_KEY)) + KafkaProtocol.OFFSET_FETCH_KEY, + version=reqver)) message.append(write_short_string(group)) message.append(struct.pack('>i', len(grouped_payloads))) diff --git a/test/test_protocol.py b/test/test_protocol.py index 0938228..1f967f2 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -560,6 +560,34 @@ class TestProtocol(unittest.TestCase): decoded = KafkaProtocol.decode_metadata_response(encoded) self.assertEqual(decoded, (node_brokers, topic_partitions)) + def test_encode_consumer_metadata_request(self): + expected = b"".join([ + struct.pack(">i", 17), # Total length of the request + struct.pack('>h', 10), # API key consumer metadata + struct.pack('>h', 0), # API version + struct.pack('>i', 4), # Correlation ID + struct.pack('>h3s', 3, b"cid"),# The client ID + struct.pack('>h2s', 2, b"g1"), # Group "g1" + ]) + + encoded = KafkaProtocol.encode_consumer_metadata_request(b"cid", 4, b"g1") + + self.assertEqual(encoded, expected) + + def test_decode_consumer_metadata_response(self): + encoded = b"".join([ + struct.pack(">i", 42), # Correlation ID + struct.pack(">h", 0), # No Error + struct.pack(">i", 1), # Broker ID + struct.pack(">h23s", 23, b"brokers1.kafka.rdio.com"), # Broker Host + struct.pack(">i", 1000), # Broker Port + ]) + + results = KafkaProtocol.decode_consumer_metadata_response(encoded) + self.assertEqual(results, + ConsumerMetadataResponse(error = 0, node = 1, host = b'brokers1.kafka.rdio.com', port = 1000) + ) + def test_encode_offset_request(self): expected = b"".join([ struct.pack(">i", 21), # Total length of the request From 5d8d5412e576c5514497be3809ea899378f40e56 Mon Sep 17 00:00:00 2001 From: tpalino Date: Wed, 1 Jul 2015 22:47:02 -0700 Subject: [PATCH 209/250] Add missing import for ConsumerMetadataResponse --- test/test_protocol.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_protocol.py b/test/test_protocol.py index 1f967f2..f5bc289 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -13,7 +13,7 @@ from kafka.common import ( ProduceResponse, FetchResponse, OffsetAndMessage, BrokerMetadata, TopicMetadata, PartitionMetadata, TopicAndPartition, KafkaUnavailableError, UnsupportedCodecError, ConsumerFetchSizeTooSmall, - ProtocolError + ProtocolError, ConsumerMetadataResponse ) from kafka.protocol import ( ATTRIBUTE_CODEC_MASK, CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY, KafkaProtocol, From da03827d12520bd9c8c5b35bb43e35168f09771a Mon Sep 17 00:00:00 2001 From: tpalino Date: Thu, 2 Jul 2015 09:32:37 -0700 Subject: [PATCH 210/250] Fix misnamed node to nodeId in test --- test/test_protocol.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_protocol.py b/test/test_protocol.py index f5bc289..ac7bea6 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -585,7 +585,7 @@ class TestProtocol(unittest.TestCase): results = KafkaProtocol.decode_consumer_metadata_response(encoded) self.assertEqual(results, - ConsumerMetadataResponse(error = 0, node = 1, host = b'brokers1.kafka.rdio.com', port = 1000) + ConsumerMetadataResponse(error = 0, nodeId = 1, host = b'brokers1.kafka.rdio.com', port = 1000) ) def test_encode_offset_request(self): From b47736dc26c28bd5446f6d4020944a3cbce5d83f Mon Sep 17 00:00:00 2001 From: Ulysse Carion Date: Tue, 7 Jul 2015 16:03:44 -0700 Subject: [PATCH 211/250] Add KafkaConsumer#close --- kafka/consumer/kafka.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 11c4221..efe9105 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -268,6 +268,10 @@ class KafkaConsumer(object): # Reset message iterator in case we were in the middle of one self._reset_message_iterator() + def close(self): + """Close this consumer's underlying client.""" + self._client.close() + def next(self): """Return the next available message From 718e5fb66da5dca449aa31d305b8867fba4f783c Mon Sep 17 00:00:00 2001 From: Oliver Jowett Date: Sun, 19 Jul 2015 10:57:37 +0000 Subject: [PATCH 212/250] Treat KafkaUnavailableError like other errors. --- kafka/client.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 817c621..dbd9863 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -162,11 +162,16 @@ class KafkaClient(object): payloads_by_broker = collections.defaultdict(list) for payload in payloads: - leader = self._get_leader_for_partition(payload.topic, - payload.partition) - - payloads_by_broker[leader].append(payload) - brokers_for_payloads.append(leader) + try: + leader = self._get_leader_for_partition(payload.topic, + payload.partition) + payloads_by_broker[leader].append(payload) + brokers_for_payloads.append(leader) + except KafkaUnavailableError as e: + log.warning('KafkaUnavailableError attempting to send request ' + 'on topic %s partition %d', payload.topic, payload.partition) + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = FailedPayloadsErrors(payload) # For each broker, send the list of request payloads # and collect the responses and errors From 3376ed1cef3e29877f773017117d90192ccf9a5e Mon Sep 17 00:00:00 2001 From: Oliver Jowett Date: Mon, 20 Jul 2015 20:25:02 +0000 Subject: [PATCH 213/250] Errors -> Error typo --- kafka/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index dbd9863..87b51c3 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -171,7 +171,7 @@ class KafkaClient(object): log.warning('KafkaUnavailableError attempting to send request ' 'on topic %s partition %d', payload.topic, payload.partition) topic_partition = (payload.topic, payload.partition) - responses[topic_partition] = FailedPayloadsErrors(payload) + responses[topic_partition] = FailedPayloadsError(payload) # For each broker, send the list of request payloads # and collect the responses and errors From 00ceada0281d8cbfeec3b676cd94f7c64ce351f0 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Tue, 21 Jul 2015 11:09:46 -0700 Subject: [PATCH 214/250] Use debug logging level for metadata request --- kafka/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 817c621..d8d77e4 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -363,8 +363,8 @@ class KafkaClient(object): resp = self.send_metadata_request(topics) - log.info('Updating broker metadata: %s', resp.brokers) - log.info('Updating topic metadata: %s', resp.topics) + log.debug('Updating broker metadata: %s', resp.brokers) + log.debug('Updating topic metadata: %s', resp.topics) self.brokers = dict([(broker.nodeId, broker) for broker in resp.brokers]) From 77e5180a377197c8157a19d5603ad2653c238aa3 Mon Sep 17 00:00:00 2001 From: Oliver Jowett Date: Fri, 24 Jul 2015 10:53:01 +0000 Subject: [PATCH 215/250] Init responses before we use it. --- kafka/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index 87b51c3..9846acf 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -161,6 +161,7 @@ class KafkaClient(object): brokers_for_payloads = [] payloads_by_broker = collections.defaultdict(list) + responses = {} for payload in payloads: try: leader = self._get_leader_for_partition(payload.topic, @@ -175,7 +176,6 @@ class KafkaClient(object): # For each broker, send the list of request payloads # and collect the responses and errors - responses = {} broker_failures = [] for broker, payloads in payloads_by_broker.items(): requestId = self._next_id() From 1f857351391f1d50dd57b404cec313e61cf312f3 Mon Sep 17 00:00:00 2001 From: toli Date: Thu, 23 Jul 2015 17:06:41 -0700 Subject: [PATCH 216/250] Fixing https://github.com/mumrah/kafka-python/issues/434 Reworked the if statement logic to only call stop() on not-stopped objects. added tests --- kafka/producer/base.py | 2 +- test/test_producer.py | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 3c826cd..8feb27c 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -322,7 +322,7 @@ class Producer(object): self.thread.start() def cleanup(obj): - if obj.stopped: + if not obj.stopped: obj.stop() self._cleanup_func = cleanup atexit.register(cleanup, self) diff --git a/test/test_producer.py b/test/test_producer.py index 27272f6..087e3ae 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -91,6 +91,20 @@ class TestKafkaProducer(unittest.TestCase): with self.assertRaises(FailedPayloadsError): producer.send_messages('foobar', b'test message') + def test_cleanup_stop_is_called_on_not_stopped_object(self): + producer = Producer(MagicMock(), async=True) + producer.stopped = True + with patch('kafka.producer.base.Producer.stop') as base_stop: + producer._cleanup_func(producer) + self.assertEqual(base_stop.call_count, 0) + + def test_cleanup_stop_is_not_called_on_stopped_object(self): + producer = Producer(MagicMock(), async=True) + producer.stopped = False + with patch('kafka.producer.base.Producer.stop') as base_stop: + producer._cleanup_func(producer) + self.assertEqual(base_stop.call_count, 1) + class TestKafkaProducerSendUpstream(unittest.TestCase): From bb6283df6f9331595020d1d8c28818eeb1131957 Mon Sep 17 00:00:00 2001 From: Chris Chamberlin Date: Tue, 28 Jul 2015 22:38:28 -0700 Subject: [PATCH 217/250] Fix translation of Java murmur2 code, fix byte encoding for Python 3. Avoid further type changes when Murmur2Partitioner is passed a byte array. Change leftover-bytes logic to match the Java switch statement, and add tests to verify that partition selection matches Java implementation. --- kafka/partitioner/hashed.py | 22 ++++++++++++---------- test/test_partitioner.py | 23 +++++++++++++++++++++++ 2 files changed, 35 insertions(+), 10 deletions(-) create mode 100644 test/test_partitioner.py diff --git a/kafka/partitioner/hashed.py b/kafka/partitioner/hashed.py index 6393ce2..d5d6d27 100644 --- a/kafka/partitioner/hashed.py +++ b/kafka/partitioner/hashed.py @@ -1,3 +1,5 @@ +import six + from .base import Partitioner @@ -43,14 +45,16 @@ def murmur2(key): Based on java client, see org.apache.kafka.common.utils.Utils.murmur2 Args: - key: if not a bytearray, converted via bytearray(str(key)) + key: if not a bytes type, encoded using default encoding Returns: MurmurHash2 of key bytearray """ - # Convert key to a bytearray - if not isinstance(key, bytearray): - data = bytearray(str(key)) + # Convert key to bytes or bytearray + if isinstance(key, bytearray) or (six.PY3 and isinstance(key, bytes)): + data = key + else: + data = bytearray(str(key).encode()) length = len(data) seed = 0x9747b28c @@ -61,7 +65,7 @@ def murmur2(key): # Initialize the hash to a random value h = seed ^ length - length4 = length / 4 + length4 = length // 4 for i in range(length4): i4 = i * 4 @@ -84,15 +88,13 @@ def murmur2(key): # Handle the last few bytes of the input array extra_bytes = length % 4 - if extra_bytes == 3: + if extra_bytes >= 3: h ^= (data[(length & ~3) + 2] & 0xff) << 16 h &= 0xffffffff - - if extra_bytes == 2: + if extra_bytes >= 2: h ^= (data[(length & ~3) + 1] & 0xff) << 8 h &= 0xffffffff - - if extra_bytes == 1: + if extra_bytes >= 1: h ^= (data[length & ~3] & 0xff) h &= 0xffffffff h *= m diff --git a/test/test_partitioner.py b/test/test_partitioner.py new file mode 100644 index 0000000..67cd83b --- /dev/null +++ b/test/test_partitioner.py @@ -0,0 +1,23 @@ +import six +from . import unittest + +from kafka.partitioner import (Murmur2Partitioner) + +class TestMurmurPartitioner(unittest.TestCase): + def test_hash_bytes(self): + p = Murmur2Partitioner(range(1000)) + self.assertEqual(p.partition(bytearray(b'test')), p.partition(b'test')) + + def test_hash_encoding(self): + p = Murmur2Partitioner(range(1000)) + self.assertEqual(p.partition('test'), p.partition(u'test')) + + def test_murmur2_java_compatibility(self): + p = Murmur2Partitioner(range(1000)) + # compare with output from Kafka's org.apache.kafka.clients.producer.Partitioner + self.assertEqual(681, p.partition(b'')) + self.assertEqual(524, p.partition(b'a')) + self.assertEqual(434, p.partition(b'ab')) + self.assertEqual(107, p.partition(b'abc')) + self.assertEqual(566, p.partition(b'123456789')) + self.assertEqual(742, p.partition(b'\x00 ')) From 6d0403d6127884053b7c7099152d389d228ca134 Mon Sep 17 00:00:00 2001 From: Kubilay Kocak Date: Fri, 31 Jul 2015 16:03:27 +1000 Subject: [PATCH 218/250] Include LICENSE in PyPI sdist --- MANIFEST.in | 1 + 1 file changed, 1 insertion(+) diff --git a/MANIFEST.in b/MANIFEST.in index bdd6505..1731afa 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -1 +1,2 @@ recursive-include kafka *.py +include LICENSE From e59e6cf4f6d6e935705c0e1bd6aa751446cfd62b Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 13 Aug 2015 12:12:49 -0700 Subject: [PATCH 219/250] set default config on consumer class to support subclass config changes. --- kafka/consumer/kafka.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 11c4221..2141101 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -53,6 +53,7 @@ DEPRECATED_CONFIG_KEYS = { class KafkaConsumer(object): """A simpler kafka consumer""" + DEFAULT_CONFIG = deepcopy(DEFAULT_CONSUMER_CONFIG) def __init__(self, *topics, **configs): self.configure(**configs) @@ -111,8 +112,8 @@ class KafkaConsumer(object): """ configs = self._deprecate_configs(**configs) self._config = {} - for key in DEFAULT_CONSUMER_CONFIG: - self._config[key] = configs.pop(key, DEFAULT_CONSUMER_CONFIG[key]) + for key in self.DEFAULT_CONFIG: + self._config[key] = configs.pop(key, self.DEFAULT_CONFIG[key]) if configs: raise KafkaConfigurationError('Unknown configuration key(s): ' + From 4d516fbd496096ae30191e13b7c2e378654b3759 Mon Sep 17 00:00:00 2001 From: trbs Date: Sat, 12 Sep 2015 18:22:50 +0200 Subject: [PATCH 220/250] allow to specify compression level for codecs which support this --- kafka/codec.py | 7 +++++-- kafka/producer/base.py | 12 ++++++++---- kafka/protocol.py | 8 ++++---- 3 files changed, 17 insertions(+), 10 deletions(-) diff --git a/kafka/codec.py b/kafka/codec.py index 19f405b..a9373c7 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -22,12 +22,15 @@ def has_snappy(): return _HAS_SNAPPY -def gzip_encode(payload): +def gzip_encode(payload, compresslevel=None): + if not compresslevel: + compresslevel = 9 + with BytesIO() as buf: # Gzip context manager introduced in python 2.6 # so old-fashioned way until we decide to not support 2.6 - gzipper = gzip.GzipFile(fileobj=buf, mode="w") + gzipper = gzip.GzipFile(fileobj=buf, mode="w", compresslevel=compresslevel) try: gzipper.write(payload) finally: diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 3c826cd..7b7b7c1 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -47,7 +47,8 @@ SYNC_FAIL_ON_ERROR_DEFAULT = True def _send_upstream(queue, client, codec, batch_time, batch_size, req_acks, ack_timeout, retry_options, stop_event, log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR, - stop_timeout=ASYNC_STOP_TIMEOUT_SECS): + stop_timeout=ASYNC_STOP_TIMEOUT_SECS, + codec_compresslevel=None): """Private method to manage producing messages asynchronously Listens on the queue for a specified number of messages or until @@ -123,7 +124,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # Send collected requests upstream for topic_partition, msg in msgset.items(): - messages = create_message_set(msg, codec, key) + messages = create_message_set(msg, codec, key, codec_compresslevel) req = ProduceRequest(topic_partition.topic, topic_partition.partition, tuple(messages)) @@ -267,6 +268,7 @@ class Producer(object): req_acks=ACK_AFTER_LOCAL_WRITE, ack_timeout=DEFAULT_ACK_TIMEOUT, codec=None, + codec_compresslevel=None, sync_fail_on_error=SYNC_FAIL_ON_ERROR_DEFAULT, async=False, batch_send=False, # deprecated, use async @@ -297,6 +299,7 @@ class Producer(object): raise UnsupportedCodecError("Codec 0x%02x unsupported" % codec) self.codec = codec + self.codec_compresslevel = codec_compresslevel if self.async: # Messages are sent through this queue @@ -314,7 +317,8 @@ class Producer(object): self.req_acks, self.ack_timeout, async_retry_options, self.thread_stop_event), kwargs={'log_messages_on_error': async_log_messages_on_error, - 'stop_timeout': async_stop_timeout} + 'stop_timeout': async_stop_timeout, + 'codec_compresslevel': self.codec_compresslevel} ) # Thread will die if main thread exits @@ -382,7 +386,7 @@ class Producer(object): 'Current queue size %d.' % self.queue.qsize()) resp = [] else: - messages = create_message_set([(m, key) for m in msg], self.codec, key) + messages = create_message_set([(m, key) for m in msg], self.codec, key, self.codec_compresslevel) req = ProduceRequest(topic, partition, messages) try: resp = self.client.send_produce_request( diff --git a/kafka/protocol.py b/kafka/protocol.py index d5adf89..a916974 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -547,7 +547,7 @@ def create_message(payload, key=None): return Message(0, 0, key, payload) -def create_gzip_message(payloads, key=None): +def create_gzip_message(payloads, key=None, compresslevel=None): """ Construct a Gzipped Message containing multiple Messages @@ -562,7 +562,7 @@ def create_gzip_message(payloads, key=None): message_set = KafkaProtocol._encode_message_set( [create_message(payload, pl_key) for payload, pl_key in payloads]) - gzipped = gzip_encode(message_set) + gzipped = gzip_encode(message_set, compresslevel=compresslevel) codec = ATTRIBUTE_CODEC_MASK & CODEC_GZIP return Message(0, 0x00 | codec, key, gzipped) @@ -589,7 +589,7 @@ def create_snappy_message(payloads, key=None): return Message(0, 0x00 | codec, key, snapped) -def create_message_set(messages, codec=CODEC_NONE, key=None): +def create_message_set(messages, codec=CODEC_NONE, key=None, compresslevel=None): """Create a message set using the given codec. If codec is CODEC_NONE, return a list of raw Kafka messages. Otherwise, @@ -598,7 +598,7 @@ def create_message_set(messages, codec=CODEC_NONE, key=None): if codec == CODEC_NONE: return [create_message(m, k) for m, k in messages] elif codec == CODEC_GZIP: - return [create_gzip_message(messages, key)] + return [create_gzip_message(messages, key, compresslevel)] elif codec == CODEC_SNAPPY: return [create_snappy_message(messages, key)] else: From 7e19de555007b1059af78097aa70c8810a473513 Mon Sep 17 00:00:00 2001 From: Ross Duggan Date: Mon, 14 Sep 2015 16:30:01 +0100 Subject: [PATCH 221/250] Support retry semantics in MultiProcessConsumer --- kafka/conn.py | 4 ++ kafka/consumer/base.py | 1 + kafka/consumer/multiprocess.py | 98 +++++++++++++++++++--------------- 3 files changed, 60 insertions(+), 43 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 432e10b..e6a1f74 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -151,6 +151,10 @@ class KafkaConnection(local): """ log.debug("Reading response %d from Kafka" % request_id) + # Make sure we have a connection + if not self._sock: + self.reinit() + # Read the size off of the header resp = self._read_bytes(4) (size,) = struct.unpack('>i', resp) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 0800327..a22d039 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -29,6 +29,7 @@ ITER_TIMEOUT_SECONDS = 60 NO_MESSAGES_WAIT_TIME_SECONDS = 0.1 FULL_QUEUE_WAIT_TIME_SECONDS = 0.1 +MAX_BACKOFF_SECONDS = 60 class Consumer(object): """ diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index d03eb95..0bef4c5 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -9,11 +9,13 @@ except ImportError: from queue import Empty, Full # python 2 import time +from ..common import KafkaError from .base import ( Consumer, AUTO_COMMIT_MSG_COUNT, AUTO_COMMIT_INTERVAL, NO_MESSAGES_WAIT_TIME_SECONDS, - FULL_QUEUE_WAIT_TIME_SECONDS + FULL_QUEUE_WAIT_TIME_SECONDS, + MAX_BACKOFF_SECONDS, ) from .simple import SimpleConsumer @@ -33,57 +35,67 @@ def _mp_consume(client, group, topic, queue, size, events, **consumer_options): functionality breaks unless this function is kept outside of a class """ - # Make the child processes open separate socket connections - client.reinit() + # Initial interval for retries in seconds. + interval = 1 + while not events.exit.is_set(): + try: + # Make the child processes open separate socket connections + client.reinit() - # We will start consumers without auto-commit. Auto-commit will be - # done by the master controller process. - consumer = SimpleConsumer(client, group, topic, - auto_commit=False, - auto_commit_every_n=None, - auto_commit_every_t=None, - **consumer_options) + # We will start consumers without auto-commit. Auto-commit will be + # done by the master controller process. + consumer = SimpleConsumer(client, group, topic, + auto_commit=False, + auto_commit_every_n=None, + auto_commit_every_t=None, + **consumer_options) - # Ensure that the consumer provides the partition information - consumer.provide_partition_info() + # Ensure that the consumer provides the partition information + consumer.provide_partition_info() - while True: - # Wait till the controller indicates us to start consumption - events.start.wait() - - # If we are asked to quit, do so - if events.exit.is_set(): - break - - # Consume messages and add them to the queue. If the controller - # indicates a specific number of messages, follow that advice - count = 0 - - message = consumer.get_message() - if message: while True: - try: - queue.put(message, timeout=FULL_QUEUE_WAIT_TIME_SECONDS) + # Wait till the controller indicates us to start consumption + events.start.wait() + + # If we are asked to quit, do so + if events.exit.is_set(): break - except Full: - if events.exit.is_set(): break - count += 1 + # Consume messages and add them to the queue. If the controller + # indicates a specific number of messages, follow that advice + count = 0 - # We have reached the required size. The controller might have - # more than what he needs. Wait for a while. - # Without this logic, it is possible that we run into a big - # loop consuming all available messages before the controller - # can reset the 'start' event - if count == size.value: - events.pause.wait() + message = consumer.get_message() + if message: + while True: + try: + queue.put(message, timeout=FULL_QUEUE_WAIT_TIME_SECONDS) + break + except Full: + if events.exit.is_set(): break - else: - # In case we did not receive any message, give up the CPU for - # a while before we try again - time.sleep(NO_MESSAGES_WAIT_TIME_SECONDS) + count += 1 - consumer.stop() + # We have reached the required size. The controller might have + # more than what he needs. Wait for a while. + # Without this logic, it is possible that we run into a big + # loop consuming all available messages before the controller + # can reset the 'start' event + if count == size.value: + events.pause.wait() + + else: + # In case we did not receive any message, give up the CPU for + # a while before we try again + time.sleep(NO_MESSAGES_WAIT_TIME_SECONDS) + + consumer.stop() + + except KafkaError as e: + # Retry with exponential backoff + log.error("Problem communicating with Kafka (%s), retrying in %d seconds..." % (e, interval)) + time.sleep(interval) + interval = interval*2 if interval*2 < MAX_BACKOFF_SECONDS else MAX_BACKOFF_SECONDS class MultiProcessConsumer(Consumer): From e74a8ba4942891c62ef35f70472f10ee067f89b6 Mon Sep 17 00:00:00 2001 From: Oskari Saarenmaa Date: Fri, 18 Sep 2015 14:06:51 +0300 Subject: [PATCH 222/250] Consumers get_messages: allow blocking until some messages are received Modified MultiProcessConsumer's and SimpleConsumer's `block` argument to allow integer value which defines the number of messages to block for. This allows callers to ask for a high number of messages and block only until some of them are received. Otherwise callers would have to request messages one by one or block for some time. --- kafka/consumer/multiprocess.py | 14 +++++++++----- kafka/consumer/simple.py | 15 +++++++++------ test/test_consumer_integration.py | 18 ++++++++++++++++++ 3 files changed, 36 insertions(+), 11 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index d03eb95..046271b 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -226,10 +226,12 @@ class MultiProcessConsumer(Consumer): Keyword Arguments: count: Indicates the maximum number of messages to be fetched - block: If True, the API will block till some messages are fetched. - timeout: If block is True, the function will block for the specified - time (in seconds) until count messages is fetched. If None, - it will block forever. + block: If True, the API will block till all messages are fetched. + If block is a positive integer the API will block until that + many messages are fetched. + timeout: When blocking is requested the function will block for + the specified time (in seconds) until count messages is + fetched. If None, it will block forever. """ messages = [] @@ -252,8 +254,10 @@ class MultiProcessConsumer(Consumer): if self.queue.empty(): self.events.start.set() + block_next_call = block is True or block > len(messages) try: - partition, message = self.queue.get(block, timeout) + partition, message = self.queue.get(block_next_call, + timeout) except Empty: break diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 733baa8..6e18290 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -272,10 +272,12 @@ class SimpleConsumer(Consumer): Keyword Arguments: count: Indicates the maximum number of messages to be fetched - block: If True, the API will block till some messages are fetched. - timeout: If block is True, the function will block for the specified - time (in seconds) until count messages is fetched. If None, - it will block forever. + block: If True, the API will block till all messages are fetched. + If block is a positive integer the API will block until that + many messages are fetched. + timeout: When blocking is requested the function will block for + the specified time (in seconds) until count messages is + fetched. If None, it will block forever. """ messages = [] if timeout is not None: @@ -286,12 +288,13 @@ class SimpleConsumer(Consumer): while len(messages) < count: block_time = timeout - time.time() log.debug('calling _get_message block=%s timeout=%s', block, block_time) - result = self._get_message(block, block_time, + block_next_call = block is True or block > len(messages) + result = self._get_message(block_next_call, block_time, get_partition_info=True, update_offset=False) log.debug('got %s from _get_messages', result) if not result: - if block and (timeout is None or time.time() <= timeout): + if block_next_call and (timeout is None or time.time() <= timeout): continue break diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 52b3e85..fee53f5 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -204,6 +204,14 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assert_message_count(messages, 5) self.assertGreaterEqual(t.interval, 1) + # Ask for 10 messages, 5 in queue, ask to block for 1 message or 1 + # second, get 5 back, no blocking + self.send_messages(0, range(0, 5)) + with Timer() as t: + messages = consumer.get_messages(count=10, block=1, timeout=1) + self.assert_message_count(messages, 5) + self.assertLessEqual(t.interval, 1) + consumer.stop() @kafka_versions("all") @@ -272,6 +280,16 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assert_message_count(messages, 5) self.assertGreaterEqual(t.interval, 1) + # Ask for 10 messages, 5 in queue, ask to block for 1 message or 1 + # second, get at least one back, no blocking + self.send_messages(0, range(0, 5)) + with Timer() as t: + messages = consumer.get_messages(count=10, block=1, timeout=1) + received_message_count = len(messages) + self.assertGreaterEqual(received_message_count, 1) + self.assert_message_count(messages, received_message_count) + self.assertLessEqual(t.interval, 1) + consumer.stop() @kafka_versions("all") From 64cda819d9e87634b075725321b729aeede8471a Mon Sep 17 00:00:00 2001 From: Balthasar Schopman Date: Thu, 22 Oct 2015 17:12:06 +0200 Subject: [PATCH 223/250] Prevents crashing communication thread of async producer If an uncaught exception occurs in _send_messages() the thread sending data to Kafka (asynchronously) will crash and the queue will never be emptied. To reproduce: 1) Run an Async producer. 2) Kill the Kafka server. 3) Restart the Kafka server. The communication thread dies shortly after step 2. After step 3 the communication does not resume without this commit. The changes in both files prevent an Exception from being thrown through to do main communication process, which could cause the crash. --- kafka/client.py | 6 +++++- kafka/producer/base.py | 5 ++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 13777a4..c05e142 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -478,7 +478,11 @@ class KafkaClient(object): else: decoder = KafkaProtocol.decode_produce_response - resps = self._send_broker_aware_request(payloads, encoder, decoder) + try: + resps = self._send_broker_aware_request(payloads, encoder, decoder) + except Exception: + if fail_on_error: + raise return [resp if not callback else callback(resp) for resp in resps if resp is not None and diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 3c826cd..2281547 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -185,7 +185,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # refresh topic metadata before next retry if retry_state['do_refresh']: log.warn('Async producer forcing metadata refresh metadata before retrying') - client.load_metadata_for_topics() + try: + client.load_metadata_for_topics() + except Exception as e: + log.error("Async producer couldn't reload topic metadata. Error: `%s`", e.message) # Apply retry limit, dropping messages that are over request_tries = dict( From 04920bb89f9d73e4028dbd487719975c65954592 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Mon, 12 Oct 2015 00:11:18 -0700 Subject: [PATCH 224/250] Unblocking broker aware request --- kafka/client.py | 52 ++++++++++++++++++++++++++++++------------------- kafka/conn.py | 5 +++++ 2 files changed, 37 insertions(+), 20 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 13777a4..68277ed 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -2,6 +2,7 @@ import collections import copy import functools import logging +import select import time import kafka.common @@ -177,6 +178,10 @@ class KafkaClient(object): # For each broker, send the list of request payloads # and collect the responses and errors broker_failures = [] + + # For each KafkaConnection we store the real socket so that we can use + # a select to perform unblocking I/O + socket_connection = {} for broker, payloads in payloads_by_broker.items(): requestId = self._next_id() log.debug('Request %s to %s: %s', requestId, broker, payloads) @@ -210,27 +215,34 @@ class KafkaClient(object): topic_partition = (payload.topic, payload.partition) responses[topic_partition] = None continue - - try: - response = conn.recv(requestId) - except ConnectionError as e: - broker_failures.append(broker) - log.warning('ConnectionError attempting to receive a ' - 'response to request %s from server %s: %s', - requestId, broker, e) - - for payload in payloads: - topic_partition = (payload.topic, payload.partition) - responses[topic_partition] = FailedPayloadsError(payload) - else: - _resps = [] - for payload_response in decoder_fn(response): - topic_partition = (payload_response.topic, - payload_response.partition) - responses[topic_partition] = payload_response - _resps.append(payload_response) - log.debug('Response %s: %s', requestId, _resps) + socket_connection[conn.get_connected_socket()] = (conn, broker) + + conn = None + while socket_connection: + sockets = socket_connection.keys() + rlist, _, _ = select.select(sockets, [], [], None) + conn, broker = socket_connection.pop(rlist[0]) + try: + response = conn.recv(requestId) + except ConnectionError as e: + broker_failures.append(broker) + log.warning('ConnectionError attempting to receive a ' + 'response to request %s from server %s: %s', + requestId, broker, e) + + for payload in payloads: + topic_partition = (payload.topic, payload.partition) + responses[topic_partition] = FailedPayloadsError(payload) + + else: + _resps = [] + for payload_response in decoder_fn(response): + topic_partition = (payload_response.topic, + payload_response.partition) + responses[topic_partition] = payload_response + _resps.append(payload_response) + log.debug('Response %s: %s', requestId, _resps) # Connection errors generally mean stale metadata # although sometimes it means incorrect api request diff --git a/kafka/conn.py b/kafka/conn.py index 432e10b..f1a12dc 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -118,6 +118,11 @@ class KafkaConnection(local): # TODO multiplex socket communication to allow for multi-threaded clients + def get_connected_socket(self): + if not self._sock: + self.reinit() + return self._sock + def send(self, request_id, payload): """ Send a request to Kafka From c2adeeab057b825c8cccae67aac822be02293211 Mon Sep 17 00:00:00 2001 From: Enrico Canzonieri Date: Sat, 24 Oct 2015 16:50:46 -0700 Subject: [PATCH 225/250] Add tests. Bug fix. Rename socket_conn dict. --- kafka/client.py | 14 +++++++------- test/test_conn.py | 17 +++++++++++++++++ 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 68277ed..6603a47 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -179,9 +179,9 @@ class KafkaClient(object): # and collect the responses and errors broker_failures = [] - # For each KafkaConnection we store the real socket so that we can use + # For each KafkaConnection keep the real socket so that we can use # a select to perform unblocking I/O - socket_connection = {} + connections_by_socket = {} for broker, payloads in payloads_by_broker.items(): requestId = self._next_id() log.debug('Request %s to %s: %s', requestId, broker, payloads) @@ -216,13 +216,13 @@ class KafkaClient(object): responses[topic_partition] = None continue else: - socket_connection[conn.get_connected_socket()] = (conn, broker) + connections_by_socket[conn.get_connected_socket()] = (conn, broker) conn = None - while socket_connection: - sockets = socket_connection.keys() + while connections_by_socket: + sockets = connections_by_socket.keys() rlist, _, _ = select.select(sockets, [], [], None) - conn, broker = socket_connection.pop(rlist[0]) + conn, broker = connections_by_socket.pop(rlist[0]) try: response = conn.recv(requestId) except ConnectionError as e: @@ -231,7 +231,7 @@ class KafkaClient(object): 'response to request %s from server %s: %s', requestId, broker, e) - for payload in payloads: + for payload in payloads_by_broker[broker]: topic_partition = (payload.topic, payload.partition) responses[topic_partition] = FailedPayloadsError(payload) diff --git a/test/test_conn.py b/test/test_conn.py index 2b70344..1bdfc1e 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -165,6 +165,23 @@ class ConnTest(unittest.TestCase): 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_get_connected_socket(self): + s = self.conn.get_connected_socket() + + self.assertEqual(s, self.MockCreateConn()) + + def test_get_connected_socket_on_dirty_conn(self): + # Dirty the connection + try: + self.conn._raise_connection_error() + except ConnectionError: + pass + + # Test that get_connected_socket tries to connect + self.assertEqual(self.MockCreateConn.call_count, 0) + self.conn.get_connected_socket() + self.assertEqual(self.MockCreateConn.call_count, 1) + def test_close__object_is_reusable(self): # test that sending to a closed connection From cfbdc05f27f4ba9f89d720c08015e48f7c43b2b2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 2 Dec 2015 14:38:02 -0800 Subject: [PATCH 226/250] Add Murmur2Partitioner to kafka __all__ imports - fix issue 471 --- kafka/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/__init__.py b/kafka/__init__.py index 396a8b8..2fc59c6 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -10,7 +10,7 @@ from kafka.protocol import ( create_message, create_gzip_message, create_snappy_message ) from kafka.producer import SimpleProducer, KeyedProducer -from kafka.partitioner import RoundRobinPartitioner, HashedPartitioner +from kafka.partitioner import RoundRobinPartitioner, HashedPartitioner, Murmur2Partitioner from kafka.consumer import SimpleConsumer, MultiProcessConsumer, KafkaConsumer __all__ = [ From ee6b9cb5b1310c48a3ed5b66be0dd0c4dd16dc43 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 2 Dec 2015 15:00:35 -0800 Subject: [PATCH 227/250] Fix python3 / python2 comments re queue/Queue --- kafka/consumer/multiprocess.py | 10 +++++----- kafka/consumer/simple.py | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 5e421d6..18a5014 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -4,9 +4,9 @@ from collections import namedtuple import logging from multiprocessing import Process, Manager as MPManager try: - from Queue import Empty, Full # python 3 + import queue # python 3 except ImportError: - from queue import Empty, Full # python 2 + import Queue as queue # python 2 import time from ..common import KafkaError @@ -71,7 +71,7 @@ def _mp_consume(client, group, topic, queue, size, events, **consumer_options): try: queue.put(message, timeout=FULL_QUEUE_WAIT_TIME_SECONDS) break - except Full: + except queue.Full: if events.exit.is_set(): break count += 1 @@ -220,7 +220,7 @@ class MultiProcessConsumer(Consumer): # TODO: This is a hack and will make the consumer block for # at least one second. Need to find a better way of doing this partition, message = self.queue.get(block=True, timeout=1) - except Empty: + except queue.Empty: break # Count, check and commit messages if necessary @@ -270,7 +270,7 @@ class MultiProcessConsumer(Consumer): try: partition, message = self.queue.get(block_next_call, timeout) - except Empty: + except queue.Empty: break _msg = (partition, message) if self.partition_info else message diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index d8b5826..aad229a 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -6,9 +6,9 @@ except ImportError: from itertools import izip_longest as izip_longest, repeat # python 2 import logging try: - from Queue import Empty, Queue # python 3 + import queue # python 3 except ImportError: - from queue import Empty, Queue # python 2 + import Queue as queue # python 2 import sys import time @@ -136,7 +136,7 @@ class SimpleConsumer(Consumer): self.fetch_offsets = self.offsets.copy() self.iter_timeout = iter_timeout self.auto_offset_reset = auto_offset_reset - self.queue = Queue() + self.queue = queue.Queue() def __repr__(self): return '' % \ @@ -257,7 +257,7 @@ class SimpleConsumer(Consumer): if self.auto_commit: self.commit() - self.queue = Queue() + self.queue = queue.Queue() def get_messages(self, count=1, block=True, timeout=0.1): """ @@ -341,7 +341,7 @@ class SimpleConsumer(Consumer): return partition, message else: return message - except Empty: + except queue.Empty: log.debug('internal queue empty after fetch - returning None') return None From 16f61547787e9cb5be855ec1c4391b3b56765ab8 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Wed, 2 Dec 2015 16:23:01 -0800 Subject: [PATCH 228/250] new pylint disables for pylint 1.5.1 tested locally with `pylint --rcfile=pylint.rc -E kafka test`. should pass travis builds. --- kafka/consumer/kafka.py | 2 +- kafka/consumer/simple.py | 2 +- test/fixtures.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 2141101..21b2bf6 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -661,7 +661,7 @@ class KafkaConsumer(object): # 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 + raise # pylint: disable-msg=E0704 (offset, ) = self.get_partition_offsets(topic, partition, request_time_ms, max_num_offsets=1) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index aad229a..7c63246 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -166,7 +166,7 @@ class SimpleConsumer(Consumer): # 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 + raise # pylint: disable-msg=E0704 # send_offset_request log.info('Resetting topic-partition offset to %s for %s:%d', diff --git a/test/fixtures.py b/test/fixtures.py index 164d0d7..0ae1c1e 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -8,7 +8,7 @@ import time from six.moves import urllib import uuid -from six.moves.urllib.parse import urlparse # pylint: disable-msg=E0611 +from six.moves.urllib.parse import urlparse # pylint: disable-msg=E0611,F0401 from test.service import ExternalService, SpawnedService from test.testutil import get_open_port From 770a9bb4da5839ff4fd7d6f62732f000a1bb88bf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 3 Dec 2015 12:43:07 -0800 Subject: [PATCH 229/250] Update docs and links wrt maintainer change (mumrah -> dpkp) --- AUTHORS.md | 58 ++++++++++++++++++++++++++++++++++++------------ README.rst | 10 ++++----- docs/index.rst | 4 ++-- docs/install.rst | 8 +++---- setup.py | 6 ++--- 5 files changed, 58 insertions(+), 28 deletions(-) diff --git a/AUTHORS.md b/AUTHORS.md index d9ce2ed..99022ff 100644 --- a/AUTHORS.md +++ b/AUTHORS.md @@ -1,19 +1,49 @@ -# Contributors - -Top contributors, listed by contribution. See https://github.com/mumrah/kafka-python/graphs/contributors for the full list - -* David Arthur, [@mumrah](https://github.com/mumrah) +# Current Maintainer * Dana Powers, [@dpkp](https://github.com/dpkp) -* Mahendra M, [@mahendra](https://github.com/mahendra) -* Mark Roberts, [@wizzat](https://github.com/wizzat) -* Omar, [@rdiomar](https://github.com/rdiomar) - RIP, Omar. 2014 -* Viktor Shlapakov, [@vshlapakov](https://github.com/vshlapakov) -* Bruno Renié, [@brutasse](https://github.com/brutasse) -* Marc Labbé, [@mrtheb](https://github.com/mrtheb) -* John Anderson, [@sontek](https://github.com/sontek) -* Ivan Pouzyrevsky, [@sandello](https://github.com/sandello) -* Thomas Dimson, [@cosbynator](https://github.com/cosbynator) + +# Original Author and First Commit +* David Arthur, [@mumrah](https://github.com/mumrah) + +# Contributors - 2015 (alpha by username) * Alex Couture-Beil, [@alexcb](https://github.com/alexcb) +* Ali-Akber Saifee, [@alisaifee](https://github.com/alisaifee) +* Christophe-Marie Duquesne, [@chmduquesne](https://github.com/chmduquesne) +* Thomas Dimson, [@cosbynator](https://github.com/cosbynator) +* Kasper Jacobsen, [@Dinoshauer](https://github.com/Dinoshauer) +* Ross Duggan, [@duggan](https://github.com/duggan) +* Enrico Canzonieri, [@ecanzonieri](https://github.com/ecanzonieri) +* haosdent, [@haosdent](https://github.com/haosdent) +* Arturo Filastò, [@hellais](https://github.com/hellais) +* Job Evers‐Meltzer, [@jobevers](https://github.com/jobevers) +* Martin Olveyra, [@kalessin](https://github.com/kalessin) +* Kubilay Kocak, [@koobs](https://github.com/koobs) +* Matthew L Daniel +* Eric Hewitt, [@meandthewallaby](https://github.com/meandthewallaby) +* Oliver Jowett [@mutability](https://github.com/mutability) +* Shaolei Zhou, [@reAsOn2010](https://github.com/reAsOn2010) +* Oskari Saarenmaa, [@saaros](https://github.com/saaros) +* John Anderson, [@sontek](https://github.com/sontek) +* Eduard Iskandarov, [@toidi](https://github.com/toidi) +* Todd Palino, [@toddpalino](https://github.com/toddpalino) +* trbs, [@trbs](https://github.com/trbs) +* Viktor Shlapakov, [@vshlapakov](https://github.com/vshlapakov) +* Will Daly, [@wedaly](https://github.com/wedaly) +* Warren Kiser, [@wkiser](https://github.com/wkiser) +* William Ting, [@wting](https://github.com/wting) * Zack Dever, [@zackdever](https://github.com/zackdever) +# More Contributors +* Bruno Renié, [@brutasse](https://github.com/brutasse) +* Thomas Dimson, [@cosbynator](https://github.com/cosbynator) +* Jesse Myers, [@jessemyers](https://github.com/jessemyers) +* Mahendra M, [@mahendra](https://github.com/mahendra) +* Miguel Eduardo Gil Biraud, [@mgilbir](https://github.com/mgilbir) +* Marc Labbé, [@mrtheb](https://github.com/mrtheb) +* Patrick Lucas, [@patricklucas](https://github.com/patricklucas) +* Omar Ghishan, [@rdiomar](https://github.com/rdiomar) - RIP, Omar. 2014 +* Ivan Pouzyrevsky, [@sandello](https://github.com/sandello) +* Lou Marvin Caraig, [@se7entyse7en](https://github.com/se7entyse7en) +* waliaashish85, [@waliaashish85](https://github.com/waliaashish85) +* Mark Roberts, [@wizzat](https://github.com/wizzat) + Thanks to all who have contributed! diff --git a/README.rst b/README.rst index e957ee3..c05a240 100644 --- a/README.rst +++ b/README.rst @@ -1,11 +1,11 @@ Kafka Python client ------------------------ -.. image:: https://api.travis-ci.org/mumrah/kafka-python.png?branch=master - :target: https://travis-ci.org/mumrah/kafka-python +.. image:: https://api.travis-ci.org/dpkp/kafka-python.png?branch=master + :target: https://travis-ci.org/dpkp/kafka-python :alt: Build Status -.. image:: https://coveralls.io/repos/mumrah/kafka-python/badge.svg?branch=master - :target: https://coveralls.io/r/mumrah/kafka-python?branch=master +.. image:: https://coveralls.io/repos/dpkp/kafka-python/badge.svg?branch=master + :target: https://coveralls.io/r/dpkp/kafka-python?branch=master :alt: Coverage Status .. image:: https://readthedocs.org/projects/kafka-python/badge/?version=latest @@ -50,4 +50,4 @@ Python versions - pypy (tested on pypy 2.5.0 / python 2.7.8) .. _Full documentation available on ReadTheDocs: http://kafka-python.readthedocs.org/en/latest/ -.. _0.9.4: https://github.com/mumrah/kafka-python/releases/tag/v0.9.4 +.. _0.9.4: https://github.com/dpkp/kafka-python/releases/tag/v0.9.4 diff --git a/docs/index.rst b/docs/index.rst index c499d4c..b7e7ccd 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -16,7 +16,7 @@ see https://groups.google.com/forum/m/#!forum/kafka-clients Status ------ -The current stable version of this package is `0.9.4 `_ and is compatible with: +The current stable version of this package is `0.9.4 `_ and is compatible with: Kafka broker versions @@ -36,7 +36,7 @@ Python versions License ------- -Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE `_. +Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE `_. Contents diff --git a/docs/install.rst b/docs/install.rst index 1dd6d4e..2bc6911 100644 --- a/docs/install.rst +++ b/docs/install.rst @@ -11,7 +11,7 @@ Pip: pip install kafka-python -Releases are also listed at https://github.com/mumrah/kafka-python/releases +Releases are also listed at https://github.com/dpkp/kafka-python/releases Bleeding-Edge @@ -19,21 +19,21 @@ Bleeding-Edge .. code:: bash - git clone https://github.com/mumrah/kafka-python + git clone https://github.com/dpkp/kafka-python pip install ./kafka-python Setuptools: .. code:: bash - git clone https://github.com/mumrah/kafka-python + git clone https://github.com/dpkp/kafka-python easy_install ./kafka-python Using `setup.py` directly: .. code:: bash - git clone https://github.com/mumrah/kafka-python + git clone https://github.com/dpkp/kafka-python cd kafka-python python setup.py install diff --git a/setup.py b/setup.py index 8e4fb66..d50d28d 100644 --- a/setup.py +++ b/setup.py @@ -45,9 +45,9 @@ setup( "kafka.producer", ], - author="David Arthur", - author_email="mumrah@gmail.com", - url="https://github.com/mumrah/kafka-python", + author="Dana Powers", + author_email="dana.powers@gmail.com", + url="https://github.com/dpkp/kafka-python", license="Apache License 2.0", description="Pure Python client for Apache Kafka", long_description=README, From eab0a55962c2578fb90fb80b5e78b3bdbd56dbcd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 10:40:56 -0800 Subject: [PATCH 230/250] Add 0.8.2.2 and 0.9.0.0 resources for integration tests; update default official releases --- build_integration.sh | 2 +- servers/0.8.2.2/resources/kafka.properties | 124 ++++++++++++++++++ servers/0.8.2.2/resources/log4j.properties | 24 ++++ .../0.8.2.2/resources/zookeeper.properties | 21 +++ servers/0.9.0.0/resources/kafka.properties | 124 ++++++++++++++++++ servers/0.9.0.0/resources/log4j.properties | 24 ++++ .../0.9.0.0/resources/zookeeper.properties | 21 +++ 7 files changed, 339 insertions(+), 1 deletion(-) create mode 100644 servers/0.8.2.2/resources/kafka.properties create mode 100644 servers/0.8.2.2/resources/log4j.properties create mode 100644 servers/0.8.2.2/resources/zookeeper.properties create mode 100644 servers/0.9.0.0/resources/kafka.properties create mode 100644 servers/0.9.0.0/resources/log4j.properties create mode 100644 servers/0.9.0.0/resources/zookeeper.properties diff --git a/build_integration.sh b/build_integration.sh index 5395bb8..aa9c399 100755 --- a/build_integration.sh +++ b/build_integration.sh @@ -1,7 +1,7 @@ #!/bin/bash # Versions available for testing via binary distributions -OFFICIAL_RELEASES="0.8.0 0.8.1 0.8.1.1 0.8.2.1" +OFFICIAL_RELEASES="0.8.1.1 0.8.2.2 0.9.0.0" # Useful configuration vars, with sensible defaults if [ -z "$SCALA_VERSION" ]; then diff --git a/servers/0.8.2.2/resources/kafka.properties b/servers/0.8.2.2/resources/kafka.properties new file mode 100644 index 0000000..685aed1 --- /dev/null +++ b/servers/0.8.2.2/resources/kafka.properties @@ -0,0 +1,124 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id={broker_id} + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port={port} + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +host.name={host} + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs={tmp_dir}/data + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions={partitions} +default.replication.factor={replicas} + +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=536870912 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=60000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/0.8.2.2/resources/log4j.properties b/servers/0.8.2.2/resources/log4j.properties new file mode 100644 index 0000000..f863b3b --- /dev/null +++ b/servers/0.8.2.2/resources/log4j.properties @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.logger.kafka=DEBUG, stdout +log4j.logger.org.I0Itec.zkclient.ZkClient=INFO, stdout +log4j.logger.org.apache.zookeeper=INFO, stdout diff --git a/servers/0.8.2.2/resources/zookeeper.properties b/servers/0.8.2.2/resources/zookeeper.properties new file mode 100644 index 0000000..e3fd097 --- /dev/null +++ b/servers/0.8.2.2/resources/zookeeper.properties @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir={tmp_dir} +# the port at which the clients will connect +clientPort={port} +clientPortAddress={host} +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 diff --git a/servers/0.9.0.0/resources/kafka.properties b/servers/0.9.0.0/resources/kafka.properties new file mode 100644 index 0000000..685aed1 --- /dev/null +++ b/servers/0.9.0.0/resources/kafka.properties @@ -0,0 +1,124 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id={broker_id} + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port={port} + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +host.name={host} + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs={tmp_dir}/data + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions={partitions} +default.replication.factor={replicas} + +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=536870912 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=60000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 +# We want to expire kafka broker sessions quickly when brokers die b/c we restart them quickly +zookeeper.session.timeout.ms=500 diff --git a/servers/0.9.0.0/resources/log4j.properties b/servers/0.9.0.0/resources/log4j.properties new file mode 100644 index 0000000..f863b3b --- /dev/null +++ b/servers/0.9.0.0/resources/log4j.properties @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + +log4j.logger.kafka=DEBUG, stdout +log4j.logger.org.I0Itec.zkclient.ZkClient=INFO, stdout +log4j.logger.org.apache.zookeeper=INFO, stdout diff --git a/servers/0.9.0.0/resources/zookeeper.properties b/servers/0.9.0.0/resources/zookeeper.properties new file mode 100644 index 0000000..e3fd097 --- /dev/null +++ b/servers/0.9.0.0/resources/zookeeper.properties @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir={tmp_dir} +# the port at which the clients will connect +clientPort={port} +clientPortAddress={host} +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 From e289ab5931ad1de35489ee4e5ade25cd3c7903fd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 10:15:27 -0800 Subject: [PATCH 231/250] Update travis config to use container infrastructure --- .travis.yml | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 136c19f..b0fa475 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,6 +5,7 @@ python: - 2.7 - 3.3 - 3.4 + - 3.5 - pypy env: @@ -14,16 +15,19 @@ env: - KAFKA_VERSION=0.8.1.1 - KAFKA_VERSION=0.8.2.1 +sudo: false + +addons: + apt: + packages: + - libsnappy-dev + before_install: - - sudo apt-get install libsnappy-dev - ./build_integration.sh install: - pip install tox coveralls - pip install . - # Deal with issue on Travis builders re: multiprocessing.Queue :( - # See https://github.com/travis-ci/travis-cookbooks/issues/155 - - sudo rm -rf /dev/shm && sudo ln -s /run/shm /dev/shm deploy: provider: pypi From 7355bf435977305e020621954ccbca4ddd405291 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 10:45:16 -0800 Subject: [PATCH 232/250] Cache pip and servers/ for travis builds --- .travis.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.travis.yml b/.travis.yml index b0fa475..18ed579 100644 --- a/.travis.yml +++ b/.travis.yml @@ -22,6 +22,11 @@ addons: packages: - libsnappy-dev +cache: + directories: + - $HOME/.cache/pip + - servers/ + before_install: - ./build_integration.sh From 71a789133bf0fc80ec65d5c8588bfc240acd522f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 10:45:56 -0800 Subject: [PATCH 233/250] Add py35 to tox testing move python3 configs to default and special case py26/py27 add py35 to travis_selector.sh --- tox.ini | 12 +++++++----- travis_selector.sh | 2 ++ 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/tox.ini b/tox.ini index a69dc99..1ee1e16 100644 --- a/tox.ini +++ b/tox.ini @@ -1,10 +1,8 @@ [tox] -envlist = lint, py26, py27, pypy, py33, py34, docs +envlist = lint, py26, py27, pypy, py33, py34, py35, docs [testenv] deps = - six - unittest2 nose nose-timer coverage @@ -17,16 +15,20 @@ setenv = PROJECT_ROOT = {toxinidir} passenv = KAFKA_VERSION -[testenv:py33] +[testenv:py26] deps = + six + unittest2 nose nose-timer coverage mock python-snappy -[testenv:py34] +[testenv:py27] deps = + six + unittest2 nose nose-timer coverage diff --git a/travis_selector.sh b/travis_selector.sh index 7a2f45f..5700cbc 100755 --- a/travis_selector.sh +++ b/travis_selector.sh @@ -3,6 +3,8 @@ if [ $1 == "pypy" ]; then echo "pypy" +elif [ $1 == "3.5" ]; then + echo "py35" elif [ $1 == "3.4" ]; then echo "py34" elif [ $1 == "3.3" ]; then From 25a220fd08720cf503dd870bf1b53d653d8aa9c2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 11:17:45 -0800 Subject: [PATCH 234/250] Update setup.py to show python 3.5 compatibility explicitly --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index d50d28d..5fc2417 100644 --- a/setup.py +++ b/setup.py @@ -64,6 +64,7 @@ setup( "Programming Language :: Python :: 3", "Programming Language :: Python :: 3.3", "Programming Language :: Python :: 3.4", + "Programming Language :: Python :: 3.5", "Programming Language :: Python :: Implementation :: PyPy", "Topic :: Software Development :: Libraries :: Python Modules", ] From 8a74c20345e80c28cd92a0eec86fb3f593da6ed3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 11:35:56 -0800 Subject: [PATCH 235/250] Update kafka broker list for travis testing (add 0.9.0.0, bump 0.8.2.2) --- .travis.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 18ed579..1f0baa6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -13,7 +13,8 @@ env: - KAFKA_VERSION=0.8.0 - KAFKA_VERSION=0.8.1 - KAFKA_VERSION=0.8.1.1 - - KAFKA_VERSION=0.8.2.1 + - KAFKA_VERSION=0.8.2.2 + - KAFKA_VERSION=0.9.0.0 sudo: false From 7dec992a4ebd4d98008aaa1e65a32f46db1b064a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 16:29:29 -0800 Subject: [PATCH 236/250] client.reinit() can raise an exception; catch in async producer --- kafka/producer/base.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 8774c66..f2c7cfe 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -78,9 +78,17 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, retrying messages after stop_event is set, defaults to 30. """ request_tries = {} - client.reinit() - stop_at = None + while not stop_event.is_set(): + try: + client.reinit() + except Exception as e: + log.warn('Async producer failed to connect to brokers; backoff for %s(ms) before retrying', retry_options.backoff_ms) + time.sleep(float(retry_options.backoff_ms) / 1000) + else: + break + + stop_at = None while not (stop_event.is_set() and queue.empty() and not request_tries): # Handle stop_timeout From fb024355f07735d148c035dfd51b279d1b8e59df Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 17:15:49 -0800 Subject: [PATCH 237/250] Cleanup new producer tests... --- test/test_producer.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/test/test_producer.py b/test/test_producer.py index 3c026e8..31282bf 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -111,19 +111,19 @@ class TestKafkaProducer(unittest.TestCase): with self.assertRaises(FailedPayloadsError): producer.send_messages('foobar', b'test message') - def test_cleanup_stop_is_called_on_not_stopped_object(self): + def test_cleanup_is_not_called_on_stopped_producer(self): producer = Producer(MagicMock(), async=True) producer.stopped = True - with patch('kafka.producer.base.Producer.stop') as base_stop: + with patch.object(producer, 'stop') as mocked_stop: producer._cleanup_func(producer) - self.assertEqual(base_stop.call_count, 0) + self.assertEqual(mocked_stop.call_count, 0) - def test_cleanup_stop_is_not_called_on_stopped_object(self): + def test_cleanup_is_called_on_running_producer(self): producer = Producer(MagicMock(), async=True) producer.stopped = False - with patch('kafka.producer.base.Producer.stop') as base_stop: + with patch.object(producer, 'stop') as mocked_stop: producer._cleanup_func(producer) - self.assertEqual(base_stop.call_count, 1) + self.assertEqual(mocked_stop.call_count, 1) class TestKafkaProducerSendUpstream(unittest.TestCase): From 0ab97a99088c2871c3ab30dbc0dbf13f5f414433 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 18:02:22 -0800 Subject: [PATCH 238/250] Fix requestId handling in send_broker_aware_request --- kafka/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 810fa46..b3dcf51 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -236,13 +236,13 @@ class KafkaClient(object): responses[topic_partition] = None continue else: - connections_by_socket[conn.get_connected_socket()] = (conn, broker) + connections_by_socket[conn.get_connected_socket()] = (conn, broker, requestId) conn = None while connections_by_socket: sockets = connections_by_socket.keys() rlist, _, _ = select.select(sockets, [], [], None) - conn, broker = connections_by_socket.pop(rlist[0]) + conn, broker, requestId = connections_by_socket.pop(rlist[0]) try: response = conn.recv(requestId) except ConnectionError as e: From b687b4c5d4788d64efe9b7bcfb776e57d6fbcc8e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 5 Dec 2015 03:36:12 -0800 Subject: [PATCH 239/250] Use producer.stop() to flush messages in async producer test --- test/test_producer_integration.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index 46b6851..c99ed63 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -204,13 +204,11 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): resp = producer.send_messages(self.topic, self.msg("one")) self.assertEqual(len(resp), 0) - # wait for the server to report a new highwatermark - while self.current_offset(self.topic, partition) == start_offset: - time.sleep(0.1) + # flush messages + producer.stop() self.assert_fetch_offset(partition, start_offset, [ self.msg("one") ]) - producer.stop() @kafka_versions("all") def test_batched_simple_producer__triggers_by_message(self): From 2916bb865085ae2b883bc74dc4988b6b7723917d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 5 Dec 2015 03:59:09 -0800 Subject: [PATCH 240/250] Revert broken send_produce_request try/except from PR 467 (resps=None) --- kafka/client.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index b3dcf51..9018bb4 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -607,11 +607,7 @@ class KafkaClient(object): else: decoder = KafkaProtocol.decode_produce_response - try: - resps = self._send_broker_aware_request(payloads, encoder, decoder) - except Exception: - if fail_on_error: - raise + resps = self._send_broker_aware_request(payloads, encoder, decoder) return [resp if not callback else callback(resp) for resp in resps if resp is not None and From 5682ff63534bb2579a6835d40afcad170f6bdd7c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 5 Dec 2015 04:05:48 -0800 Subject: [PATCH 241/250] Producer.stop() now blocks until async thread completes (drop confusing timeout arg) --- kafka/producer/base.py | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index f2c7cfe..1ba4f5b 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -415,17 +415,22 @@ class Producer(object): raise return resp - def stop(self, timeout=1): + def stop(self): """ - Stop the producer. Optionally wait for the specified timeout before - forcefully cleaning up. + Stop the producer (async mode). Blocks until async thread completes. """ + if not self.async: + log.warning("producer.stop() called, but producer is not async") + return + + if self.stopped: + log.warning("producer.stop() called, but producer is already stopped") + return + if self.async: self.queue.put((STOP_ASYNC_PRODUCER, None, None)) - self.thread.join(timeout) - - if self.thread.is_alive(): - self.thread_stop_event.set() + self.thread_stop_event.set() + self.thread.join() if hasattr(self, '_cleanup_func'): # Remove cleanup handler now that we've stopped From 31ee4ed2af6da77bf4ad84589e7cc7d6272b40be Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 12:18:22 -0800 Subject: [PATCH 242/250] Log deprecation warning for timeout argument in Producer.stop() --- kafka/producer/base.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 1ba4f5b..39b1f84 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -415,16 +415,20 @@ class Producer(object): raise return resp - def stop(self): + def stop(self, timeout=None): """ Stop the producer (async mode). Blocks until async thread completes. """ + if timeout is not None: + log.warning('timeout argument to stop() is deprecated - ' + 'it will be removed in future release') + if not self.async: - log.warning("producer.stop() called, but producer is not async") + log.warning('producer.stop() called, but producer is not async') return if self.stopped: - log.warning("producer.stop() called, but producer is already stopped") + log.warning('producer.stop() called, but producer is already stopped') return if self.async: From c3b40b75696e4e40d51994db74b626feda6e19d1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 12:19:09 -0800 Subject: [PATCH 243/250] Update Changelog in preparation for 0.9.5 release --- CHANGES.md | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index c94cbd5..0cec2f7 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,40 @@ +# 0.9.5 (Unreleased) + +Consumers +* Initial support for consumer coordinator [offsets only] (toddpalino PR 420) +* Allow blocking until some messages are received in SimpleConsumer (saaros PR 457) +* Support subclass config changes in KafkaConsumer (zackdever PR 446) +* Support retry semantics in MultiProcessConsumer (barricadeio PR 456) +* Support partition_info in MultiProcessConsumer (scrapinghub PR 418) +* Enable seek() to an absolute offset in SimpleConsumer (haosdent PR 412) + +Producers +* Cath client.reinit() exceptions in async producer (dpkp) +* Producer.stop() now blocks until async thread completes (dpkp PR 485) +* Catch errors during load_metadata_for_topics in async producer (bschopman PR 467) +* Add compression-level support for codecs that support it (trbs PR 454) +* Fix translation of Java murmur2 code, fix byte encoding for Python 3 (chrischamberlin PR 439) +* Only call stop() on not-stopped producer objects (docker-hub PR 435) +* Allow null payload for deletion feature (scrapinghub PR 409) + +Clients +* Use non-blocking io for broker aware requests (ecanzonieri PR 473) +* Use debug logging level for metadata request (ecanzonieri PR 415) +* Catch KafkaUnavailableError in _send_broker_aware_request (mutability PR 436) +* Lower logging level on replica not available and commit (ecanzonieri PR 415) + +Documentation +* Update docs and links wrt maintainer change (mumrah -> dpkp) + +Internals +* Add py35 to tox testing +* Update travis config to use container infrastructure +* Add 0.8.2.2 and 0.9.0.0 resources for integration tests; update default official releases +* new pylint disables for pylint 1.5.1 (zackdever PR 481) +* Fix python3 / python2 comments re queue/Queue (dpkp) +* Add Murmur2Partitioner to kafka __all__ imports (dpkp Issue 471) +* Include LICENSE in PyPI sdist (koobs PR 441) + # 0.9.4 (June 11, 2015) Consumers From 74f04bc07e363bbb7b1a9ed14ed52d4eb12d5031 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 16:56:00 -0800 Subject: [PATCH 244/250] Add KafkaConsumer.close() PR to CHANGES --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 0cec2f7..6683fc4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -7,6 +7,7 @@ Consumers * Support retry semantics in MultiProcessConsumer (barricadeio PR 456) * Support partition_info in MultiProcessConsumer (scrapinghub PR 418) * Enable seek() to an absolute offset in SimpleConsumer (haosdent PR 412) +* Add KafkaConsumer.close() (ucarion PR 426) Producers * Cath client.reinit() exceptions in async producer (dpkp) From 51edb421c740838739c8ced5713cbeb037d1ea5c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Dec 2015 11:29:45 -0800 Subject: [PATCH 245/250] Update README for new release --- README.rst | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/README.rst b/README.rst index c05a240..1151a0f 100644 --- a/README.rst +++ b/README.rst @@ -19,35 +19,41 @@ high-level consumer and producer classes. Request batching is supported by the protocol as well as broker-aware request routing. Gzip and Snappy compression is also supported for message sets. -http://kafka.apache.org/ +Coordinated Consumer Groups are currently not supported (see Issue #38). On Freenode IRC at #kafka-python, as well as #apache-kafka For general discussion of kafka-client design and implementation (not python specific), see https://groups.google.com/forum/#!forum/kafka-clients +For information about Apache Kafka generally, see https://kafka.apache.org/ + License ---------- -Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE` +Apache License, v2.0. See `LICENSE` +Copyright 2015, David Arthur, Dana Powers, and Contributors (See `CONTRIBUTORS`) Status ---------- -The current stable version of this package is `0.9.4`_ and is compatible with: +The current stable version of this package is `0.9.5`_ and is compatible with: Kafka broker versions -- 0.8.2.1 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] +- 0.9.0.0 +- 0.8.2.2 +- 0.8.2.1 - 0.8.1.1 - 0.8.1 - 0.8.0 Python versions -- 2.6 (tested on 2.6.9) -- 2.7 (tested on 2.7.9) -- 3.3 (tested on 3.3.5) +- 3.5 (tested on 3.5.0) - 3.4 (tested on 3.4.2) +- 3.3 (tested on 3.3.5) +- 2.7 (tested on 2.7.9) +- 2.6 (tested on 2.6.9) - pypy (tested on pypy 2.5.0 / python 2.7.8) .. _Full documentation available on ReadTheDocs: http://kafka-python.readthedocs.org/en/latest/ -.. _0.9.4: https://github.com/dpkp/kafka-python/releases/tag/v0.9.4 +.. _0.9.5: https://github.com/dpkp/kafka-python/releases/tag/v0.9.5 From ca7c96c211b4d3405d60f1aa07a38a03926dbd82 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 16:44:27 -0800 Subject: [PATCH 246/250] Update README links --- README.rst | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/README.rst b/README.rst index 1151a0f..cc72a8b 100644 --- a/README.rst +++ b/README.rst @@ -12,14 +12,14 @@ Kafka Python client :target: http://kafka-python.readthedocs.org/en/latest/ :alt: Full documentation available on ReadTheDocs -`Full documentation available on ReadTheDocs`_ +[Full documentation available on ReadTheDocs](https://kafka-python.readthedocs.org/en/latest/) This module provides low-level protocol support for Apache Kafka as well as high-level consumer and producer classes. Request batching is supported by the protocol as well as broker-aware request routing. Gzip and Snappy compression is also supported for message sets. -Coordinated Consumer Groups are currently not supported (see Issue #38). +Coordinated Consumer Group support is under development - see Issue #38. On Freenode IRC at #kafka-python, as well as #apache-kafka @@ -30,12 +30,15 @@ For information about Apache Kafka generally, see https://kafka.apache.org/ License ---------- -Apache License, v2.0. See `LICENSE` -Copyright 2015, David Arthur, Dana Powers, and Contributors (See `CONTRIBUTORS`) +Apache License, v2.0. See [LICENSE](https://github.com/dpkp/kafka-python/blob/master/LICENSE) +Copyright 2015, David Arthur, Dana Powers, and Contributors +(See [AUTHORS](https://github.com/dpkp/kafka-python/blob/master/AUTHORS.md)) Status ---------- -The current stable version of this package is `0.9.5`_ and is compatible with: +The current stable version of this package is +[0.9.5](https://github.com/dpkp/kafka-python/releases/tag/v0.9.5) +and is compatible with: Kafka broker versions @@ -55,5 +58,3 @@ Python versions - 2.6 (tested on 2.6.9) - pypy (tested on pypy 2.5.0 / python 2.7.8) -.. _Full documentation available on ReadTheDocs: http://kafka-python.readthedocs.org/en/latest/ -.. _0.9.5: https://github.com/dpkp/kafka-python/releases/tag/v0.9.5 From 0f07871951c374d6371b53e942c62e0f708718ed Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 16:50:12 -0800 Subject: [PATCH 247/250] Include README.rst, CHANGES.md, and AUTHORS.md in manifest --- MANIFEST.in | 3 +++ 1 file changed, 3 insertions(+) diff --git a/MANIFEST.in b/MANIFEST.in index 1731afa..01e6a4d 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -1,2 +1,5 @@ recursive-include kafka *.py +include README.rst include LICENSE +include AUTHORS.md +include CHANGES.md From 205c9506c6795bfb7f3bb38ad585f9fa29a94f88 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 17:55:20 -0800 Subject: [PATCH 248/250] Update docs for new release --- docs/conf.py | 8 ++++---- docs/index.rst | 24 ++++++++++++++++-------- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/docs/conf.py b/docs/conf.py index 2979560..dc68fd4 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -49,7 +49,7 @@ master_doc = 'index' # General information about the project. project = u'kafka-python' -copyright = u'2015, David Arthur' +copyright = u'2015 - David Arthur, Dana Powers, and Contributors' # The version info for the project you're documenting, acts as replacement for # |version| and |release|, also used in various other places throughout the @@ -203,7 +203,7 @@ latex_elements = { # author, documentclass [howto, manual, or own class]). latex_documents = [ ('index', 'kafka-python.tex', u'kafka-python Documentation', - u'David Arthur', 'manual'), + u'Dana Powers', 'manual'), ] # The name of an image file (relative to this directory) to place at the top of @@ -233,7 +233,7 @@ latex_documents = [ # (source start file, name, description, authors, manual section). man_pages = [ ('index', 'kafka-python', u'kafka-python Documentation', - [u'David Arthur'], 1) + [u'Dana Powers'], 1) ] # If true, show URL addresses after external links. @@ -247,7 +247,7 @@ man_pages = [ # dir menu entry, description, category) texinfo_documents = [ ('index', 'kafka-python', u'kafka-python Documentation', - u'David Arthur', 'kafka-python', 'One line description of project.', + u'Dana Powers', 'kafka-python', 'One line description of project.', 'Miscellaneous'), ] diff --git a/docs/index.rst b/docs/index.rst index b7e7ccd..fa77a8e 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -6,37 +6,45 @@ high-level consumer and producer classes. Request batching is supported by the protocol as well as broker-aware request routing. Gzip and Snappy compression is also supported for message sets. -http://kafka.apache.org/ +Coordinated Consumer Group support is under development - see Issue #38. On Freenode IRC at #kafka-python, as well as #apache-kafka For general discussion of kafka-client design and implementation (not python specific), see https://groups.google.com/forum/m/#!forum/kafka-clients +For information about Apache Kafka generally, see https://kafka.apache.org/ + Status ------ -The current stable version of this package is `0.9.4 `_ and is compatible with: +The current stable version of this package is `0.9.5 `_ and is compatible with: Kafka broker versions -* 0.8.2.1 [offset management currently ZK only -- does not support ConsumerCoordinator offset management APIs] +* 0.9.0.0 +* 0.8.2.2 +* 0.8.2.1 * 0.8.1.1 * 0.8.1 * 0.8.0 Python versions -* 2.6 (tested on 2.6.9) -* 2.7 (tested on 2.7.9) -* 3.3 (tested on 3.3.5) +* 3.5 (tested on 3.5.0) * 3.4 (tested on 3.4.2) +* 3.3 (tested on 3.3.5) +* 2.7 (tested on 2.7.9) +* 2.6 (tested on 2.6.9) * pypy (tested on pypy 2.5.0 / python 2.7.8) License ------- -Copyright 2015, David Arthur under Apache License, v2.0. See `LICENSE `_. +Apache License, v2.0. See `LICENSE `_. + +Copyright 2015, David Arthur, Dana Powers, and Contributors +(See `AUTHORS `_). Contents @@ -45,9 +53,9 @@ Contents .. toctree:: :maxdepth: 2 + usage install tests - usage API reference Indices and tables From 89d9d6f56bd4c3bb7a09a79e1301cf58c70e43ab Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Dec 2015 00:01:35 -0800 Subject: [PATCH 249/250] Update README links (restructured not markdown) --- README.rst | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/README.rst b/README.rst index cc72a8b..99a3889 100644 --- a/README.rst +++ b/README.rst @@ -12,8 +12,6 @@ Kafka Python client :target: http://kafka-python.readthedocs.org/en/latest/ :alt: Full documentation available on ReadTheDocs -[Full documentation available on ReadTheDocs](https://kafka-python.readthedocs.org/en/latest/) - This module provides low-level protocol support for Apache Kafka as well as high-level consumer and producer classes. Request batching is supported by the protocol as well as broker-aware request routing. Gzip and Snappy compression @@ -21,6 +19,8 @@ is also supported for message sets. Coordinated Consumer Group support is under development - see Issue #38. +Full documentation available on `Read the Docs `_ + On Freenode IRC at #kafka-python, as well as #apache-kafka For general discussion of kafka-client design and implementation (not python specific), @@ -30,14 +30,14 @@ For information about Apache Kafka generally, see https://kafka.apache.org/ License ---------- -Apache License, v2.0. See [LICENSE](https://github.com/dpkp/kafka-python/blob/master/LICENSE) +Apache License, v2.0. See `LICENSE `_ Copyright 2015, David Arthur, Dana Powers, and Contributors -(See [AUTHORS](https://github.com/dpkp/kafka-python/blob/master/AUTHORS.md)) +(See `AUTHORS `_ Status ---------- The current stable version of this package is -[0.9.5](https://github.com/dpkp/kafka-python/releases/tag/v0.9.5) +`0.9.5 `_ and is compatible with: Kafka broker versions @@ -57,4 +57,3 @@ Python versions - 2.7 (tested on 2.7.9) - 2.6 (tested on 2.6.9) - pypy (tested on pypy 2.5.0 / python 2.7.8) - From 99d4a3a8b1dbae514b1c6d367908010b65fc8d0c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Dec 2015 15:32:59 -0800 Subject: [PATCH 250/250] Release 0.9.5 --- CHANGES.md | 2 +- kafka/version.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 6683fc4..edcee01 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,4 +1,4 @@ -# 0.9.5 (Unreleased) +# 0.9.5 (Dec 6, 2015) Consumers * Initial support for consumer coordinator [offsets only] (toddpalino PR 420) diff --git a/kafka/version.py b/kafka/version.py index cd64b48..9272695 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '0.9.5-dev' +__version__ = '0.9.5'