From 7c3cfbcfc3e0266dd5e99183ce4ad13d7d9f547f Mon Sep 17 00:00:00 2001 From: Jim Lim Date: Wed, 2 Oct 2013 11:55:09 -0700 Subject: [PATCH 001/109] update README.md to link to travis-ci's build page --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 01a241ceb..d688ea8d8 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,6 @@ # Kafka Python client -![travis](https://travis-ci.org/mumrah/kafka-python.png) +[![Build Status](https://travis-ci.org/mumrah/kafka-python.png)](https://travis-ci.org/mumrah/kafka-python) 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 3dbf3b60c3f5ce8096a8be43463286ba661fa734 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 2 Oct 2013 16:14:02 -0400 Subject: [PATCH 002/109] Adding a POWERED-BY.md file to people to give us props --- POWERED-BY.md | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 POWERED-BY.md diff --git a/POWERED-BY.md b/POWERED-BY.md new file mode 100644 index 000000000..bef42720d --- /dev/null +++ b/POWERED-BY.md @@ -0,0 +1,6 @@ +# Project/People/Companies using kafka-python + +If you're using this library and care to give us a shout out, please fork the project, +add yourself here, and submit a pull request. Thanks! + +* @mumrah, adding myself as an example From 8cc4666f0746a44626990a0f87898697ebc97634 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 2 Oct 2013 16:15:06 -0400 Subject: [PATCH 003/109] Update POWERED-BY.md Apparently GitHub flavored markdown doesn't fully work in md files? (@mumrah didn't auto-link) --- POWERED-BY.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/POWERED-BY.md b/POWERED-BY.md index bef42720d..f2e323c3e 100644 --- a/POWERED-BY.md +++ b/POWERED-BY.md @@ -3,4 +3,4 @@ If you're using this library and care to give us a shout out, please fork the project, add yourself here, and submit a pull request. Thanks! -* @mumrah, adding myself as an example +* [@mumrah](https://github.com/mumrah), adding myself as an example From 1d4bc3f6312aa51447d26b5de1fe4b4669584096 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 2 Oct 2013 16:32:33 -0400 Subject: [PATCH 004/109] Updating authors, thanks to all the contributors! Also testing out this -> [skip ci] --- AUTHORS.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/AUTHORS.md b/AUTHORS.md index f8b3c6c7a..dfcf3e724 100644 --- a/AUTHORS.md +++ b/AUTHORS.md @@ -5,6 +5,8 @@ Ordered by contribution * David Arthur, [@mumrah](https://github.com/mumrah) * Mahendra M, [@mahendra](https://github.com/mahendra) * Ivan Pouzyrevsky, [@sandello](https://github.com/sandello) +* Jim Lim, [@jimjh](https://github.com/jimjh) +* StevenLeRoux, [@StevenLeRoux](https://github.com/StevenLeRoux) * [@anentropic](https://github.com/anentropic) * Ben Frederickson, [@benfred](https://github.com/benfred) From c304e3d4b080ce720e32abb820ab9120b8669d23 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Tue, 1 Oct 2013 15:07:47 -0400 Subject: [PATCH 005/109] Cherry-pick mrtheb/kafka-python 8b9c7e51 Sync tests and fixtures with kafka 0.8.0-beta1 tag Conflicts: README.md kafka-src --- README.md | 3 ++- kafka-src | 2 +- test/fixtures.py | 17 ++++++----------- test/resources/kafka.properties | 8 ++++---- test/test_integration.py | 32 ++++++++++++++++++++------------ 5 files changed, 33 insertions(+), 29 deletions(-) diff --git a/README.md b/README.md index d688ea8d8..c9f782d12 100644 --- a/README.md +++ b/README.md @@ -18,7 +18,7 @@ Copyright 2013, David Arthur under Apache License, v2.0. See `LICENSE` # Status I'm following the version numbers of Kafka, plus one number to indicate the -version of this project. The current version is 0.8.1-1. This version is under +version of this project. The current version is 0.8.0-1. This version is under development, APIs are subject to change. # Usage @@ -196,6 +196,7 @@ git submodule update cd kafka-src ./sbt update ./sbt package +./sbt assembly-package-dependency ``` And then run the tests. This will actually start up real local Zookeeper diff --git a/kafka-src b/kafka-src index 9ff4e8eb1..7b43f0132 160000 --- a/kafka-src +++ b/kafka-src @@ -1 +1 @@ -Subproject commit 9ff4e8eb10e0ddd86f257e99d55971a132426605 +Subproject commit 7b43f0132ce93a231b21e943d665ec4227f67b6b diff --git a/test/fixtures.py b/test/fixtures.py index abaaa5cd3..00c1afd00 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -17,6 +17,7 @@ PROJECT_ROOT = os.path.abspath(os.path.join(os.path.dirname(__file__), "..")) KAFKA_ROOT = os.path.join(PROJECT_ROOT, "kafka-src") IVY_ROOT = os.path.expanduser("~/.ivy2/cache") +SCALA_VERSION = '2.8.0' if "PROJECT_ROOT" in os.environ: PROJECT_ROOT = os.environ["PROJECT_ROOT"] @@ -24,6 +25,8 @@ KAFKA_ROOT = os.environ["KAFKA_ROOT"] if "IVY_ROOT" in os.environ: IVY_ROOT = os.environ["IVY_ROOT"] +if "SCALA_VERSION" in os.environ: + SCALA_VERSION = os.environ["SCALA_VERSION"] def test_resource(file): @@ -33,16 +36,8 @@ def test_resource(file): def test_classpath(): # ./kafka-src/bin/kafka-run-class.sh is the authority. jars = ["."] - jars.append(IVY_ROOT + "/org.xerial.snappy/snappy-java/bundles/snappy-java-1.0.4.1.jar") - jars.append(IVY_ROOT + "/org.scala-lang/scala-library/jars/scala-library-2.8.0.jar") - jars.append(IVY_ROOT + "/org.scala-lang/scala-compiler/jars/scala-compiler-2.8.0.jar") - jars.append(IVY_ROOT + "/log4j/log4j/jars/log4j-1.2.15.jar") - jars.append(IVY_ROOT + "/org.slf4j/slf4j-api/jars/slf4j-api-1.6.4.jar") - jars.append(IVY_ROOT + "/org.apache.zookeeper/zookeeper/jars/zookeeper-3.3.4.jar") - jars.append(IVY_ROOT + "/net.sf.jopt-simple/jopt-simple/jars/jopt-simple-3.2.jar") - jars.extend(glob.glob(KAFKA_ROOT + "/core/target/scala-2.8.0/*.jar")) - jars.extend(glob.glob(KAFKA_ROOT + "/core/lib/*.jar")) - jars.extend(glob.glob(KAFKA_ROOT + "/perf/target/scala-2.8.0/kafka*.jar")) + # assume all dependencies have been packaged into one jar with sbt-assembly's task "assembly-package-dependency" + jars.extend(glob.glob(KAFKA_ROOT + "/core/target/scala-%s/*.jar" % SCALA_VERSION)) jars = filter(os.path.exists, map(os.path.abspath, jars)) return ":".join(jars) @@ -314,7 +309,7 @@ def open(self): print("*** Starting Kafka...") self.child.start() - self.child.wait_for(r"\[Kafka Server \d+\], started") + self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id) print("*** Done!") def close(self): diff --git a/test/resources/kafka.properties b/test/resources/kafka.properties index 2c8416f29..d42c0971e 100644 --- a/test/resources/kafka.properties +++ b/test/resources/kafka.properties @@ -4,9 +4,9 @@ # 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. @@ -47,8 +47,8 @@ log.cleanup.interval.mins=1 ############################# Zookeeper ############################# -zk.connect={zk_host}:{zk_port}/{zk_chroot} -zk.connection.timeout.ms=1000000 +zookeeper.connect={zk_host}:{zk_port}/{zk_chroot} +zookeeper.connection.timeout.ms=1000000 kafka.metrics.polling.interval.secs=5 kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/test/test_integration.py b/test/test_integration.py index bf1acc8cd..d8ead594a 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -242,6 +242,7 @@ def test_produce_consume_two_partitions(self): # Offset Tests # #################### + @unittest.skip('commmit offset not supported in this version') def test_commit_fetch_offsets(self): req = OffsetCommitRequest("test_commit_fetch_offsets", 0, 42, "metadata") (resp,) = self.client.send_offset_commit_request("group", [req]) @@ -401,8 +402,9 @@ def test_acks_local_write(self): producer.stop() def test_acks_cluster_commit(self): - producer = SimpleProducer(self.client, "test_acks_cluster_commit", - req_acks=SimpleProducer.ACK_AFTER_CLUSTER_COMMIT) + producer = SimpleProducer( + self.client, "test_acks_cluster_commit", + req_acks=SimpleProducer.ACK_AFTER_CLUSTER_COMMIT) resp = producer.send_messages("one") self.assertEquals(len(resp), 1) @@ -548,11 +550,11 @@ def test_batched_simple_producer(self): class TestConsumer(unittest.TestCase): @classmethod - def setUpClass(cls): # noqa + def setUpClass(cls): cls.zk = ZookeeperFixture.instance() cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) - cls.client = KafkaClient(cls.server2.host, cls.server2.port) + cls.client = KafkaClient(cls.server2.host, cls.server2.port, bufsize=8192) @classmethod def tearDownClass(cls): # noqa @@ -581,7 +583,7 @@ def test_simple_consumer(self): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer") + consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer", auto_commit=False) all_messages = [] for message in consumer: all_messages.append(message) @@ -604,6 +606,11 @@ def test_simple_consumer(self): self.assertEquals(len(all_messages), 13) + consumer.stop() + + def test_simple_consumer_blocking(self): + consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer_blocking", auto_commit=False) + # Blocking API start = datetime.now() messages = consumer.get_messages(block=True, timeout=5) @@ -612,13 +619,13 @@ def test_simple_consumer(self): self.assertEqual(len(messages), 0) # Send 10 messages - produce = ProduceRequest("test_simple_consumer", 0, messages=[ + produce = ProduceRequest("test_simple_consumer_blocking", 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) for resp in self.client.send_produce_request([produce]): self.assertEquals(resp.error, 0) - self.assertEquals(resp.offset, 100) + self.assertEquals(resp.offset, 0) # Fetch 5 messages messages = consumer.get_messages(count=5, block=True, timeout=5) @@ -650,7 +657,7 @@ def test_simple_consumer_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = SimpleConsumer(self.client, "group1", "test_simple_pending") + consumer = SimpleConsumer(self.client, "group1", "test_simple_pending", auto_commit=False) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -676,7 +683,7 @@ def test_multi_process_consumer(self): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = MultiProcessConsumer(self.client, "grp1", "test_mpconsumer") + consumer = MultiProcessConsumer(self.client, "grp1", "test_mpconsumer", auto_commit=False) all_messages = [] for message in consumer: all_messages.append(message) @@ -732,7 +739,7 @@ def test_multi_proc_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = MultiProcessConsumer(self.client, "group1", "test_mppending") + consumer = MultiProcessConsumer(self.client, "group1", "test_mppending", auto_commit=False) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -749,7 +756,7 @@ def test_large_messages(self): self.assertEquals(resp.offset, 0) # Produce 10 messages that are too large (bigger than default fetch size) - messages2=[create_message(random_string(5000)) for i in range(10)] + messages2 = [create_message(random_string(5000)) for i in range(10)] produce2 = ProduceRequest("test_large_messages", 0, messages2) for resp in self.client.send_produce_request([produce2]): @@ -757,12 +764,13 @@ def test_large_messages(self): self.assertEquals(resp.offset, 10) # Consumer should still get all of them - consumer = SimpleConsumer(self.client, "group1", "test_large_messages") + consumer = SimpleConsumer(self.client, "group1", "test_large_messages", auto_commit=False) all_messages = messages1 + messages2 for i, message in enumerate(consumer): self.assertEquals(all_messages[i], message.message) self.assertEquals(i, 19) + def random_string(l): s = "".join(random.choice(string.letters) for i in xrange(l)) return s From deb584c90802d382ca376331a8bf31bfb12d94dc Mon Sep 17 00:00:00 2001 From: mrtheb Date: Tue, 1 Oct 2013 15:15:36 -0400 Subject: [PATCH 006/109] Cherry-pick mrtheb/kafka-python 2b016b69 Set FetchRequest MaxBytes value to bufsize instead of fetchsize (=MinBytes) --- kafka/consumer.py | 5 ++++- kafka/util.py | 7 ++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index c338337ff..3efffdf2f 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -359,7 +359,10 @@ def __iter_partition__(self, partition, offset): fetch_size = self.fetch_min_bytes while True: - req = FetchRequest(self.topic, partition, offset, fetch_size) + # use MaxBytes = client's bufsize since we're only + # fetching one topic + partition + req = FetchRequest( + self.topic, partition, offset, self.client.bufsize) (resp,) = self.client.send_fetch_request([req], max_wait_time=self.fetch_max_wait_time, diff --git a/kafka/util.py b/kafka/util.py index 259e2854e..598ba848c 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -38,7 +38,8 @@ def read_short_string(data, cur): def read_int_string(data, cur): if len(data) < cur + 4: - raise BufferUnderflowError("Not enough data left") + raise BufferUnderflowError( + "Not enough data left to read string len (%d < %d)" % (len(data), cur + 4)) (strLen,) = struct.unpack('>i', data[cur:cur + 4]) if strLen == -1: @@ -46,7 +47,8 @@ def read_int_string(data, cur): cur += 4 if len(data) < cur + strLen: - raise BufferUnderflowError("Not enough data left") + raise BufferUnderflowError( + "Not enough data left to read string (%d < %d)" % (len(data), cur + strLen)) out = data[cur:cur + strLen] return (out, cur + strLen) @@ -68,7 +70,6 @@ def group_by_topic_and_partition(tuples): return out - class ReentrantTimer(object): """ A timer that can be restarted, unlike threading.Timer From 9c868d47cdd51a978fd1847b79f90b1ac8d7688f Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 12:23:22 -0400 Subject: [PATCH 007/109] Add assembly-package-dependency to Travis config --- .travis.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index c1b71d70f..0b317cabf 100644 --- a/.travis.yml +++ b/.travis.yml @@ -7,8 +7,7 @@ python: before_install: - git submodule update --init --recursive - cd kafka-src - - ./sbt update - - ./sbt package + - ./sbt clean update package assembly-package-dependency - cd - install: From e898500ac6ce1ee1ef7d44b90b89c28b8a2c3c5d Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 12:41:25 -0400 Subject: [PATCH 008/109] Travis-CI workaround for multiprocessing.Queue OSError See https://github.com/travis-ci/travis-cookbooks/issues/155 for background --- .travis.yml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 0b317cabf..77348c0da 100644 --- a/.travis.yml +++ b/.travis.yml @@ -11,7 +11,10 @@ before_install: - cd - install: - pip install . + - 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 script: - python -m test.test_unit From e39e05f8a50b7528a22fed99dc67d561cbd79c41 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 12:59:21 -0400 Subject: [PATCH 009/109] Disable unit tests for 2.6, close #57 --- README.md | 2 +- test/test_unit.py | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index c9f782d12..fe6693de9 100644 --- a/README.md +++ b/README.md @@ -7,7 +7,7 @@ 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. -Compatible with Apache Kafka 0.8.1 +Compatible with Python 2.6+ and Apache Kafka 0.8.1 http://kafka.apache.org/ diff --git a/test/test_unit.py b/test/test_unit.py index c796c949f..c92f2caf2 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -1,6 +1,7 @@ import os import random import struct +import sys import unittest from kafka.client import KafkaClient, ProduceRequest, FetchRequest @@ -13,6 +14,10 @@ ITERATIONS = 1000 STRLEN = 100 +# TODO remove this and fix unit tests +if sys.version_info < (2,7): + print("Skipping unit tests for Python <2.7") + exit(0) def random_string(): return os.urandom(random.randint(1, STRLEN)) From f70ef1d3d6712f16b5a13f186bf815ea567b8437 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 13:11:33 -0400 Subject: [PATCH 010/109] Revert "Disable unit tests for 2.6, close #57" This reverts commit e39e05f8a50b7528a22fed99dc67d561cbd79c41. --- README.md | 2 +- test/test_unit.py | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/README.md b/README.md index fe6693de9..c9f782d12 100644 --- a/README.md +++ b/README.md @@ -7,7 +7,7 @@ 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. -Compatible with Python 2.6+ and Apache Kafka 0.8.1 +Compatible with Apache Kafka 0.8.1 http://kafka.apache.org/ diff --git a/test/test_unit.py b/test/test_unit.py index c92f2caf2..c796c949f 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -1,7 +1,6 @@ import os import random import struct -import sys import unittest from kafka.client import KafkaClient, ProduceRequest, FetchRequest @@ -14,10 +13,6 @@ ITERATIONS = 1000 STRLEN = 100 -# TODO remove this and fix unit tests -if sys.version_info < (2,7): - print("Skipping unit tests for Python <2.7") - exit(0) def random_string(): return os.urandom(random.randint(1, STRLEN)) From 0866be7844570a67208f9324e7ce1afa833ddd98 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 13:12:04 -0400 Subject: [PATCH 011/109] Remove Python 2.6 from CI builds, ref #57 --- .travis.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 77348c0da..f7d0ed4eb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,7 +1,6 @@ language: python python: - - 2.6 - 2.7 before_install: From f833050f0af1f9aba0df84770f824714cec61728 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 13:43:48 -0400 Subject: [PATCH 012/109] Updating year in LICENSE [skip ci] --- LICENSE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/LICENSE b/LICENSE index a85fd9436..efe9d9682 100644 --- a/LICENSE +++ b/LICENSE @@ -186,7 +186,7 @@ same "printed page" as the copyright notice for easier identification within third-party archives. - Copyright 2012 David Arthur + Copyright 2013 David Arthur Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. From e87cddef1b9a8360d94ffae1e7fd3c82790cd2a6 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 17:56:29 +0400 Subject: [PATCH 013/109] style: fix whitespaces --- kafka/client.py | 2 +- kafka/common.py | 4 ++++ kafka/conn.py | 2 ++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index be61e91ec..ae489e1f9 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -22,7 +22,7 @@ class KafkaClient(object): def __init__(self, host, port, bufsize=4096, client_id=CLIENT_ID): # We need one connection to bootstrap - self.bufsize = bufsize + self.bufsize = bufsize self.client_id = client_id self.conns = { # (host, port) -> KafkaConnection (host, port): KafkaConnection(host, port, bufsize) diff --git a/kafka/common.py b/kafka/common.py index 0a1d3143b..8f3154cc7 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -69,14 +69,18 @@ class ErrorMapping(object): # Exceptions # ################# + class BufferUnderflowError(Exception): pass + class ChecksumError(Exception): pass + class ConsumerFetchSizeTooSmall(Exception): pass + class ConsumerNoMoreData(Exception): pass diff --git a/kafka/conn.py b/kafka/conn.py index 7103253d1..00c4ce2fd 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -3,8 +3,10 @@ import struct from threading import local + log = logging.getLogger("kafka") + class KafkaConnection(local): """ A socket connection to a single Kafka broker From e392e0c2010562c10eca210056edef057e344e30 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:00:57 +0400 Subject: [PATCH 014/109] style: use triple quotes for docstrings --- kafka/client.py | 8 ++++++-- kafka/conn.py | 8 ++++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index ae489e1f9..5fb38429b 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -37,7 +37,9 @@ def __init__(self, host, port, bufsize=4096, client_id=CLIENT_ID): ################## def _get_conn_for_broker(self, broker): - "Get or create a connection to a broker" + """ + Get or create a connection to a broker + """ if (broker.host, broker.port) not in self.conns: self.conns[(broker.host, broker.port)] = \ KafkaConnection(broker.host, broker.port, self.bufsize) @@ -97,7 +99,9 @@ def _load_metadata_for_topics(self, *topics): self.topic_partitions[topic].append(partition) def _next_id(self): - "Generate a new correlation id" + """ + Generate a new correlation id + """ return KafkaClient.ID_GEN.next() def _send_broker_unaware_request(self, requestId, request): diff --git a/kafka/conn.py b/kafka/conn.py index 00c4ce2fd..04f97f233 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -80,13 +80,17 @@ def send(self, requestId, payload): raise RuntimeError("Kafka went away") def recv(self, requestId): - "Get a response from Kafka" + """ + Get a response from Kafka + """ log.debug("Reading response %d from Kafka" % requestId) self.data = self._consume_response() return self.data def close(self): - "Close this connection" + """ + Close this connection + """ self._sock.close() def reinit(self): From 8b05e6240bfce76868d2126aad75b78af1a17c60 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:06:28 +0400 Subject: [PATCH 015/109] style: fix camelCase variable names Conflicts: kafka/util.py --- kafka/client.py | 6 +++--- kafka/conn.py | 4 ++-- kafka/protocol.py | 26 +++++++++++++------------- kafka/util.py | 23 +++++++++++------------ 4 files changed, 29 insertions(+), 30 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 5fb38429b..2fdb271eb 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -61,11 +61,11 @@ def _load_metadata_for_topics(self, *topics): Discover brokers and metadata for a set of topics. This method will recurse in the event of a retry. """ - requestId = self._next_id() + request_id = self._next_id() request = KafkaProtocol.encode_metadata_request(self.client_id, - requestId, topics) + request_id, topics) - response = self._send_broker_unaware_request(requestId, request) + response = self._send_broker_unaware_request(request_id, request) if response is None: raise Exception("All servers failed to process request") diff --git a/kafka/conn.py b/kafka/conn.py index 04f97f233..c41c08d61 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -79,11 +79,11 @@ def send(self, requestId, payload): if sent != None: raise RuntimeError("Kafka went away") - def recv(self, requestId): + def recv(self, request_id): """ Get a response from Kafka """ - log.debug("Reading response %d from Kafka" % requestId) + log.debug("Reading response %d from Kafka" % request_id) self.data = self._consume_response() return self.data diff --git a/kafka/protocol.py b/kafka/protocol.py index c2b017ef6..d0ea646a4 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -361,11 +361,11 @@ def decode_metadata_response(cls, data): ====== data: bytes to decode """ - ((correlation_id, numBrokers), cur) = relative_unpack('>ii', data, 0) + ((correlation_id, numbrokers), cur) = relative_unpack('>ii', data, 0) # Broker info brokers = {} - for i in range(numBrokers): + for i in range(numbrokers): ((nodeId, ), cur) = relative_unpack('>i', data, cur) (host, cur) = read_short_string(data, cur) ((port,), cur) = relative_unpack('>i', data, cur) @@ -373,31 +373,31 @@ def decode_metadata_response(cls, data): # Topic info ((num_topics,), cur) = relative_unpack('>i', data, cur) - topicMetadata = {} + topic_metadata = {} for i in range(num_topics): - ((topicError,), cur) = relative_unpack('>h', data, cur) - (topicName, cur) = read_short_string(data, cur) + ((topic_error,), cur) = relative_unpack('>h', data, cur) + (topic_name, cur) = read_short_string(data, cur) ((num_partitions,), cur) = relative_unpack('>i', data, cur) - partitionMetadata = {} + partition_metadata = {} for j in range(num_partitions): - ((partitionErrorCode, partition, leader, numReplicas), cur) = \ + ((partition_error_code, partition, leader, numReplicas), cur) = \ relative_unpack('>hiii', data, cur) (replicas, cur) = relative_unpack('>%di' % numReplicas, data, cur) - ((numIsr,), cur) = relative_unpack('>i', data, cur) - (isr, cur) = relative_unpack('>%di' % numIsr, data, cur) + ((num_isr,), cur) = relative_unpack('>i', data, cur) + (isr, cur) = relative_unpack('>%di' % num_isr, data, cur) - partitionMetadata[partition] = \ - PartitionMetadata(topicName, partition, leader, + partition_metadata[partition] = \ + PartitionMetadata(topic_name, partition, leader, replicas, isr) - topicMetadata[topicName] = partitionMetadata + topic_metadata[topic_name] = partition_metadata - return (brokers, topicMetadata) + return (brokers, topic_metadata) @classmethod def encode_offset_commit_request(cls, client_id, correlation_id, diff --git a/kafka/util.py b/kafka/util.py index 598ba848c..e4a3db0b4 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -24,16 +24,16 @@ def read_short_string(data, cur): if len(data) < cur + 2: raise BufferUnderflowError("Not enough data left") - (strLen,) = struct.unpack('>h', data[cur:cur + 2]) - if strLen == -1: + (strlen,) = struct.unpack('>h', data[cur:cur + 2]) + if strlen == -1: return (None, cur + 2) cur += 2 - if len(data) < cur + strLen: + if len(data) < cur + strlen: raise BufferUnderflowError("Not enough data left") - out = data[cur:cur + strLen] - return (out, cur + strLen) + out = data[cur:cur + strlen] + return (out, cur + strlen) def read_int_string(data, cur): @@ -41,17 +41,16 @@ def read_int_string(data, cur): raise BufferUnderflowError( "Not enough data left to read string len (%d < %d)" % (len(data), cur + 4)) - (strLen,) = struct.unpack('>i', data[cur:cur + 4]) - if strLen == -1: + (strlen,) = struct.unpack('>i', data[cur:cur + 4]) + if strlen == -1: return (None, cur + 4) cur += 4 - if len(data) < cur + strLen: - raise BufferUnderflowError( - "Not enough data left to read string (%d < %d)" % (len(data), cur + strLen)) + if len(data) < cur + strlen: + raise BufferUnderflowError("Not enough data left") - out = data[cur:cur + strLen] - return (out, cur + strLen) + out = data[cur:cur + strlen] + return (out, cur + strlen) def relative_unpack(fmt, data, cur): From 3a56dfd624591e5690069db8ade55bc66c804c57 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:08:51 +0400 Subject: [PATCH 016/109] style: fix camelCase variable names once more --- kafka/conn.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index c41c08d61..764cbf6b2 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -53,12 +53,12 @@ def _consume_response_iter(self): raise Exception("Got no response from Kafka") (size,) = struct.unpack('>i', resp) - messageSize = size - 4 - log.debug("About to read %d bytes from Kafka", messageSize) + messagesize = size - 4 + log.debug("About to read %d bytes from Kafka", messagesize) # Read the remainder of the response total = 0 - while total < messageSize: + while total < messagesize: resp = self._sock.recv(self.bufsize) log.debug("Read %d bytes from Kafka", len(resp)) if resp == "": From a6b314d9630a1067c8b381976fe6234afa8a9948 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:09:47 +0400 Subject: [PATCH 017/109] style: remove extra brackets --- kafka/protocol.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/protocol.py b/kafka/protocol.py index d0ea646a4..421e19b17 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -397,7 +397,7 @@ def decode_metadata_response(cls, data): topic_metadata[topic_name] = partition_metadata - return (brokers, topic_metadata) + return brokers, topic_metadata @classmethod def encode_offset_commit_request(cls, client_id, correlation_id, From e82f04e3b8828169b375aa51699874e69e022888 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:11:07 +0400 Subject: [PATCH 018/109] style: remove extra brackets one more time --- kafka/util.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/kafka/util.py b/kafka/util.py index e4a3db0b4..96b37456e 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -26,14 +26,14 @@ def read_short_string(data, cur): (strlen,) = struct.unpack('>h', data[cur:cur + 2]) if strlen == -1: - return (None, cur + 2) + return None, cur + 2 cur += 2 if len(data) < cur + strlen: raise BufferUnderflowError("Not enough data left") out = data[cur:cur + strlen] - return (out, cur + strlen) + return out, cur + strlen def read_int_string(data, cur): @@ -43,14 +43,14 @@ def read_int_string(data, cur): (strlen,) = struct.unpack('>i', data[cur:cur + 4]) if strlen == -1: - return (None, cur + 4) + return None, cur + 4 cur += 4 if len(data) < cur + strlen: raise BufferUnderflowError("Not enough data left") out = data[cur:cur + strlen] - return (out, cur + strlen) + return out, cur + strlen def relative_unpack(fmt, data, cur): @@ -59,7 +59,7 @@ def relative_unpack(fmt, data, cur): raise BufferUnderflowError("Not enough data left") out = struct.unpack(fmt, data[cur:cur + size]) - return (out, cur + size) + return out, cur + size def group_by_topic_and_partition(tuples): From 3ba54c6e05a3f77af0c38eff03c6ba58066becc1 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:13:34 +0400 Subject: [PATCH 019/109] style: fix camelCase variable names again --- kafka/conn.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 764cbf6b2..7798a7534 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -72,9 +72,9 @@ def _consume_response_iter(self): # TODO multiplex socket communication to allow for multi-threaded clients - def send(self, requestId, payload): + def send(self, request_id, payload): "Send a request to Kafka" - log.debug("About to send %d bytes to Kafka, request %d" % (len(payload), requestId)) + log.debug("About to send %d bytes to Kafka, request %d" % (len(payload), request_id)) sent = self._sock.sendall(payload) if sent != None: raise RuntimeError("Kafka went away") From 4ccfa5ce76eadc9f95fb90a1741b667a7a48d2d1 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:19:54 +0400 Subject: [PATCH 020/109] don't forget to call superclass __init__ --- kafka/conn.py | 1 + kafka/partitioner.py | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index 7798a7534..29efbf1e1 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -17,6 +17,7 @@ class KafkaConnection(local): since the Kafka API includes a correlation id. """ def __init__(self, host, port, bufsize=4096): + super(KafkaConnection, self).__init__() self.host = host self.port = port self.bufsize = bufsize diff --git a/kafka/partitioner.py b/kafka/partitioner.py index 84db4d513..817e4168e 100644 --- a/kafka/partitioner.py +++ b/kafka/partitioner.py @@ -31,7 +31,8 @@ class RoundRobinPartitioner(Partitioner): in a round robin fashion """ def __init__(self, partitions): - self._set_partitions(partitions) + super(RoundRobinPartitioner, self).__init__(partitions) + self.iterpart = cycle(partitions) def _set_partitions(self, partitions): self.partitions = partitions From 7ac75c575c446029a706ce922e646f95805c776e Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:22:00 +0400 Subject: [PATCH 021/109] use NotImplementedError instead of NotImplementedError --- kafka/partitioner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/partitioner.py b/kafka/partitioner.py index 817e4168e..8190c34f9 100644 --- a/kafka/partitioner.py +++ b/kafka/partitioner.py @@ -22,7 +22,7 @@ def partition(self, key, partitions): may look like an overhead, but it will be useful (in future) when we handle cases like rebalancing """ - raise NotImplemented('partition function has to be implemented') + raise NotImplementedError('partition function has to be implemented') class RoundRobinPartitioner(Partitioner): From 11f22fb3d74b69f98324ac82b786d4dedefe2dc5 Mon Sep 17 00:00:00 2001 From: Vetoshkin Nikita Date: Thu, 3 Oct 2013 18:26:34 +0400 Subject: [PATCH 022/109] remove unused exception instance variables --- kafka/consumer.py | 2 +- kafka/producer.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 3efffdf2f..a97e8c00d 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -299,7 +299,7 @@ def get_messages(self, count=1, block=True, timeout=0.1): while count > 0: try: messages.append(next(iterator)) - except StopIteration as exp: + except StopIteration: break count -= 1 diff --git a/kafka/producer.py b/kafka/producer.py index 06e468dfc..5f23285ab 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -112,7 +112,7 @@ def _send_upstream(self, queue): try: self.client.send_produce_request(reqs, acks=self.req_acks, timeout=self.ack_timeout) - except Exception as exp: + except Exception: log.error("Error sending message", exc_info=sys.exc_info()) def send_messages(self, partition, *msg): From b0cacc948539d180e4a634a06a10232770deb187 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 3 Oct 2013 20:12:33 -0400 Subject: [PATCH 023/109] Update kafka-src to 0.8.0 RC1 resolves #58 --- kafka-src | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-src b/kafka-src index 7b43f0132..5bd33c151 160000 --- a/kafka-src +++ b/kafka-src @@ -1 +1 @@ -Subproject commit 7b43f0132ce93a231b21e943d665ec4227f67b6b +Subproject commit 5bd33c1517bb2e7734166dc3e787ac90a4ef8f86 From a03f0c86b8a504c0e3185cac1611131dba24f625 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Thu, 3 Oct 2013 22:52:04 -0400 Subject: [PATCH 024/109] flake8 pass (pep8 and pyflakes) --- kafka/client.py | 41 ++++++++++++++++++------------------ kafka/conn.py | 20 ++++++++++++------ kafka/consumer.py | 53 +++++++++++++++++++++++++++-------------------- kafka/protocol.py | 38 ++++++++++++++++----------------- kafka/queue.py | 19 ++++++++++------- kafka/util.py | 6 +++--- 6 files changed, 97 insertions(+), 80 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 2fdb271eb..a31ea66ae 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -1,14 +1,9 @@ -import base64 from collections import defaultdict from functools import partial -from itertools import count, cycle import logging -from operator import attrgetter -import struct import time -import zlib -from kafka.common import * +from kafka.common import count, ErrorMapping, TopicAndPartition from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol @@ -212,8 +207,10 @@ def send_produce_request(self, payloads=[], acks=1, timeout=1000, order of input payloads """ - encoder = partial(KafkaProtocol.encode_produce_request, - acks=acks, timeout=timeout) + encoder = partial( + KafkaProtocol.encode_produce_request, + acks=acks, + timeout=timeout) if acks == 0: decoder = None @@ -226,10 +223,10 @@ def send_produce_request(self, payloads=[], acks=1, timeout=1000, for resp in resps: # Check for errors if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("ProduceRequest for %s failed with " - "errorcode=%d" % ( - TopicAndPartition(resp.topic, resp.partition), - resp.error)) + raise Exception( + "ProduceRequest for %s failed with errorcode=%d" % + (TopicAndPartition(resp.topic, resp.partition), + resp.error)) # Run the callback if callback is not None: @@ -251,17 +248,18 @@ def send_fetch_request(self, payloads=[], fail_on_error=True, max_wait_time=max_wait_time, min_bytes=min_bytes) - resps = self._send_broker_aware_request(payloads, encoder, - KafkaProtocol.decode_fetch_response) + resps = self._send_broker_aware_request( + payloads, encoder, + KafkaProtocol.decode_fetch_response) out = [] for resp in resps: # Check for errors if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("FetchRequest for %s failed with " - "errorcode=%d" % ( - TopicAndPartition(resp.topic, resp.partition), - resp.error)) + raise Exception( + "FetchRequest for %s failed with errorcode=%d" % + (TopicAndPartition(resp.topic, resp.partition), + resp.error)) # Run the callback if callback is not None: @@ -272,9 +270,10 @@ def send_fetch_request(self, payloads=[], fail_on_error=True, def send_offset_request(self, payloads=[], fail_on_error=True, callback=None): - resps = self._send_broker_aware_request(payloads, - KafkaProtocol.encode_offset_request, - KafkaProtocol.decode_offset_response) + resps = self._send_broker_aware_request( + payloads, + KafkaProtocol.encode_offset_request, + KafkaProtocol.decode_offset_response) out = [] for resp in resps: diff --git a/kafka/conn.py b/kafka/conn.py index 29efbf1e1..e85fd110f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -3,6 +3,8 @@ import struct from threading import local +from kafka.common import BufferUnderflowError + log = logging.getLogger("kafka") @@ -12,7 +14,7 @@ class KafkaConnection(local): A socket connection to a single Kafka broker This class is _not_ thread safe. Each call to `send` must be followed - by a call to `recv` in order to get the correct response. Eventually, + by a call to `recv` in order to get the correct response. Eventually, we can do something in here to facilitate multiplexed requests/responses since the Kafka API includes a correlation id. """ @@ -43,7 +45,7 @@ def _consume_response(self): def _consume_response_iter(self): """ - This method handles the response header and error messages. It + This method handles the response header and error messages. It then returns an iterator for the chunks of the response """ log.debug("Handling response from Kafka") @@ -57,13 +59,15 @@ def _consume_response_iter(self): messagesize = size - 4 log.debug("About to read %d bytes from Kafka", messagesize) - # Read the remainder of the response + # Read the remainder of the response total = 0 while total < messagesize: resp = self._sock.recv(self.bufsize) log.debug("Read %d bytes from Kafka", len(resp)) if resp == "": - raise BufferUnderflowError("Not enough data to read this response") + raise BufferUnderflowError( + "Not enough data to read this response") + total += len(resp) yield resp @@ -75,9 +79,13 @@ def _consume_response_iter(self): def send(self, request_id, payload): "Send a request to Kafka" - log.debug("About to send %d bytes to Kafka, request %d" % (len(payload), request_id)) + + log.debug( + "About to send %d bytes to Kafka, request %d" % + (len(payload), request_id)) + sent = self._sock.sendall(payload) - if sent != None: + if sent is not None: raise RuntimeError("Kafka went away") def recv(self, request_id): diff --git a/kafka/consumer.py b/kafka/consumer.py index a97e8c00d..7d44f2891 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -8,7 +8,7 @@ from kafka.common import ( ErrorMapping, FetchRequest, - OffsetRequest, OffsetFetchRequest, OffsetCommitRequest, + OffsetRequest, OffsetCommitRequest, ConsumerFetchSizeTooSmall, ConsumerNoMoreData ) @@ -223,11 +223,12 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, self.fetch_min_bytes = fetch_size_bytes self.fetch_started = defaultdict(bool) # defaults to false - super(SimpleConsumer, self).__init__(client, group, topic, - partitions=partitions, - auto_commit=auto_commit, - auto_commit_every_n=auto_commit_every_n, - auto_commit_every_t=auto_commit_every_t) + super(SimpleConsumer, self).__init__( + client, group, topic, + partitions=partitions, + auto_commit=auto_commit, + auto_commit_every_n=auto_commit_every_n, + auto_commit_every_t=auto_commit_every_t) def provide_partition_info(self): """ @@ -275,8 +276,8 @@ def seek(self, offset, whence): resps = self.client.send_offset_request(reqs) for resp in resps: - self.offsets[resp.partition] = resp.offsets[0] + \ - deltas[resp.partition] + self.offsets[resp.partition] = \ + resp.offsets[0] + deltas[resp.partition] else: raise ValueError("Unexpected value for `whence`, %d" % whence) @@ -364,9 +365,10 @@ def __iter_partition__(self, partition, offset): req = FetchRequest( self.topic, partition, offset, self.client.bufsize) - (resp,) = self.client.send_fetch_request([req], - max_wait_time=self.fetch_max_wait_time, - min_bytes=fetch_size) + (resp,) = self.client.send_fetch_request( + [req], + max_wait_time=self.fetch_max_wait_time, + min_bytes=fetch_size) assert resp.topic == self.topic assert resp.partition == partition @@ -376,18 +378,22 @@ def __iter_partition__(self, partition, offset): for message in resp.messages: next_offset = message.offset - # update the offset before the message is yielded. This is - # so that the consumer state is not lost in certain cases. - # For eg: the message is yielded and consumed by the caller, - # but the caller does not come back into the generator again. - # The message will be consumed but the status will not be - # updated in the consumer + # update the offset before the message is yielded. This + # is so that the consumer state is not lost in certain + # cases. + # + # For eg: the message is yielded and consumed by the + # caller, but the caller does not come back into the + # generator again. The message will be consumed but the + # status will not be updated in the consumer self.fetch_started[partition] = True self.offsets[partition] = message.offset yield message except ConsumerFetchSizeTooSmall, e: - log.warn("Fetch size is too small, increasing by 1.5x and retrying") fetch_size *= 1.5 + log.warn( + "Fetch size too small, increasing to %d (1.5x) and retry", + fetch_size) continue except ConsumerNoMoreData, e: log.debug("Iteration was ended by %r", e) @@ -429,11 +435,12 @@ def __init__(self, client, group, topic, auto_commit=True, num_procs=1, partitions_per_proc=0): # Initiate the base consumer class - super(MultiProcessConsumer, self).__init__(client, group, topic, - partitions=None, - auto_commit=auto_commit, - auto_commit_every_n=auto_commit_every_n, - auto_commit_every_t=auto_commit_every_t) + super(MultiProcessConsumer, self).__init__( + client, group, topic, + partitions=None, + auto_commit=auto_commit, + auto_commit_every_n=auto_commit_every_n, + auto_commit_every_t=auto_commit_every_t) # Variables for managing and controlling the data flow from # consumer child process to master diff --git a/kafka/protocol.py b/kafka/protocol.py index 421e19b17..612acf63f 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -25,12 +25,12 @@ class KafkaProtocol(object): This class does not have any state associated with it, it is purely for organization. """ - PRODUCE_KEY = 0 - FETCH_KEY = 1 - OFFSET_KEY = 2 - METADATA_KEY = 3 + PRODUCE_KEY = 0 + FETCH_KEY = 1 + OFFSET_KEY = 2 + METADATA_KEY = 3 OFFSET_COMMIT_KEY = 6 - OFFSET_FETCH_KEY = 7 + OFFSET_FETCH_KEY = 7 ATTRIBUTE_CODEC_MASK = 0x03 CODEC_NONE = 0x00 @@ -120,8 +120,8 @@ def _decode_message_set_iter(cls, data): yield OffsetAndMessage(offset, message) except BufferUnderflowError: if read_message is False: - # If we get a partial read of a message, but haven't yielded anyhting - # there's a problem + # If we get a partial read of a message, but haven't + # yielded anyhting there's a problem raise ConsumerFetchSizeTooSmall() else: raise StopIteration() @@ -274,14 +274,14 @@ def decode_fetch_response(cls, data): for i in range(num_partitions): ((partition, error, highwater_mark_offset), cur) = \ - relative_unpack('>ihq', data, cur) + relative_unpack('>ihq', data, cur) (message_set, cur) = read_int_string(data, cur) yield FetchResponse( - topic, partition, error, - highwater_mark_offset, - KafkaProtocol._decode_message_set_iter(message_set)) + topic, partition, error, + highwater_mark_offset, + KafkaProtocol._decode_message_set_iter(message_set)) @classmethod def encode_offset_request(cls, client_id, correlation_id, payloads=None): @@ -321,7 +321,7 @@ def decode_offset_response(cls, data): for i in range(num_partitions): ((partition, error, num_offsets,), cur) = \ - relative_unpack('>ihi', data, cur) + relative_unpack('>ihi', data, cur) offsets = [] for j in range(num_offsets): @@ -383,17 +383,17 @@ def decode_metadata_response(cls, data): for j in range(num_partitions): ((partition_error_code, partition, leader, numReplicas), cur) = \ - relative_unpack('>hiii', data, cur) + relative_unpack('>hiii', data, cur) - (replicas, cur) = relative_unpack('>%di' % numReplicas, - data, cur) + (replicas, cur) = relative_unpack( + '>%di' % numReplicas, data, cur) ((num_isr,), cur) = relative_unpack('>i', data, cur) (isr, cur) = relative_unpack('>%di' % num_isr, data, cur) partition_metadata[partition] = \ - PartitionMetadata(topic_name, partition, leader, - replicas, isr) + PartitionMetadata( + topic_name, partition, leader, replicas, isr) topic_metadata[topic_name] = partition_metadata @@ -531,7 +531,7 @@ def create_gzip_message(payloads, key=None): key: bytes, a key used for partition routing (optional) """ message_set = KafkaProtocol._encode_message_set( - [create_message(payload) for payload in payloads]) + [create_message(payload) for payload in payloads]) gzipped = gzip_encode(message_set) codec = KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_GZIP @@ -552,7 +552,7 @@ def create_snappy_message(payloads, key=None): key: bytes, a key used for partition routing (optional) """ message_set = KafkaProtocol._encode_message_set( - [create_message(payload) for payload in payloads]) + [create_message(payload) for payload in payloads]) snapped = snappy_encode(message_set) codec = KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_SNAPPY diff --git a/kafka/queue.py b/kafka/queue.py index 41f1c313e..a99636922 100644 --- a/kafka/queue.py +++ b/kafka/queue.py @@ -25,8 +25,9 @@ def __init__(self, client, topic, partition, out_queue, barrier, Process.__init__(self) def __str__(self): - return "[KafkaConsumerProcess: topic=%s, partition=%s, sleep=%s]" % \ - (self.topic, self.partition, self.consumer_sleep) + return "[KafkaConsumerProcess: topic=%s, \ + partition=%s, sleep=%s]" % \ + (self.topic, self.partition, self.consumer_sleep) def run(self): self.barrier.wait() @@ -70,10 +71,12 @@ def __init__(self, client, topic, in_queue, barrier, 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) + 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() @@ -104,8 +107,8 @@ def flush(messages): last_produce = time.time() try: - msg = KafkaClient.create_message(self.in_queue.get(True, - self.producer_timeout)) + msg = KafkaClient.create_message( + self.in_queue.get(True, self.producer_timeout)) messages.append(msg) except Empty: diff --git a/kafka/util.py b/kafka/util.py index 96b37456e..54052fb03 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -1,9 +1,8 @@ from collections import defaultdict -from itertools import groupby import struct from threading import Thread, Event -from common import * +from kafka.common import BufferUnderflowError def write_int_string(s): @@ -39,7 +38,8 @@ def read_short_string(data, cur): def read_int_string(data, cur): if len(data) < cur + 4: raise BufferUnderflowError( - "Not enough data left to read string len (%d < %d)" % (len(data), cur + 4)) + "Not enough data left to read string len (%d < %d)" % + (len(data), cur + 4)) (strlen,) = struct.unpack('>i', data[cur:cur + 4]) if strlen == -1: From 59af614d1d09db6f7e0115dcf39232bf4f0ece9a Mon Sep 17 00:00:00 2001 From: mrtheb Date: Thu, 3 Oct 2013 23:01:35 -0400 Subject: [PATCH 025/109] Test fixes after flake8 run --- kafka/client.py | 3 ++- test/test_unit.py | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index a31ea66ae..965cbc5f5 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -1,9 +1,10 @@ from collections import defaultdict from functools import partial +from itertools import count import logging import time -from kafka.common import count, ErrorMapping, TopicAndPartition +from kafka.common import ErrorMapping, TopicAndPartition from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol diff --git a/test/test_unit.py b/test/test_unit.py index c796c949f..3f3af66ac 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -3,7 +3,8 @@ import struct import unittest -from kafka.client import KafkaClient, ProduceRequest, FetchRequest +from kafka.client import KafkaClient +from kafka.common import ProduceRequest, FetchRequest from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, @@ -59,7 +60,6 @@ class TestMisc(unittest.TestCase): def test_length_prefix(self): for i in xrange(ITERATIONS): s1 = random_string() - s2 = length_prefix_message(s1) self.assertEquals(struct.unpack('>i', s2[0:4])[0], len(s1)) From a6c99b287b1cf9c39068be74d72150808588dd43 Mon Sep 17 00:00:00 2001 From: Jim Lim Date: Fri, 27 Sep 2013 14:02:10 -0700 Subject: [PATCH 026/109] make changes to be more fault tolerant: clean up connections, brokers, failed_messages - add integration tests for sync producer - add integration tests for async producer w. leadership election - use log.exception --- kafka/client.py | 24 +++++-- kafka/common.py | 5 ++ kafka/conn.py | 28 ++++++--- kafka/producer.py | 11 +++- test/fixtures.py | 25 +++++--- test/resources/kafka.properties | 3 +- test/test_integration.py | 107 ++++++++++++++++++++++++++++++++ 7 files changed, 173 insertions(+), 30 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 965cbc5f5..c0a3cdb5c 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -2,9 +2,11 @@ from functools import partial from itertools import count import logging +import socket import time from kafka.common import ErrorMapping, TopicAndPartition +from kafka.common import ConnectionError, FailedPayloadsException from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol @@ -70,7 +72,7 @@ def _load_metadata_for_topics(self, *topics): log.debug("Broker metadata: %s", brokers) log.debug("Topic metadata: %s", topics) - self.brokers.update(brokers) + self.brokers = brokers self.topics_to_brokers = {} for topic, partitions in topics.items(): @@ -146,13 +148,15 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): for payload in payloads: leader = self._get_leader_for_partition(payload.topic, 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 = [] + # For each broker, send the list of request payloads for broker, payloads in payloads_by_broker.items(): conn = self._get_conn_for_broker(broker) @@ -161,15 +165,23 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): correlation_id=requestId, payloads=payloads) # Send the request, recv the response - conn.send(requestId, request) - - if decoder_fn is None: + try: + conn.send(requestId, request) + if decoder_fn is None: + continue + response = conn.recv(requestId) + except ConnectionError, e: # ignore BufferUnderflow for now + log.warning("Could not send request [%s] to server %s: %s" % (request, conn, e)) + failed_payloads += payloads + self.topics_to_brokers = {} # reset metadata continue - response = conn.recv(requestId) for response in decoder_fn(response): acc[(response.topic, response.partition)] = response + if failed_payloads: + raise FailedPayloadsException(failed_payloads) + # Order the accumulated responses by the original key order return (acc[k] for k in original_keys) if acc else () diff --git a/kafka/common.py b/kafka/common.py index 8f3154cc7..6f0dd322b 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -69,6 +69,11 @@ class ErrorMapping(object): # Exceptions # ################# +class FailedPayloadsException(Exception): + pass + +class ConnectionError(Exception): + pass class BufferUnderflowError(Exception): pass diff --git a/kafka/conn.py b/kafka/conn.py index e85fd110f..9356731a2 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -4,7 +4,7 @@ from threading import local from kafka.common import BufferUnderflowError - +from kafka.common import ConnectionError log = logging.getLogger("kafka") @@ -26,6 +26,7 @@ def __init__(self, host, port, bufsize=4096): self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((host, port)) self._sock.settimeout(10) + self._dirty = False def __str__(self): return "" % (self.host, self.port) @@ -53,7 +54,7 @@ def _consume_response_iter(self): # Read the size off of the header resp = self._sock.recv(4) if resp == "": - raise Exception("Got no response from Kafka") + self._raise_connection_error() (size,) = struct.unpack('>i', resp) messagesize = size - 4 @@ -71,6 +72,10 @@ def _consume_response_iter(self): total += len(resp) yield resp + def _raise_connection_error(self): + self._dirty = True + raise ConnectionError("Kafka @ {}:{} went away".format(self.host, self.port)) + ################## # Public API # ################## @@ -79,14 +84,16 @@ def _consume_response_iter(self): def send(self, request_id, payload): "Send a request to Kafka" - - log.debug( - "About to send %d bytes to Kafka, request %d" % - (len(payload), request_id)) - - sent = self._sock.sendall(payload) - if sent is not None: - raise RuntimeError("Kafka went away") + log.debug("About to send %d bytes to Kafka, request %d" % (len(payload), request_id)) + try: + if self._dirty: + self.reinit() + sent = self._sock.sendall(payload) + if sent is not None: + self._raise_connection_error() + except socket.error: + log.exception('Unable to send payload to Kafka') + self._raise_connection_error() def recv(self, request_id): """ @@ -110,3 +117,4 @@ def reinit(self): self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((self.host, self.port)) self._sock.settimeout(10) + self._dirty = False diff --git a/kafka/producer.py b/kafka/producer.py index 5f23285ab..cceb58421 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -7,6 +7,7 @@ import sys from kafka.common import ProduceRequest +from kafka.common import FailedPayloadsException from kafka.protocol import create_message from kafka.partitioner import HashedPartitioner @@ -113,7 +114,7 @@ def _send_upstream(self, queue): self.client.send_produce_request(reqs, acks=self.req_acks, timeout=self.ack_timeout) except Exception: - log.error("Error sending message", exc_info=sys.exc_info()) + log.exception("Unable to send message") def send_messages(self, partition, *msg): """ @@ -126,8 +127,12 @@ def send_messages(self, partition, *msg): else: messages = [create_message(m) for m in msg] req = ProduceRequest(self.topic, partition, messages) - resp = self.client.send_produce_request([req], acks=self.req_acks, - timeout=self.ack_timeout) + try: + resp = self.client.send_produce_request([req], acks=self.req_acks, + timeout=self.ack_timeout) + except Exception as e: + log.exception("Unable to send messages") + raise e return resp def stop(self, timeout=1): diff --git a/test/fixtures.py b/test/fixtures.py index 00c1afd00..946c64ffc 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -242,7 +242,7 @@ def close(self): class KafkaFixture(object): @staticmethod - def instance(broker_id, zk_host, zk_port, zk_chroot=None): + def instance(broker_id, zk_host, zk_port, zk_chroot=None, replicas=1, partitions=2): if zk_chroot is None: zk_chroot = "kafka-python_" + str(uuid.uuid4()).replace("-", "_") if "KAFKA_URI" in os.environ: @@ -251,11 +251,11 @@ def instance(broker_id, zk_host, zk_port, zk_chroot=None): fixture = ExternalService(host, port) else: (host, port) = ("127.0.0.1", get_open_port()) - fixture = KafkaFixture(host, port, broker_id, zk_host, zk_port, zk_chroot) + fixture = KafkaFixture(host, port, broker_id, zk_host, zk_port, zk_chroot, replicas, partitions) fixture.open() return fixture - def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot): + def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot, replicas=1, partitions=2): self.host = host self.port = port @@ -265,19 +265,24 @@ def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot): self.zk_port = zk_port self.zk_chroot = zk_chroot + self.replicas = replicas + self.partitions = partitions + self.tmp_dir = None self.child = None def open(self): self.tmp_dir = tempfile.mkdtemp() print("*** Running local Kafka instance") - print(" host = %s" % self.host) - print(" port = %s" % self.port) - print(" broker_id = %s" % self.broker_id) - print(" zk_host = %s" % self.zk_host) - print(" zk_port = %s" % self.zk_port) - print(" zk_chroot = %s" % self.zk_chroot) - print(" tmp_dir = %s" % self.tmp_dir) + print(" host = %s" % self.host) + print(" port = %s" % self.port) + print(" broker_id = %s" % self.broker_id) + print(" zk_host = %s" % self.zk_host) + print(" zk_port = %s" % self.zk_port) + print(" zk_chroot = %s" % self.zk_chroot) + print(" replicas = %s" % self.replicas) + print(" partitions = %s" % self.partitions) + print(" tmp_dir = %s" % self.tmp_dir) # Create directories os.mkdir(os.path.join(self.tmp_dir, "logs")) diff --git a/test/resources/kafka.properties b/test/resources/kafka.properties index d42c0971e..f8732fb46 100644 --- a/test/resources/kafka.properties +++ b/test/resources/kafka.properties @@ -32,7 +32,8 @@ socket.request.max.bytes=104857600 ############################# Log Basics ############################# log.dir={tmp_dir}/data -num.partitions=2 +num.partitions={partitions} +default.replication.factor={replicas} ############################# Log Flush Policy ############################# diff --git a/test/test_integration.py b/test/test_integration.py index d8ead594a..a10dae243 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -770,6 +770,113 @@ def test_large_messages(self): self.assertEquals(all_messages[i], message.message) self.assertEquals(i, 19) +class TestFailover(unittest.TestCase): + + @classmethod + def setUpClass(cls): + + zk_chroot = random_string(10) + replicas = 2 + 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)] + cls.client = KafkaClient(cls.brokers[0].host, cls.brokers[0].port) + + @classmethod + def tearDownClass(cls): + cls.client.close() + for broker in cls.brokers: + broker.close() + cls.zk.close() + + def test_switch_leader(self): + + key, topic, partition = random_string(5), 'test_switch_leader', 0 + producer = SimpleProducer(self.client, topic) + + for i in range(1, 4): + + # XXX unfortunately, the conns dict needs to be warmed for this to work + # XXX unfortunately, for warming to work, we need at least as many partitions as brokers + self._send_random_messages(producer, 10) + + # kil leader for partition 0 + broker = self._kill_leader(topic, partition) + + # expect failure, reload meta data + with self.assertRaises(FailedPayloadsException): + producer.send_messages('part 1') + producer.send_messages('part 2') + time.sleep(1) + + # send to new leader + self._send_random_messages(producer, 10) + + broker.open() + time.sleep(3) + + # count number of messages + count = self._count_messages('test_switch_leader group %s' % i, topic) + self.assertIn(count, range(20 * i, 22 * i + 1)) + + producer.stop() + + def test_switch_leader_async(self): + + key, topic, partition = random_string(5), 'test_switch_leader_async', 0 + producer = SimpleProducer(self.client, topic, async=True) + + for i in range(1, 4): + + self._send_random_messages(producer, 10) + + # kil leader for partition 0 + broker = self._kill_leader(topic, partition) + + # expect failure, reload meta data + producer.send_messages('part 1') + producer.send_messages('part 2') + time.sleep(1) + + # send to new leader + self._send_random_messages(producer, 10) + + broker.open() + time.sleep(3) + + # count number of messages + count = self._count_messages('test_switch_leader_async group %s' % i, topic) + self.assertIn(count, range(20 * i, 22 * i + 1)) + + producer.stop() + + def _send_random_messages(self, producer, n): + for j in range(n): + resp = producer.send_messages(random_string(10)) + if len(resp) > 0: + self.assertEquals(resp[0].error, 0) + time.sleep(1) # give it some time + + def _kill_leader(self, topic, partition): + leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] + broker = self.brokers[leader.nodeId] + broker.close() + time.sleep(1) # give it some time + return broker + + def _count_messages(self, group, topic): + client = KafkaClient(self.brokers[0].host, self.brokers[0].port) + consumer = SimpleConsumer(client, group, topic, auto_commit=False) + all_messages = [] + for message in consumer: + all_messages.append(message) + consumer.stop() + client.close() + return len(all_messages) + def random_string(l): s = "".join(random.choice(string.letters) for i in xrange(l)) From 75de0f00956eb7cf0394fcfabb6a7d63057409fe Mon Sep 17 00:00:00 2001 From: Mahendra M Date: Mon, 7 Oct 2013 15:57:44 +0530 Subject: [PATCH 027/109] Ensure that async producer works in windows. Fixes #46 As per the multiprocessing module's documentation, the objects passed to the Process() class must be pickle-able in Windows. So, the Async producer did not work in windows. To fix this we have to ensure that code which uses multiprocessing has to follow certain rules * The target=func should not be a member function * We cannot pass objects like socket() to multiprocessing This ticket fixes these issues. For KafkaClient and KafkaConnection objects, we make copies of the object and reinit() them inside the child processes. --- kafka/client.py | 11 +++++ kafka/conn.py | 15 ++++++- kafka/producer.py | 112 ++++++++++++++++++++++++++-------------------- 3 files changed, 87 insertions(+), 51 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 965cbc5f5..b7ceb2e3e 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -1,3 +1,4 @@ +import copy from collections import defaultdict from functools import partial from itertools import count @@ -181,6 +182,16 @@ def close(self): for conn in self.conns.values(): conn.close() + 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 + """ + c = copy.deepcopy(self) + for k, v in c.conns.items(): + c.conns[k] = v.copy() + return c + def reinit(self): for conn in self.conns.values(): conn.reinit() diff --git a/kafka/conn.py b/kafka/conn.py index e85fd110f..194a19cb2 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -1,3 +1,4 @@ +import copy import logging import socket import struct @@ -96,17 +97,27 @@ def recv(self, request_id): self.data = self._consume_response() return self.data + 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 + """ + c = copy.deepcopy(self) + c._sock = None + return c + def close(self): """ Close this connection """ - self._sock.close() + if self._sock: + self._sock.close() def reinit(self): """ Re-initialize the socket connection """ - self._sock.close() + self.close() self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((self.host, self.port)) self._sock.settimeout(10) diff --git a/kafka/producer.py b/kafka/producer.py index 5f23285ab..a7bfe288e 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -18,6 +18,58 @@ STOP_ASYNC_PRODUCER = -1 +def _send_upstream(topic, queue, client, batch_time, batch_size, + req_acks, ack_timeout): + """ + 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 + count = batch_size + send_at = datetime.now() + timedelta(seconds=timeout) + msgset = defaultdict(list) + + # Keep fetching till we gather enough messages or a + # timeout is reached + while count > 0 and timeout >= 0: + try: + partition, msg = queue.get(timeout=timeout) + except Empty: + break + + # Check if the controller has requested us to stop + if partition == STOP_ASYNC_PRODUCER: + stop = True + break + + # Adjust the timeout to match the remaining period + count -= 1 + timeout = (send_at - datetime.now()).total_seconds() + msgset[partition].append(msg) + + # Send collected requests upstream + reqs = [] + for partition, messages in msgset.items(): + req = ProduceRequest(topic, partition, messages) + reqs.append(req) + + try: + client.send_produce_request(reqs, + acks=req_acks, + timeout=ack_timeout) + except Exception as exp: + log.error("Error sending message", exc_info=sys.exc_info()) + + class Producer(object): """ Base class to be used by producers @@ -61,60 +113,22 @@ def __init__(self, client, async=False, self.async = async self.req_acks = req_acks self.ack_timeout = ack_timeout - self.batch_send = batch_send - self.batch_size = batch_send_every_n - self.batch_time = batch_send_every_t if self.async: self.queue = Queue() # Messages are sent through this queue - self.proc = Process(target=self._send_upstream, args=(self.queue,)) - self.proc.daemon = True # Process will die if main thread exits + self.proc = Process(target=_send_upstream, + args=(self.topic, + self.queue, + self.client.copy(), + batch_send_every_t, + batch_send_every_n, + self.req_acks, + self.ack_timeout)) + + # Process will die if main thread exits + self.proc.daemon = True self.proc.start() - def _send_upstream(self, queue): - """ - 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 - - while not stop: - timeout = self.batch_time - send_at = datetime.now() + timedelta(seconds=timeout) - count = self.batch_size - msgset = defaultdict(list) - - # Keep fetching till we gather enough messages or a - # timeout is reached - while count > 0 and timeout >= 0: - try: - partition, msg = queue.get(timeout=timeout) - except Empty: - break - - # Check if the controller has requested us to stop - if partition == STOP_ASYNC_PRODUCER: - stop = True - break - - # Adjust the timeout to match the remaining period - count -= 1 - timeout = (send_at - datetime.now()).total_seconds() - msgset[partition].append(msg) - - # Send collected requests upstream - reqs = [] - for partition, messages in msgset.items(): - req = ProduceRequest(self.topic, partition, messages) - reqs.append(req) - - try: - self.client.send_produce_request(reqs, acks=self.req_acks, - timeout=self.ack_timeout) - except Exception: - log.error("Error sending message", exc_info=sys.exc_info()) - def send_messages(self, partition, *msg): """ Helper method to send produce requests From 6e3ee64ef3d8e65de9aab601e7239fbd1ac0de93 Mon Sep 17 00:00:00 2001 From: Mahendra M Date: Tue, 8 Oct 2013 09:28:58 +0530 Subject: [PATCH 028/109] Ensure that multiprocess consumer works in windows --- kafka/consumer.py | 116 +++++++++++++++++++++++++--------------------- 1 file changed, 63 insertions(+), 53 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 7d44f2891..f2898ad43 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -404,6 +404,63 @@ def __iter_partition__(self, partition, offset): offset = next_offset + 1 +def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): + """ + A child process worker which consumes messages based on the + notifications given by the controller process + + NOTE: Ideally, this should have been a method inside the Consumer + class. However, multiprocessing module has issues in windows. The + functionality breaks unless this function is kept outside of a class + """ + + # 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, + partitions=chunk, + auto_commit=False, + auto_commit_every_n=None, + auto_commit_every_t=None) + + # 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() + + # If we are asked to quit, do so + if 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 + + for partition, message in consumer: + queue.put((partition, message)) + count += 1 + + # 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: + pause.wait() + break + + # In case we did not receive any message, give up the CPU for + # a while before we try again + if count == 0: + time.sleep(0.1) + + consumer.stop() + + class MultiProcessConsumer(Consumer): """ A consumer implementation that consumes partitions for a topic in @@ -468,63 +525,16 @@ def __init__(self, client, group, topic, auto_commit=True, self.procs = [] for chunk in chunks: chunk = filter(lambda x: x is not None, chunk) - proc = Process(target=self._consume, args=(chunk,)) + args = (client.copy(), + group, topic, chunk, + self.queue, self.start, self.exit, + self.pause, self.size) + + proc = Process(target=_mp_consume, args=args) proc.daemon = True proc.start() self.procs.append(proc) - def _consume(self, partitions): - """ - A child process worker which consumes messages based on the - notifications given by the controller process - """ - - # Make the child processes open separate socket connections - self.client.reinit() - - # We will start consumers without auto-commit. Auto-commit will be - # done by the master controller process. - consumer = SimpleConsumer(self.client, self.group, self.topic, - partitions=partitions, - auto_commit=False, - auto_commit_every_n=None, - auto_commit_every_t=None) - - # Ensure that the consumer provides the partition information - consumer.provide_partition_info() - - while True: - # Wait till the controller indicates us to start consumption - self.start.wait() - - # If we are asked to quit, do so - if self.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 - - for partition, message in consumer: - self.queue.put((partition, message)) - count += 1 - - # 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 == self.size.value: - self.pause.wait() - break - - # In case we did not receive any message, give up the CPU for - # a while before we try again - if count == 0: - time.sleep(0.1) - - consumer.stop() - def stop(self): # Set exit and start off all waiting consumers self.exit.set() From ceee715fc7bf17b28ff5a10f1f7decc8974f6506 Mon Sep 17 00:00:00 2001 From: Mahendra M Date: Tue, 8 Oct 2013 14:46:02 +0530 Subject: [PATCH 029/109] Add proper string representations for each class --- kafka/client.py | 5 ++++- kafka/conn.py | 2 +- kafka/consumer.py | 8 ++++++++ kafka/producer.py | 6 ++++++ 4 files changed, 19 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 71ededaa0..51dda1e76 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -174,7 +174,7 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): except ConnectionError, e: # ignore BufferUnderflow for now log.warning("Could not send request [%s] to server %s: %s" % (request, conn, e)) failed_payloads += payloads - self.topics_to_brokers = {} # reset metadata + self.topics_to_brokers = {} # reset metadata continue for response in decoder_fn(response): @@ -186,6 +186,9 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): # Order the accumulated responses by the original key order return (acc[k] for k in original_keys) if acc else () + def __repr__(self): + return '' % (self.client_id) + ################# # Public API # ################# diff --git a/kafka/conn.py b/kafka/conn.py index 14aebc6f5..712b24d0d 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -29,7 +29,7 @@ def __init__(self, host, port, bufsize=4096): self._sock.settimeout(10) self._dirty = False - def __str__(self): + def __repr__(self): return "" % (self.host, self.port) ################### diff --git a/kafka/consumer.py b/kafka/consumer.py index 7d44f2891..ec04f5f2a 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -230,6 +230,10 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, auto_commit_every_n=auto_commit_every_n, auto_commit_every_t=auto_commit_every_t) + def __repr__(self): + return '' % \ + (self.group, self.topic, str(self.offsets.keys())) + def provide_partition_info(self): """ Indicates that partition info must be returned by the consumer @@ -473,6 +477,10 @@ def __init__(self, client, group, topic, auto_commit=True, proc.start() self.procs.append(proc) + def __repr__(self): + return '' % \ + (self.group, self.topic, len(self.procs)) + def _consume(self, partitions): """ A child process worker which consumes messages based on the diff --git a/kafka/producer.py b/kafka/producer.py index 7ef789684..827570112 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -198,6 +198,9 @@ def send_messages(self, *msg): partition = self.next_partition.next() return super(SimpleProducer, self).send_messages(partition, *msg) + def __repr__(self): + return '' % (self.topic, self.async) + class KeyedProducer(Producer): """ @@ -239,3 +242,6 @@ def send(self, key, msg): partitions = self.client.topic_partitions[self.topic] partition = self.partitioner.partition(key, partitions) return self.send_messages(partition, msg) + + def __repr__(self): + return '' % (self.topic, self.async) From da255638aa27e42e42f73356d4d25c2c3c40ea3a Mon Sep 17 00:00:00 2001 From: Joe Crobak Date: Mon, 21 Oct 2013 20:07:43 +0000 Subject: [PATCH 030/109] Enable absolute imports for modules using Queue. When running on Linux with code on a case-insensitive file system, imports of the `Queue` module fail because python resolves the wrong file (It is trying to use a relative import of `queue.py` in the kafka directory). This change forces absolute imports via PEP328. --- kafka/consumer.py | 2 ++ kafka/producer.py | 2 ++ kafka/queue.py | 2 ++ 3 files changed, 6 insertions(+) diff --git a/kafka/consumer.py b/kafka/consumer.py index f2898ad43..3d9e4d808 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import + from collections import defaultdict from itertools import izip_longest, repeat import logging diff --git a/kafka/producer.py b/kafka/producer.py index 7ef789684..317454830 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import + from collections import defaultdict from datetime import datetime, timedelta from itertools import cycle diff --git a/kafka/queue.py b/kafka/queue.py index a99636922..ada495f78 100644 --- a/kafka/queue.py +++ b/kafka/queue.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import + from copy import copy import logging from multiprocessing import Process, Queue, Event From af3a57edb2c83c35b832e759b4c24ec72149841a Mon Sep 17 00:00:00 2001 From: David Arthur Date: Mon, 11 Nov 2013 10:20:25 -0500 Subject: [PATCH 031/109] Set host and port in ExternalService test fixture So we can run tests against an already running Kafka instance KAFKA_URI=tcp://localhost:9092 python -m test.test_integration --- test/fixtures.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test/fixtures.py b/test/fixtures.py index 946c64ffc..c771a5859 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -74,6 +74,8 @@ def render_template(source_file, target_file, binding): class ExternalService(object): def __init__(self, host, port): print("Using already running service at %s:%d" % (host, port)) + self.host = host + self.port = port def open(self): pass From 0bdff4e833f73518a7219fca04dfbc3ed201b06e Mon Sep 17 00:00:00 2001 From: Marc Labbe Date: Thu, 14 Nov 2013 09:26:49 -0500 Subject: [PATCH 032/109] Allow KafkaClient to take in a list of brokers for bootstrapping --- README.md | 8 +- example.py | 2 +- kafka/NOTES.md | 2 +- kafka/client.py | 35 +++++--- kafka/conn.py | 33 +++++-- setup.py | 5 +- test/test_integration.py | 23 +++-- test/test_unit.py | 187 ++++++++++++++++++++++++++++++++++++++- 8 files changed, 256 insertions(+), 39 deletions(-) diff --git a/README.md b/README.md index c9f782d12..edf393186 100644 --- a/README.md +++ b/README.md @@ -30,7 +30,7 @@ from kafka.client import KafkaClient from kafka.consumer import SimpleConsumer from kafka.producer import SimpleProducer, KeyedProducer -kafka = KafkaClient("localhost", 9092) +kafka = KafkaClient("localhost:9092") # To send messages synchronously producer = SimpleProducer(kafka, "my-topic") @@ -81,7 +81,7 @@ from kafka.client import KafkaClient from kafka.producer import KeyedProducer from kafka.partitioner import HashedPartitioner, RoundRobinPartitioner -kafka = KafkaClient("localhost", 9092) +kafka = KafkaClient("localhost:9092") # HashedPartitioner is default producer = KeyedProducer(kafka, "my-topic") @@ -96,7 +96,7 @@ producer = KeyedProducer(kafka, "my-topic", partitioner=RoundRobinPartitioner) from kafka.client import KafkaClient from kafka.consumer import 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) @@ -116,7 +116,7 @@ for message in consumer.get_messages(count=5, block=True, timeout=4): ```python from kafka.client import KafkaClient -kafka = KafkaClient("localhost", 9092) +kafka = KafkaClient("localhost:9092") req = ProduceRequest(topic="my-topic", partition=1, messages=[KafkaProdocol.encode_message("some message")]) resps = kafka.send_produce_request(payloads=[req], fail_on_error=True) diff --git a/example.py b/example.py index 3a2dc928b..0cf5583ee 100644 --- a/example.py +++ b/example.py @@ -14,7 +14,7 @@ def consume_example(client): print(message) def main(): - client = KafkaClient("localhost", 9092) + client = KafkaClient("localhost:9092") produce_example(client) consume_example(client) diff --git a/kafka/NOTES.md b/kafka/NOTES.md index 540cdad3b..8fb0f4744 100644 --- a/kafka/NOTES.md +++ b/kafka/NOTES.md @@ -18,7 +18,7 @@ There are a few levels of abstraction: # Possible API - client = KafkaClient("localhost", 9092) + client = KafkaClient("localhost:9092") producer = KafkaProducer(client, "topic") producer.send_string("hello") diff --git a/kafka/client.py b/kafka/client.py index 71ededaa0..81eec7d11 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -8,7 +8,7 @@ from kafka.common import ErrorMapping, TopicAndPartition from kafka.common import ConnectionError, FailedPayloadsException -from kafka.conn import KafkaConnection +from kafka.conn import collect_hosts, KafkaConnection from kafka.protocol import KafkaProtocol log = logging.getLogger("kafka") @@ -19,13 +19,15 @@ class KafkaClient(object): CLIENT_ID = "kafka-python" ID_GEN = count() - def __init__(self, host, port, bufsize=4096, client_id=CLIENT_ID): + def __init__(self, hosts, bufsize=4096, client_id=CLIENT_ID): # We need one connection to bootstrap self.bufsize = bufsize self.client_id = client_id - self.conns = { # (host, port) -> KafkaConnection - (host, port): KafkaConnection(host, port, bufsize) - } + + self.hosts = collect_hosts(hosts) + + # create connections only when we need them + self.conns = {} self.brokers = {} # broker_id -> BrokerMetadata self.topics_to_brokers = {} # topic_id -> broker_id self.topic_partitions = defaultdict(list) # topic_id -> [0, 1, 2, ...] @@ -35,15 +37,19 @@ def __init__(self, host, port, bufsize=4096, client_id=CLIENT_ID): # Private API # ################## + def _get_conn(self, host, 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(host, port, self.bufsize) + + return self.conns[host_key] + def _get_conn_for_broker(self, broker): - """ - Get or create a connection to a broker - """ - if (broker.host, broker.port) not in self.conns: - self.conns[(broker.host, broker.port)] = \ - KafkaConnection(broker.host, broker.port, self.bufsize) + "Get or create a connection to a broker" - return self.conns[(broker.host, broker.port)] + return self._get_conn(broker.host, broker.port) def _get_leader_for_partition(self, topic, partition): key = TopicAndPartition(topic, partition) @@ -108,7 +114,8 @@ def _send_broker_unaware_request(self, requestId, request): Attempt to send a broker-agnostic request to one of the available brokers. Keep trying until you succeed. """ - for conn in self.conns.values(): + for (host, port) in self.hosts: + conn = self._get_conn(host, port) try: conn.send(requestId, request) response = conn.recv(requestId) @@ -174,7 +181,7 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): except ConnectionError, e: # ignore BufferUnderflow for now log.warning("Could not send request [%s] to server %s: %s" % (request, conn, e)) failed_payloads += payloads - self.topics_to_brokers = {} # reset metadata + self.topics_to_brokers = {} # reset metadata continue for response in decoder_fn(response): diff --git a/kafka/conn.py b/kafka/conn.py index 14aebc6f5..614b1bbcc 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -2,6 +2,7 @@ import logging import socket import struct +from random import shuffle from threading import local from kafka.common import BufferUnderflowError @@ -10,6 +11,26 @@ log = logging.getLogger("kafka") +def collect_hosts(hosts, randomize=True): + """ + Collects a comma-separated set of hosts (host:port) and optionnaly + randomize the returned list. + """ + + result = [] + for host_port in hosts.split(","): + + res = host_port.split(':') + host = res[0] + port = int(res[1]) if len(res) > 1 else 9092 + result.append((host.strip(), port)) + + if randomize: + shuffle(result) + + return result + + class KafkaConnection(local): """ A socket connection to a single Kafka broker @@ -19,14 +40,14 @@ class KafkaConnection(local): we can do something in here to facilitate multiplexed requests/responses since the Kafka API includes a correlation id. """ - def __init__(self, host, port, bufsize=4096): + def __init__(self, host, port, bufsize=4096, timeout=10): super(KafkaConnection, self).__init__() self.host = host self.port = port self.bufsize = bufsize - self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self._sock.connect((host, port)) - self._sock.settimeout(10) + self.timeout = timeout + + self._sock = socket.create_connection((host, port), timeout=timeout) self._dirty = False def __str__(self): @@ -125,7 +146,5 @@ def reinit(self): Re-initialize the socket connection """ self.close() - self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self._sock.connect((self.host, self.port)) - self._sock.settimeout(10) + self._sock = socket.create_connection((self.host, self.port), timeout=self.timeout) self._dirty = False diff --git a/setup.py b/setup.py index a24691407..fedf1392e 100644 --- a/setup.py +++ b/setup.py @@ -1,11 +1,12 @@ -import os.path import sys from setuptools import setup, Command class Tox(Command): + user_options = [] + def initialize_options(self): pass @@ -21,7 +22,7 @@ def run(self): name="kafka-python", version="0.8.1-1", - install_requires=["distribute", "tox"], + install_requires=["distribute", "tox", "mock"], tests_require=["tox"], cmdclass={"test": Tox}, diff --git a/test/test_integration.py b/test/test_integration.py index a10dae243..1f37ebfcf 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -16,7 +16,7 @@ class TestKafkaClient(unittest.TestCase): def setUpClass(cls): # noqa cls.zk = ZookeeperFixture.instance() cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) - cls.client = KafkaClient(cls.server.host, cls.server.port) + cls.client = KafkaClient('%s:%d' % (cls.server.host, cls.server.port)) @classmethod def tearDownClass(cls): # noqa @@ -554,7 +554,7 @@ def setUpClass(cls): cls.zk = ZookeeperFixture.instance() cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) - cls.client = KafkaClient(cls.server2.host, cls.server2.port, bufsize=8192) + cls.client = KafkaClient('%s:%d' % (cls.server2.host, cls.server2.port), bufsize=8192) @classmethod def tearDownClass(cls): # noqa @@ -770,20 +770,23 @@ def test_large_messages(self): self.assertEquals(all_messages[i], message.message) self.assertEquals(i, 19) + class TestFailover(unittest.TestCase): @classmethod def setUpClass(cls): zk_chroot = random_string(10) - replicas = 2 + replicas = 2 partitions = 2 # mini zookeeper, 2 kafka brokers - cls.zk = ZookeeperFixture.instance() - kk_args = [cls.zk.host, cls.zk.port, zk_chroot, replicas, partitions] + 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)] - cls.client = KafkaClient(cls.brokers[0].host, cls.brokers[0].port) + + hosts = ','.join(['%s:%d' % (b.host, b.port) for b in cls.brokers]) + cls.client = KafkaClient(hosts) @classmethod def tearDownClass(cls): @@ -858,17 +861,19 @@ def _send_random_messages(self, producer, n): resp = producer.send_messages(random_string(10)) if len(resp) > 0: self.assertEquals(resp[0].error, 0) - time.sleep(1) # give it some time + time.sleep(1) # give it some time def _kill_leader(self, topic, partition): leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] broker = self.brokers[leader.nodeId] broker.close() - time.sleep(1) # give it some time + time.sleep(1) # give it some time return broker def _count_messages(self, group, topic): - client = KafkaClient(self.brokers[0].host, self.brokers[0].port) + + hosts = '%s:%d' % (self.brokers[0].host, self.brokers[0].port) + client = KafkaClient(hosts) consumer = SimpleConsumer(client, group, topic, auto_commit=False) all_messages = [] for message in consumer: diff --git a/test/test_unit.py b/test/test_unit.py index 3f3af66ac..4ea944273 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -3,13 +3,18 @@ import struct import unittest +from mock import patch + from kafka.client import KafkaClient -from kafka.common import ProduceRequest, FetchRequest from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, snappy_encode, snappy_decode ) +from kafka.common import ( + ProduceRequest, FetchRequest, + BrokerMetadata, PartitionMetadata, TopicAndPartition +) ITERATIONS = 1000 STRLEN = 100 @@ -217,5 +222,185 @@ def test_fetch_request(self): self.assertEquals(enc, expect) +class TestKafkaClient(unittest.TestCase): + + def test_send_broker_unaware_request_fail(self): + 'Tests that call fails when all hosts are unavailable' + + from mock import MagicMock + + mocked_conns = { + ('kafka01', 9092): MagicMock(), + ('kafka02', 9092): MagicMock() + } + # inject conns + mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)") + mocked_conns[('kafka02', 9092)].send.side_effect = RuntimeError("Kafka02 went away (unittest)") + + def mock_get_conn(host, port): + print 'mock_get_conn: %s:%d=%s' % (host, port, mocked_conns[(host, port)]) + return mocked_conns[(host, port)] + + # patch to avoid making requests before we want it + with patch.object(KafkaClient, '_load_metadata_for_topics'), \ + patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): + + client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + + resp = client._send_broker_unaware_request(1, 'fake request') + + self.assertIsNone(resp) + + for key, conn in mocked_conns.iteritems(): + conn.send.assert_called_with(1, 'fake request') + + def test_send_broker_unaware_request(self): + 'Tests that call fails when one of the host is available' + + from mock import MagicMock + + mocked_conns = { + ('kafka01', 9092): MagicMock(), + ('kafka02', 9092): MagicMock(), + ('kafka03', 9092): MagicMock() + } + # inject conns + mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)") + mocked_conns[('kafka02', 9092)].recv.return_value = 'valid response' + mocked_conns[('kafka03', 9092)].send.side_effect = RuntimeError("kafka03 went away (unittest)") + + def mock_get_conn(host, port): + print 'mock_get_conn: %s:%d=%s' % (host, port, mocked_conns[(host, port)]) + return mocked_conns[(host, port)] + + # patch to avoid making requests before we want it + with patch.object(KafkaClient, '_load_metadata_for_topics'), \ + patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): + + client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + + resp = client._send_broker_unaware_request(1, 'fake request') + + self.assertEqual('valid response', resp) + mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) + + @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_client_load_metadata(self, protocol, conn): + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(1, 'broker_1', 4567) + brokers[1] = BrokerMetadata(2, 'broker_2', 5678) + + topics = {} + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2]) + } + topics['topic_2'] = { + 0: PartitionMetadata('topic_2', 0, 0, [0, 1], [0, 1]), + 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(hosts='broker_1:4567') + self.assertItemsEqual( + { + TopicAndPartition('topic_1', 0): brokers[0], + TopicAndPartition('topic_2', 0): brokers[0], + TopicAndPartition('topic_2', 1): brokers[1] + }, + client.topics_to_brokers) + + @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_client_load_metadata_unassigned_partitions(self, protocol, conn): + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(0, 'broker_1', 4567) + brokers[1] = BrokerMetadata(1, 'broker_2', 5678) + + topics = {} + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, -1, [], []) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(hosts='broker_1:4567') + + self.assertItemsEqual({}, client.topics_to_brokers) + self.assertRaises( + Exception, + client._get_leader_for_partition, + 'topic_1', 0) + + # calling _get_leader_for_partition (from any broker aware request) + # will try loading metadata again for the same topic + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) + } + leader = client._get_leader_for_partition('topic_1', 0) + + self.assertEqual(brokers[0], leader) + self.assertItemsEqual( + { + TopicAndPartition('topic_1', 0): brokers[0], + }, + client.topics_to_brokers) + + @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_client_load_metadata_noleader_partitions(self, protocol, conn): + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(0, 'broker_1', 4567) + brokers[1] = BrokerMetadata(1, 'broker_2', 5678) + + topics = {} + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, -1, [], []) + } + topics['topic_2'] = { + 0: PartitionMetadata('topic_2', 0, 0, [0, 1], []), + 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(hosts='broker_1:4567') + self.assertItemsEqual( + { + TopicAndPartition('topic_2', 0): brokers[0], + TopicAndPartition('topic_2', 1): brokers[1] + }, + client.topics_to_brokers) + self.assertRaises( + Exception, + client._get_leader_for_partition, + 'topic_1', 0) + + # calling _get_leader_for_partition (from any broker aware request) + # will try loading metadata again for the same topic + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) + } + leader = client._get_leader_for_partition('topic_1', 0) + + self.assertEqual(brokers[0], leader) + self.assertItemsEqual( + { + TopicAndPartition('topic_1', 0): brokers[0], + TopicAndPartition('topic_2', 0): brokers[0], + TopicAndPartition('topic_2', 1): brokers[1] + }, + client.topics_to_brokers) + if __name__ == '__main__': unittest.main() From cc96e5c89aa65427cb0851e21a47cffa675987dc Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 6 Dec 2013 21:18:26 -0500 Subject: [PATCH 033/109] Update kafka-src to 0.8.0 tag --- kafka-src | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-src b/kafka-src index 5bd33c151..15bb3961d 160000 --- a/kafka-src +++ b/kafka-src @@ -1 +1 @@ -Subproject commit 5bd33c1517bb2e7734166dc3e787ac90a4ef8f86 +Subproject commit 15bb3961d9171c1c54c4c840a554ce2c76168163 From 8d26368433dffb65ff8d144203fc59d8162974fe Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 12 Dec 2013 17:03:35 -0800 Subject: [PATCH 034/109] allow for timeout to be None in SimpleConsumer.get_messages --- kafka/consumer.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index f2898ad43..57b5b9742 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -294,7 +294,8 @@ def get_messages(self, count=1, block=True, timeout=0.1): iterator = self.__iter__() # HACK: This splits the timeout between available partitions - timeout = timeout * 1.0 / len(self.offsets) + if timeout: + timeout = timeout * 1.0 / len(self.offsets) with FetchContext(self, block, timeout): while count > 0: From 043e7fda6b07cdd7230b785f9aa1841113e4228c Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 12 Dec 2013 17:39:19 -0800 Subject: [PATCH 035/109] fix typo in README acks_timeout -> ack_timeout --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c9f782d12..b79882773 100644 --- a/README.md +++ b/README.md @@ -48,7 +48,7 @@ producer.send_messages("async message") # by all in sync replicas before sending a response producer = SimpleProducer(kafka, "my-topic", async=False, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE, - acks_timeout=2000) + ack_timeout=2000) response = producer.send_messages("async message") From 7f1db1d34696f2ce598a7796a19eeb3102d99904 Mon Sep 17 00:00:00 2001 From: Evan Klitzke Date: Wed, 25 Dec 2013 16:16:05 -0800 Subject: [PATCH 036/109] reduce memory copies when consuming kafka responses --- kafka/conn.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 14aebc6f5..1a3e2607c 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -38,12 +38,9 @@ def __str__(self): def _consume_response(self): """ - Fully consumer the response iterator + Fully consume the response iterator """ - data = "" - for chunk in self._consume_response_iter(): - data += chunk - return data + return "".join(self._consume_response_iter()) def _consume_response_iter(self): """ From cab017a9aa92fa63550eb72e1571508b431ce791 Mon Sep 17 00:00:00 2001 From: Niek Sanders Date: Wed, 25 Dec 2013 22:00:21 -0800 Subject: [PATCH 037/109] Replaced _send_upstream datetime logic with simpler time(). --- kafka/producer.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/producer.py b/kafka/producer.py index 7ef789684..1d4733688 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -1,10 +1,10 @@ from collections import defaultdict -from datetime import datetime, timedelta from itertools import cycle from multiprocessing import Queue, Process from Queue import Empty import logging import sys +import time from kafka.common import ProduceRequest from kafka.common import FailedPayloadsException @@ -36,7 +36,7 @@ def _send_upstream(topic, queue, client, batch_time, batch_size, while not stop: timeout = batch_time count = batch_size - send_at = datetime.now() + timedelta(seconds=timeout) + send_at = time.time() + timeout msgset = defaultdict(list) # Keep fetching till we gather enough messages or a @@ -54,7 +54,7 @@ def _send_upstream(topic, queue, client, batch_time, batch_size, # Adjust the timeout to match the remaining period count -= 1 - timeout = (send_at - datetime.now()).total_seconds() + timeout = send_at - time.time() msgset[partition].append(msg) # Send collected requests upstream From 607bfa0aaefcd64a33aaaa9d82c90812020784c3 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Sat, 28 Dec 2013 14:56:00 -0500 Subject: [PATCH 038/109] Remove tox from requires Fixes #85 --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index a24691407..8fd18289f 100644 --- a/setup.py +++ b/setup.py @@ -21,7 +21,7 @@ def run(self): name="kafka-python", version="0.8.1-1", - install_requires=["distribute", "tox"], + install_requires=["distribute"], tests_require=["tox"], cmdclass={"test": Tox}, From 60ccb4dd025ec3e3da6feb77f9797aa1da723bfa Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 18 Dec 2013 16:53:03 -0800 Subject: [PATCH 039/109] Allow customizing socket timeouts. Previously, if you try to consume a message with a timeout greater than 10 seconds, but you don't receive data in those 10 seconds, a socket.timeout exception is raised. This allows a higher socket timeout to be set, or even None for no timeout. --- kafka/client.py | 7 ++++--- kafka/conn.py | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 71ededaa0..96593642f 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -19,12 +19,13 @@ class KafkaClient(object): CLIENT_ID = "kafka-python" ID_GEN = count() - def __init__(self, host, port, bufsize=4096, client_id=CLIENT_ID): + def __init__(self, host, port, bufsize=4098, client_id=CLIENT_ID, timeout=10): # We need one connection to bootstrap self.bufsize = bufsize self.client_id = client_id + self.timeout = timeout self.conns = { # (host, port) -> KafkaConnection - (host, port): KafkaConnection(host, port, bufsize) + (host, port): KafkaConnection(host, port, bufsize, timeout=timeout) } self.brokers = {} # broker_id -> BrokerMetadata self.topics_to_brokers = {} # topic_id -> broker_id @@ -41,7 +42,7 @@ def _get_conn_for_broker(self, broker): """ if (broker.host, broker.port) not in self.conns: self.conns[(broker.host, broker.port)] = \ - KafkaConnection(broker.host, broker.port, self.bufsize) + KafkaConnection(broker.host, broker.port, self.bufsize, timeout=self.timeout) return self.conns[(broker.host, broker.port)] diff --git a/kafka/conn.py b/kafka/conn.py index 1a3e2607c..6dd61cc5a 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -19,14 +19,14 @@ class KafkaConnection(local): we can do something in here to facilitate multiplexed requests/responses since the Kafka API includes a correlation id. """ - def __init__(self, host, port, bufsize=4096): + def __init__(self, host, port, bufsize=4098, timeout=10): super(KafkaConnection, self).__init__() self.host = host self.port = port self.bufsize = bufsize self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((host, port)) - self._sock.settimeout(10) + self._sock.settimeout(timeout) self._dirty = False def __str__(self): From 0f2b08d80217fb82860c51e05e819012f6acb521 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 18 Dec 2013 17:47:52 -0800 Subject: [PATCH 040/109] Read the correct number of bytes from kafka. According to the protocol documentation, the 4 byte integer at the beginning of a response represents the size of the payload only, not including those bytes. See http://goo.gl/rg5uom --- kafka/conn.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 6dd61cc5a..1997804ff 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -55,12 +55,11 @@ def _consume_response_iter(self): self._raise_connection_error() (size,) = struct.unpack('>i', resp) - messagesize = size - 4 - log.debug("About to read %d bytes from Kafka", messagesize) + log.debug("About to read %d bytes from Kafka", size) # Read the remainder of the response total = 0 - while total < messagesize: + while total < size: resp = self._sock.recv(self.bufsize) log.debug("Read %d bytes from Kafka", len(resp)) if resp == "": From 8c8ca5fa573c21e0f03c892154ba42e187153600 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 18 Dec 2013 17:51:22 -0800 Subject: [PATCH 041/109] * Guarantee reading the expected number of bytes from the socket every time * Remove bufsize from client and conn, since they're not actually enforced Notes: This commit changes behavior a bit by raising a BufferUnderflowError when no data is received for the message size rather than a ConnectionError. Since bufsize in the socket is not actually enforced, but it is used by the consumer when creating requests, moving it there until a better solution is implemented. --- kafka/client.py | 7 +++---- kafka/conn.py | 48 ++++++++++++++++++++++------------------------- kafka/consumer.py | 7 +++++-- 3 files changed, 30 insertions(+), 32 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 96593642f..bd3a21406 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -19,13 +19,12 @@ class KafkaClient(object): CLIENT_ID = "kafka-python" ID_GEN = count() - def __init__(self, host, port, bufsize=4098, client_id=CLIENT_ID, timeout=10): + def __init__(self, host, port, client_id=CLIENT_ID, timeout=10): # We need one connection to bootstrap - self.bufsize = bufsize self.client_id = client_id self.timeout = timeout self.conns = { # (host, port) -> KafkaConnection - (host, port): KafkaConnection(host, port, bufsize, timeout=timeout) + (host, port): KafkaConnection(host, port, timeout=timeout) } self.brokers = {} # broker_id -> BrokerMetadata self.topics_to_brokers = {} # topic_id -> broker_id @@ -42,7 +41,7 @@ def _get_conn_for_broker(self, broker): """ if (broker.host, broker.port) not in self.conns: self.conns[(broker.host, broker.port)] = \ - KafkaConnection(broker.host, broker.port, self.bufsize, timeout=self.timeout) + KafkaConnection(broker.host, broker.port, timeout=self.timeout) return self.conns[(broker.host, broker.port)] diff --git a/kafka/conn.py b/kafka/conn.py index 1997804ff..ca62f52b4 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -19,11 +19,10 @@ class KafkaConnection(local): we can do something in here to facilitate multiplexed requests/responses since the Kafka API includes a correlation id. """ - def __init__(self, host, port, bufsize=4098, timeout=10): + def __init__(self, host, port, timeout=10): super(KafkaConnection, self).__init__() self.host = host self.port = port - self.bufsize = bufsize self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((host, port)) self._sock.settimeout(timeout) @@ -36,38 +35,35 @@ def __str__(self): # Private API # ################### - def _consume_response(self): - """ - Fully consume the response iterator - """ - return "".join(self._consume_response_iter()) + def _read_bytes(self, num_bytes): + bytes_left = num_bytes + resp = '' + log.debug("About to read %d bytes from Kafka", num_bytes) + + while bytes_left: + data = self._sock.recv(bytes_left) + if data == '': + raise BufferUnderflowError("Not enough data to read this response") + bytes_left -= len(data) + log.debug("Read %d/%d bytes from Kafka", num_bytes - bytes_left, num_bytes) + resp += data + + return resp - def _consume_response_iter(self): + def _consume_response(self): """ This method handles the response header and error messages. It - then returns an iterator for the chunks of the response + then returns the response """ - log.debug("Handling response from Kafka") - + log.debug("Expecting response from Kafka") # Read the size off of the header - resp = self._sock.recv(4) - if resp == "": - self._raise_connection_error() - (size,) = struct.unpack('>i', resp) + resp = self._read_bytes(4) - log.debug("About to read %d bytes from Kafka", size) + (size,) = struct.unpack('>i', resp) # Read the remainder of the response - total = 0 - while total < size: - resp = self._sock.recv(self.bufsize) - log.debug("Read %d bytes from Kafka", len(resp)) - if resp == "": - raise BufferUnderflowError( - "Not enough data to read this response") - - total += len(resp) - yield resp + resp = self._read_bytes(size) + return str(resp) def _raise_connection_error(self): self._dirty = True diff --git a/kafka/consumer.py b/kafka/consumer.py index 57b5b9742..bead1dd21 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -22,6 +22,7 @@ FETCH_DEFAULT_BLOCK_TIMEOUT = 1 FETCH_MAX_WAIT_TIME = 100 FETCH_MIN_BYTES = 4096 +FETCH_BUFFER_SIZE_BYTES = 4096 class FetchContext(object): @@ -216,8 +217,10 @@ class SimpleConsumer(Consumer): def __init__(self, client, group, topic, auto_commit=True, partitions=None, auto_commit_every_n=AUTO_COMMIT_MSG_COUNT, auto_commit_every_t=AUTO_COMMIT_INTERVAL, - fetch_size_bytes=FETCH_MIN_BYTES): + fetch_size_bytes=FETCH_MIN_BYTES, + buffer_size=FETCH_BUFFER_SIZE_BYTES): + self.buffer_size = 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 @@ -364,7 +367,7 @@ def __iter_partition__(self, partition, offset): # use MaxBytes = client's bufsize since we're only # fetching one topic + partition req = FetchRequest( - self.topic, partition, offset, self.client.bufsize) + self.topic, partition, offset, self.buffer_size) (resp,) = self.client.send_fetch_request( [req], From 4d6bafae7ece327ccccc8f2e42ffef1eed061096 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 18 Dec 2013 17:54:17 -0800 Subject: [PATCH 042/109] Allow None timeout in FetchContext even if block is False --- kafka/consumer.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index bead1dd21..5d2775dde 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -33,10 +33,10 @@ def __init__(self, consumer, block, timeout): self.consumer = consumer self.block = block - if block and not timeout: - timeout = FETCH_DEFAULT_BLOCK_TIMEOUT - - self.timeout = timeout * 1000 + if block: + if not timeout: + timeout = FETCH_DEFAULT_BLOCK_TIMEOUT + self.timeout = timeout * 1000 def __enter__(self): """Set fetch values based on blocking status""" From 5dd8d81c9e47ee21c22945b90221c67baa7852b9 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 18 Dec 2013 17:56:29 -0800 Subject: [PATCH 043/109] Reset consumer fields to original values rather than defaults in FetchContext --- kafka/consumer.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 5d2775dde..600c8c77b 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -40,6 +40,8 @@ def __init__(self, consumer, block, timeout): def __enter__(self): """Set fetch values based on blocking status""" + self.orig_fetch_max_wait_time = self.consumer.fetch_max_wait_time + self.orig_fetch_min_bytes = self.consumer.fetch_min_bytes if self.block: self.consumer.fetch_max_wait_time = self.timeout self.consumer.fetch_min_bytes = 1 @@ -47,9 +49,9 @@ def __enter__(self): self.consumer.fetch_min_bytes = 0 def __exit__(self, type, value, traceback): - """Reset values to default""" - self.consumer.fetch_max_wait_time = FETCH_MAX_WAIT_TIME - self.consumer.fetch_min_bytes = FETCH_MIN_BYTES + """Reset values""" + self.consumer.fetch_max_wait_time = self.orig_fetch_max_wait_time + self.consumer.fetch_min_bytes = self.orig_fetch_min_bytes class Consumer(object): From 0c7cf2569e384fcdde67b86689d64bafbaed953f Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 18 Dec 2013 18:36:32 -0800 Subject: [PATCH 044/109] SimpleConsumer flow changes: * Combine partition fetch requests into a single request * Put the messages received in a queue and update offsets * Grab as many messages from the queue as requested * When the queue is empty, request more * timeout param for get_messages() is the actual timeout for getting those messages * Based on https://github.com/mumrah/kafka-python/pull/74 - don't increase min_bytes if the consumer fetch buffer size is too small. Notes: Change MultiProcessConsumer and _mp_consume() accordingly. Previously, when querying each partition separately, it was possible to block waiting for messages on partition 0 even if there are new ones in partition 1. These changes allow us to block while waiting for messages on all partitions, and reduce total number of kafka requests. Use Queue.Queue for single proc Queue instead of already imported multiprocessing.Queue because the latter doesn't seem to guarantee immediate availability of items after a put: >>> from multiprocessing import Queue >>> q = Queue() >>> q.put(1); q.get_nowait() Traceback (most recent call last): File "", line 1, in File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/multiprocessing/queues.py", line 152, in get_nowait return self.get(False) File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/multiprocessing/queues.py", line 134, in get raise Empty Queue.Empty --- kafka/consumer.py | 182 ++++++++++++++++++---------------------------- 1 file changed, 70 insertions(+), 112 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 600c8c77b..a5a3e26e8 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -3,8 +3,8 @@ import logging import time from threading import Lock -from multiprocessing import Process, Queue, Event, Value -from Queue import Empty +from multiprocessing import Process, Queue as MPQueue, Event, Value +from Queue import Empty, Queue from kafka.common import ( ErrorMapping, FetchRequest, @@ -227,6 +227,7 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, self.fetch_max_wait_time = FETCH_MAX_WAIT_TIME self.fetch_min_bytes = fetch_size_bytes self.fetch_started = defaultdict(bool) # defaults to false + self.queue = Queue(buffer_size) super(SimpleConsumer, self).__init__( client, group, topic, @@ -292,122 +293,75 @@ def get_messages(self, count=1, block=True, timeout=0.1): count: Indicates the maximum number of messages to be fetched block: If True, the API will block till some messages are fetched. - timeout: If None, and block=True, the API will block infinitely. - If >0, API will block for specified time (in seconds) + 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. """ messages = [] - iterator = self.__iter__() - - # HACK: This splits the timeout between available partitions if timeout: - timeout = timeout * 1.0 / len(self.offsets) + max_time = time.time() + timeout - with FetchContext(self, block, timeout): - while count > 0: - try: - messages.append(next(iterator)) - except StopIteration: - break + while count > 0 and (timeout is None or timeout > 0): + message = self.get_message(block, timeout) + if message: + messages.append(message) count -= 1 + else: + # Ran out of messages for the last request. If we're not blocking, break. + if not block: + break + if timeout: + timeout = max_time - time.time() return messages - def __iter__(self): - """ - Create an iterate per partition. Iterate through them calling next() - until they are all exhausted. - """ - iters = {} - for partition, offset in self.offsets.items(): - iters[partition] = self.__iter_partition__(partition, offset) - - if len(iters) == 0: - return - - while True: - if len(iters) == 0: - break - - for partition, it in iters.items(): - try: - if self.partition_info: - yield (partition, it.next()) - else: - yield it.next() - except StopIteration: - log.debug("Done iterating over partition %s" % partition) - del iters[partition] - - # skip auto-commit since we didn't yield anything - continue - - # Count, check and commit messages if necessary - self.count_since_commit += 1 - self._auto_commit() - - def __iter_partition__(self, partition, offset): - """ - Iterate over the messages in a partition. Create a FetchRequest - to get back a batch of messages, yield them one at a time. - After a batch is exhausted, start a new batch unless we've reached - the end of this partition. - """ - - # The offset that is stored in the consumer is the offset that - # we have consumed. In subsequent iterations, we are supposed to - # fetch the next message (that is from the next offset) - # However, for the 0th message, the offset should be as-is. - # An OffsetFetchRequest to Kafka gives 0 for a new queue. This is - # problematic, since 0 is offset of a message which we have not yet - # consumed. - if self.fetch_started[partition]: - offset += 1 - - fetch_size = self.fetch_min_bytes + def get_message(self, block=True, timeout=0.1): + if self.queue.empty(): + with FetchContext(self, block, timeout): + self._fetch() + try: + return self.queue.get_nowait() + except Empty: + return None + def __iter__(self): while True: - # use MaxBytes = client's bufsize since we're only - # fetching one topic + partition - req = FetchRequest( - self.topic, partition, offset, self.buffer_size) - - (resp,) = self.client.send_fetch_request( - [req], - max_wait_time=self.fetch_max_wait_time, - min_bytes=fetch_size) - - assert resp.topic == self.topic - assert resp.partition == partition + message = self.get_message(True, 100) + if message: + yield message + else: + # In case we did not receive any message, give up the CPU for + # a while before we try again + time.sleep(0.1) - next_offset = None + def _fetch(self): + requests = [] + partitions = self.offsets.keys() + for partition in partitions: + requests.append(FetchRequest(self.topic, partition, self.offsets[partition], self.buffer_size)) + responses = self.client.send_fetch_request( + requests, + max_wait_time=int(self.fetch_max_wait_time), + min_bytes=self.fetch_min_bytes) + for resp in responses: + partition = resp.partition try: for message in resp.messages: - next_offset = message.offset - - # update the offset before the message is yielded. This - # is so that the consumer state is not lost in certain - # cases. - # - # For eg: the message is yielded and consumed by the - # caller, but the caller does not come back into the - # generator again. The message will be consumed but the - # status will not be updated in the consumer - self.fetch_started[partition] = True - self.offsets[partition] = message.offset - yield message + self.offsets[partition] = message.offset + 1 + # Count, check and commit messages if necessary + self.count_since_commit += 1 + self._auto_commit() + if self.partition_info: + self.queue.put((partition, message)) + else: + self.queue.put(message) except ConsumerFetchSizeTooSmall, e: - fetch_size *= 1.5 - log.warn( - "Fetch size too small, increasing to %d (1.5x) and retry", - fetch_size) - continue + self.buffer_size *= 2 + log.warn("Fetch size too small, increasing to %d (2x) and retry", self.buffer_size) except ConsumerNoMoreData, e: log.debug("Iteration was ended by %r", e) - - if next_offset is None: - break - else: - offset = next_offset + 1 + except StopIteration: + # Stop iterating through this partition + log.debug("Done iterating over partition %s" % partition) def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): @@ -446,8 +400,9 @@ def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): # indicates a specific number of messages, follow that advice count = 0 - for partition, message in consumer: - queue.put((partition, message)) + message = consumer.get_message() + if message: + queue.put(message) count += 1 # We have reached the required size. The controller might have @@ -457,11 +412,10 @@ def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): # can reset the 'start' event if count == size.value: pause.wait() - break - # In case we did not receive any message, give up the CPU for - # a while before we try again - if count == 0: + else: + # In case we did not receive any message, give up the CPU for + # a while before we try again time.sleep(0.1) consumer.stop() @@ -507,7 +461,7 @@ def __init__(self, client, group, topic, auto_commit=True, # Variables for managing and controlling the data flow from # consumer child process to master - self.queue = Queue(1024) # Child consumers dump messages into this + 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 @@ -589,8 +543,8 @@ def get_messages(self, count=1, block=True, timeout=10): count: Indicates the maximum number of messages to be fetched block: If True, the API will block till some messages are fetched. - timeout: If None, and block=True, the API will block infinitely. - If >0, API will block for specified time (in seconds) + 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. """ messages = [] @@ -601,7 +555,10 @@ def get_messages(self, count=1, block=True, timeout=10): self.size.value = count self.pause.clear() - while count > 0: + if timeout: + max_time = time.time() + timeout + + while count > 0 and (timeout is None or timeout > 0): # Trigger consumption only if the queue is empty # By doing this, we will ensure that consumers do not # go into overdrive and keep consuming thousands of @@ -621,6 +578,7 @@ def get_messages(self, count=1, block=True, timeout=10): self.count_since_commit += 1 self._auto_commit() count -= 1 + timeout = max_time - time.time() self.size.value = 0 self.start.clear() From b68523fd28dec0741a15cbb3b5a78d954c559d2d Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 19 Dec 2013 11:36:15 -0800 Subject: [PATCH 045/109] Remove SimpleConsumer queue size limit since it can cause the iterator to block forever if it's reached. --- kafka/consumer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index a5a3e26e8..b27b84526 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -227,7 +227,7 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, self.fetch_max_wait_time = FETCH_MAX_WAIT_TIME self.fetch_min_bytes = fetch_size_bytes self.fetch_started = defaultdict(bool) # defaults to false - self.queue = Queue(buffer_size) + self.queue = Queue() super(SimpleConsumer, self).__init__( client, group, topic, From 450faeb328fe085a35c70e0dcbf93cbd44277362 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 19 Dec 2013 13:19:40 -0800 Subject: [PATCH 046/109] Add buffer_size param description to docstring --- kafka/consumer.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index b27b84526..7e53f083f 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -207,8 +207,9 @@ class SimpleConsumer(Consumer): before a commit auto_commit_every_t: default 5000. How much time (in milliseconds) to wait before commit - fetch_size_bytes: number of bytes to request in a FetchRequest + buffer_size: initial number of bytes to tell kafka we have + available. This will double every time it's not enough Auto commit details: If both auto_commit_every_n and auto_commit_every_t are set, they will From dc4198bddc9f721ef18b41d8d7714bfa968eec7d Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 19 Dec 2013 13:24:28 -0800 Subject: [PATCH 047/109] Add iter_timeout option to SimpleConsumer. If not None, it causes the iterator to exit when reached. Also put constant timeout values in pre-defined constants --- kafka/consumer.py | 28 ++++++++++++++++++++++------ 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 7e53f083f..5fa7332e1 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -24,6 +24,9 @@ FETCH_MIN_BYTES = 4096 FETCH_BUFFER_SIZE_BYTES = 4096 +ITER_TIMEOUT_SECONDS = 60 +NO_MESSAGES_WAIT_TIME_SECONDS = 0.1 + class FetchContext(object): """ @@ -210,6 +213,9 @@ class SimpleConsumer(Consumer): fetch_size_bytes: number of bytes to request in a FetchRequest buffer_size: initial number of bytes to tell kafka we have available. This will double every time it's not enough + 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 commit details: If both auto_commit_every_n and auto_commit_every_t are set, they will @@ -221,13 +227,15 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, auto_commit_every_n=AUTO_COMMIT_MSG_COUNT, auto_commit_every_t=AUTO_COMMIT_INTERVAL, fetch_size_bytes=FETCH_MIN_BYTES, - buffer_size=FETCH_BUFFER_SIZE_BYTES): + buffer_size=FETCH_BUFFER_SIZE_BYTES, + iter_timeout=None): self.buffer_size = 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_started = defaultdict(bool) # defaults to false + self.iter_timeout = iter_timeout self.queue = Queue() super(SimpleConsumer, self).__init__( @@ -325,14 +333,22 @@ def get_message(self, block=True, timeout=0.1): return None def __iter__(self): + if self.iter_timeout is None: + timeout = ITER_TIMEOUT_SECONDS + else: + timeout = self.iter_timeout + while True: - message = self.get_message(True, 100) + message = self.get_message(True, timeout) if message: yield message + elif self.iter_timeout is None: + # We did not receive any message yet but we don't have a + # timeout, so give up the CPU for a while before trying again + time.sleep(NO_MESSAGES_WAIT_TIME_SECONDS) else: - # In case we did not receive any message, give up the CPU for - # a while before we try again - time.sleep(0.1) + # Timed out waiting for a message + break def _fetch(self): requests = [] @@ -417,7 +433,7 @@ def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): else: # In case we did not receive any message, give up the CPU for # a while before we try again - time.sleep(0.1) + time.sleep(NO_MESSAGES_WAIT_TIME_SECONDS) consumer.stop() From c1ba5101b7a54382b2a68b23ba777785104e9877 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 19 Dec 2013 13:35:53 -0800 Subject: [PATCH 048/109] Add comments and maintain 80 character line limit --- kafka/consumer.py | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 5fa7332e1..ff08da498 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -302,8 +302,9 @@ def get_messages(self, count=1, block=True, timeout=0.1): 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. + 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. """ messages = [] if timeout: @@ -315,16 +316,20 @@ def get_messages(self, count=1, block=True, timeout=0.1): messages.append(message) count -= 1 else: - # Ran out of messages for the last request. If we're not blocking, break. + # Ran out of messages for the last request. if not block: + # If we're not blocking, break. break if timeout: + # If we're blocking and have a timeout, reduce it to the + # appropriate value timeout = max_time - time.time() return messages def get_message(self, block=True, timeout=0.1): if self.queue.empty(): + # We're out of messages, go grab some more. with FetchContext(self, block, timeout): self._fetch() try: @@ -351,29 +356,39 @@ def __iter__(self): break def _fetch(self): + # Create fetch request payloads for all the partitions requests = [] partitions = self.offsets.keys() for partition in partitions: - requests.append(FetchRequest(self.topic, partition, self.offsets[partition], self.buffer_size)) + requests.append(FetchRequest(self.topic, partition, + self.offsets[partition], + self.buffer_size)) + # Send request responses = self.client.send_fetch_request( requests, max_wait_time=int(self.fetch_max_wait_time), min_bytes=self.fetch_min_bytes) + for resp in responses: partition = resp.partition try: for message in resp.messages: + # Update partition offset self.offsets[partition] = message.offset + 1 + # Count, check and commit messages if necessary self.count_since_commit += 1 self._auto_commit() + + # Put the message in our queue if self.partition_info: self.queue.put((partition, message)) else: self.queue.put(message) except ConsumerFetchSizeTooSmall, e: self.buffer_size *= 2 - log.warn("Fetch size too small, increasing to %d (2x) and retry", self.buffer_size) + log.warn("Fetch size too small, increase to %d (2x) and retry", + self.buffer_size) except ConsumerNoMoreData, e: log.debug("Iteration was ended by %r", e) except StopIteration: @@ -560,8 +575,9 @@ def get_messages(self, count=1, block=True, timeout=10): 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. + 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. """ messages = [] From 93b6579a0f5192b733e403acc38fc5ae952d4a08 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 2 Jan 2014 18:10:57 -0800 Subject: [PATCH 049/109] Add and fix comments to protocol.py --- kafka/protocol.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/kafka/protocol.py b/kafka/protocol.py index 612acf63f..74a0dce0e 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -121,7 +121,7 @@ def _decode_message_set_iter(cls, data): except BufferUnderflowError: if read_message is False: # If we get a partial read of a message, but haven't - # yielded anyhting there's a problem + # yielded anything there's a problem raise ConsumerFetchSizeTooSmall() else: raise StopIteration() @@ -171,7 +171,7 @@ def encode_produce_request(cls, client_id, correlation_id, Params ====== client_id: string - correlation_id: string + correlation_id: int payloads: list of ProduceRequest acks: How "acky" you want the request to be 0: immediate response @@ -231,7 +231,7 @@ def encode_fetch_request(cls, client_id, correlation_id, payloads=None, Params ====== client_id: string - correlation_id: string + correlation_id: int payloads: list of FetchRequest max_wait_time: int, how long to block waiting on min_bytes of data min_bytes: int, the minimum number of bytes to accumulate before @@ -338,7 +338,7 @@ def encode_metadata_request(cls, client_id, correlation_id, topics=None): Params ====== client_id: string - correlation_id: string + correlation_id: int topics: list of strings """ topics = [] if topics is None else topics @@ -376,12 +376,16 @@ def decode_metadata_response(cls, data): topic_metadata = {} for i in range(num_topics): + # NOTE: topic_error is discarded. Should probably be returned with + # the topic metadata. ((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): + # NOTE: partition_error_code is discarded. Should probably be + # returned with the partition metadata. ((partition_error_code, partition, leader, numReplicas), cur) = \ relative_unpack('>hiii', data, cur) @@ -408,7 +412,7 @@ def encode_offset_commit_request(cls, client_id, correlation_id, Params ====== client_id: string - correlation_id: string + correlation_id: int group: string, the consumer group you are committing offsets for payloads: list of OffsetCommitRequest """ @@ -459,7 +463,7 @@ def encode_offset_fetch_request(cls, client_id, correlation_id, Params ====== client_id: string - correlation_id: string + correlation_id: int group: string, the consumer group you are fetching offsets for payloads: list of OffsetFetchRequest """ From 009ed92894b03a95a473359e64c5499665697b10 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 2 Jan 2014 18:14:44 -0800 Subject: [PATCH 050/109] Add note about questionable error handling while decoding messages. Will remove once any error handling issues are resolved. --- kafka/protocol.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/kafka/protocol.py b/kafka/protocol.py index 74a0dce0e..54b8eeeeb 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -119,6 +119,14 @@ def _decode_message_set_iter(cls, data): read_message = True yield OffsetAndMessage(offset, message) except BufferUnderflowError: + # NOTE: Not sure this is correct error handling: + # Is it possible to get a BUE if the message set is somewhere + # in the middle of the fetch response? If so, we probably have + # an issue that's not fetch size too small. + # Aren't we ignoring errors if we fail to unpack data by + # raising StopIteration()? + # If _decode_message() raises a ChecksumError, couldn't that + # also be due to the fetch size being too small? if read_message is False: # If we get a partial read of a message, but haven't # yielded anything there's a problem From b6b1ba09402dbe7fe13b9c1e806faa02db42ebdb Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 2 Jan 2014 18:16:41 -0800 Subject: [PATCH 051/109] Fix unit tests. This is pretty much a rewrite. The tests that involve offset requests/responses are not implemented since that API is not supported in kafka 0.8 yet. Only kafka.codec and kafka.protocol are currently tested, so there is more work to be done here. --- test/test_unit.py | 464 +++++++++++++++++++++++++++++++--------------- 1 file changed, 314 insertions(+), 150 deletions(-) diff --git a/test/test_unit.py b/test/test_unit.py index 3f3af66ac..93d88a151 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -3,13 +3,22 @@ import struct import unittest -from kafka.client import KafkaClient -from kafka.common import ProduceRequest, FetchRequest +from kafka.common import ( + ProduceRequest, FetchRequest, Message, ChecksumError, + ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, + OffsetAndMessage, BrokerMetadata, PartitionMetadata +) from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, snappy_encode, snappy_decode ) +from kafka.protocol import ( + create_gzip_message, + create_message, + create_snappy_message, + KafkaProtocol +) ITERATIONS = 1000 STRLEN = 100 @@ -20,16 +29,13 @@ def random_string(): class TestPackage(unittest.TestCase): - @unittest.expectedFailure + def test_top_level_namespace(self): import kafka as kafka1 self.assertEquals(kafka1.KafkaClient.__name__, "KafkaClient") - self.assertEquals(kafka1.gzip_encode.__name__, "gzip_encode") - self.assertEquals(kafka1.snappy_encode.__name__, "snappy_encode") self.assertEquals(kafka1.client.__name__, "kafka.client") self.assertEquals(kafka1.codec.__name__, "kafka.codec") - @unittest.expectedFailure def test_submodule_namespace(self): import kafka.client as client1 self.assertEquals(client1.__name__, "kafka.client") @@ -48,19 +54,8 @@ def test_submodule_namespace(self): from kafka import KafkaClient as KafkaClient2 self.assertEquals(KafkaClient2.__name__, "KafkaClient") - from kafka import gzip_encode as gzip_encode2 - self.assertEquals(gzip_encode2.__name__, "gzip_encode") - - from kafka import snappy_encode as snappy_encode2 - self.assertEquals(snappy_encode2.__name__, "snappy_encode") - - -class TestMisc(unittest.TestCase): - @unittest.expectedFailure - def test_length_prefix(self): - for i in xrange(ITERATIONS): - s1 = random_string() - self.assertEquals(struct.unpack('>i', s2[0:4])[0], len(s1)) + from kafka.codec import snappy_encode + self.assertEquals(snappy_encode.__name__, "snappy_encode") class TestCodec(unittest.TestCase): @@ -81,140 +76,309 @@ def test_snappy(self): self.assertEquals(s1, s2) -# XXX(sandello): These really should be protocol tests. -class TestMessage(unittest.TestCase): - @unittest.expectedFailure - def test_create(self): - msg = KafkaClient.create_message("testing") - self.assertEquals(msg.payload, "testing") - self.assertEquals(msg.magic, 1) - self.assertEquals(msg.attributes, 0) - self.assertEquals(msg.crc, -386704890) +class TestProtocol(unittest.TestCase): + + def test_create_message(self): + payload = "test" + key = "key" + msg = create_message(payload, key) + self.assertEqual(msg.magic, 0) + self.assertEqual(msg.attributes, 0) + self.assertEqual(msg.key, key) + self.assertEqual(msg.value, payload) - @unittest.expectedFailure def test_create_gzip(self): - msg = KafkaClient.create_gzip_message("testing") - self.assertEquals(msg.magic, 1) - self.assertEquals(msg.attributes, 1) - # Can't check the crc or payload for gzip since it's non-deterministic - (messages, _) = KafkaClient.read_message_set(gzip_decode(msg.payload)) - inner = messages[0] - self.assertEquals(inner.magic, 1) - self.assertEquals(inner.attributes, 0) - self.assertEquals(inner.payload, "testing") - self.assertEquals(inner.crc, -386704890) - - @unittest.expectedFailure + payloads = ["v1", "v2"] + msg = create_gzip_message(payloads) + self.assertEqual(msg.magic, 0) + self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK & + KafkaProtocol.CODEC_GZIP) + self.assertEqual(msg.key, None) + # Need to decode to check since gzipped payload is non-deterministic + decoded = gzip_decode(msg.value) + expect = ("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x10L\x9f[\xc2" + "\x00\x00\xff\xff\xff\xff\x00\x00\x00\x02v1\x00\x00\x00\x00" + "\x00\x00\x00\x00\x00\x00\x00\x10\xd5\x96\nx\x00\x00\xff\xff" + "\xff\xff\x00\x00\x00\x02v2") + self.assertEqual(decoded, expect) + def test_create_snappy(self): - msg = KafkaClient.create_snappy_message("testing") - self.assertEquals(msg.magic, 1) - self.assertEquals(msg.attributes, 2) - self.assertEquals(msg.crc, -62350868) - (messages, _) = KafkaClient.read_message_set(snappy_decode(msg.payload)) - inner = messages[0] - self.assertEquals(inner.magic, 1) - self.assertEquals(inner.attributes, 0) - self.assertEquals(inner.payload, "testing") - self.assertEquals(inner.crc, -386704890) - - @unittest.expectedFailure - def test_message_simple(self): - msg = KafkaClient.create_message("testing") - enc = KafkaClient.encode_message(msg) - expect = "\x00\x00\x00\r\x01\x00\xe8\xf3Z\x06testing" - self.assertEquals(enc, expect) - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), 1) - self.assertEquals(messages[0], msg) - - @unittest.expectedFailure - def test_message_list(self): - msgs = [ - KafkaClient.create_message("one"), - KafkaClient.create_message("two"), - KafkaClient.create_message("three") - ] - enc = KafkaClient.encode_message_set(msgs) - expect = ("\x00\x00\x00\t\x01\x00zl\x86\xf1one\x00\x00\x00\t\x01\x00\x11" - "\xca\x8aftwo\x00\x00\x00\x0b\x01\x00F\xc5\xd8\xf5three") - self.assertEquals(enc, expect) - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), 3) - self.assertEquals(messages[0].payload, "one") - self.assertEquals(messages[1].payload, "two") - self.assertEquals(messages[2].payload, "three") - - @unittest.expectedFailure - def test_message_gzip(self): - msg = KafkaClient.create_gzip_message("one", "two", "three") - enc = KafkaClient.encode_message(msg) - # Can't check the bytes directly since Gzip is non-deterministic - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), 3) - self.assertEquals(messages[0].payload, "one") - self.assertEquals(messages[1].payload, "two") - self.assertEquals(messages[2].payload, "three") - - @unittest.expectedFailure - def test_message_snappy(self): - msg = KafkaClient.create_snappy_message("one", "two", "three") - enc = KafkaClient.encode_message(msg) - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), 3) - self.assertEquals(messages[0].payload, "one") - self.assertEquals(messages[1].payload, "two") - self.assertEquals(messages[2].payload, "three") - - @unittest.expectedFailure - def test_message_simple_random(self): - for i in xrange(ITERATIONS): - n = random.randint(0, 10) - msgs = [KafkaClient.create_message(random_string()) for j in range(n)] - enc = KafkaClient.encode_message_set(msgs) - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), n) - for j in range(n): - self.assertEquals(messages[j], msgs[j]) - - @unittest.expectedFailure - def test_message_gzip_random(self): - for i in xrange(ITERATIONS): - n = random.randint(1, 10) - strings = [random_string() for j in range(n)] - msg = KafkaClient.create_gzip_message(*strings) - enc = KafkaClient.encode_message(msg) - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), n) - for j in range(n): - self.assertEquals(messages[j].payload, strings[j]) - - @unittest.expectedFailure - def test_message_snappy_random(self): - for i in xrange(ITERATIONS): - n = random.randint(1, 10) - strings = [random_string() for j in range(n)] - msg = KafkaClient.create_snappy_message(*strings) - enc = KafkaClient.encode_message(msg) - (messages, read) = KafkaClient.read_message_set(enc) - self.assertEquals(len(messages), n) - for j in range(n): - self.assertEquals(messages[j].payload, strings[j]) - - -class TestRequests(unittest.TestCase): - @unittest.expectedFailure - def test_produce_request(self): - req = ProduceRequest("my-topic", 0, [KafkaClient.create_message("testing")]) - enc = KafkaClient.encode_produce_request(req) - expect = "\x00\x00\x00\x08my-topic\x00\x00\x00\x00\x00\x00\x00\x11\x00\x00\x00\r\x01\x00\xe8\xf3Z\x06testing" - self.assertEquals(enc, expect) - - @unittest.expectedFailure - def test_fetch_request(self): - req = FetchRequest("my-topic", 0, 0, 1024) - enc = KafkaClient.encode_fetch_request(req) - expect = "\x00\x01\x00\x08my-topic\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x04\x00" - self.assertEquals(enc, expect) + payloads = ["v1", "v2"] + msg = create_snappy_message(payloads) + self.assertEqual(msg.magic, 0) + self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK & + KafkaProtocol.CODEC_SNAPPY) + self.assertEqual(msg.key, None) + expect = ("8\x00\x00\x19\x01@\x10L\x9f[\xc2\x00\x00\xff\xff\xff\xff" + "\x00\x00\x00\x02v1\x19\x1bD\x00\x10\xd5\x96\nx\x00\x00\xff" + "\xff\xff\xff\x00\x00\x00\x02v2") + self.assertEqual(msg.value, expect) + + def test_encode_message_header(self): + expect = '\x00\n\x00\x00\x00\x00\x00\x04\x00\x07client3' + encoded = KafkaProtocol._encode_message_header("client3", 4, 10) + self.assertEqual(encoded, expect) + + def test_encode_message(self): + message = create_message("test", "key") + encoded = KafkaProtocol._encode_message(message) + expect = "\xaa\xf1\x8f[\x00\x00\x00\x00\x00\x03key\x00\x00\x00\x04test" + self.assertEqual(encoded, expect) + + def test_encode_message_failure(self): + self.assertRaises(Exception, KafkaProtocol._encode_message, + Message(1, 0, "key", "test")) + + def test_encode_message_set(self): + message_set = [create_message("v1", "k1"), create_message("v2", "k2")] + encoded = KafkaProtocol._encode_message_set(message_set) + expect = ("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x12W\xe7In\x00" + "\x00\x00\x00\x00\x02k1\x00\x00\x00\x02v1\x00\x00\x00\x00" + "\x00\x00\x00\x00\x00\x00\x00\x12\xff\x06\x02I\x00\x00\x00" + "\x00\x00\x02k2\x00\x00\x00\x02v2") + self.assertEqual(encoded, expect) + + def test_decode_message(self): + encoded = "\xaa\xf1\x8f[\x00\x00\x00\x00\x00\x03key\x00\x00\x00\x04test" + offset = 10 + (returned_offset, decoded_message) = \ + list(KafkaProtocol._decode_message(encoded, offset))[0] + self.assertEqual(returned_offset, offset) + self.assertEqual(decoded_message, create_message("test", "key")) + + def test_decode_message_set(self): + encoded = ('\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x10L\x9f[\xc2' + '\x00\x00\xff\xff\xff\xff\x00\x00\x00\x02v1\x00\x00\x00\x00' + '\x00\x00\x00\x00\x00\x00\x00\x10\xd5\x96\nx\x00\x00\xff' + '\xff\xff\xff\x00\x00\x00\x02v2') + iter = KafkaProtocol._decode_message_set_iter(encoded) + decoded = list(iter) + self.assertEqual(len(decoded), 2) + (returned_offset1, decoded_message1) = decoded[0] + self.assertEqual(returned_offset1, 0) + self.assertEqual(decoded_message1, create_message("v1")) + (returned_offset2, decoded_message2) = decoded[1] + self.assertEqual(returned_offset2, 0) + self.assertEqual(decoded_message2, create_message("v2")) + + def test_decode_message_gzip(self): + gzip_encoded = ('\xc0\x11\xb2\xf0\x00\x01\xff\xff\xff\xff\x00\x00\x000' + '\x1f\x8b\x08\x00\xa1\xc1\xc5R\x02\xffc`\x80\x03\x01' + '\x9f\xf9\xd1\x87\x18\x18\xfe\x03\x01\x90\xc7Tf\xc8' + '\x80$wu\x1aW\x05\x92\x9c\x11\x00z\xc0h\x888\x00\x00' + '\x00') + offset = 11 + decoded = list(KafkaProtocol._decode_message(gzip_encoded, offset)) + self.assertEqual(len(decoded), 2) + (returned_offset1, decoded_message1) = decoded[0] + self.assertEqual(returned_offset1, 0) + self.assertEqual(decoded_message1, create_message("v1")) + (returned_offset2, decoded_message2) = decoded[1] + self.assertEqual(returned_offset2, 0) + self.assertEqual(decoded_message2, create_message("v2")) + + def test_decode_message_snappy(self): + snappy_encoded = ('\xec\x80\xa1\x95\x00\x02\xff\xff\xff\xff\x00\x00' + '\x00,8\x00\x00\x19\x01@\x10L\x9f[\xc2\x00\x00\xff' + '\xff\xff\xff\x00\x00\x00\x02v1\x19\x1bD\x00\x10\xd5' + '\x96\nx\x00\x00\xff\xff\xff\xff\x00\x00\x00\x02v2') + offset = 11 + decoded = list(KafkaProtocol._decode_message(snappy_encoded, offset)) + self.assertEqual(len(decoded), 2) + (returned_offset1, decoded_message1) = decoded[0] + self.assertEqual(returned_offset1, 0) + self.assertEqual(decoded_message1, create_message("v1")) + (returned_offset2, decoded_message2) = decoded[1] + self.assertEqual(returned_offset2, 0) + self.assertEqual(decoded_message2, create_message("v2")) + + def test_decode_message_checksum_error(self): + invalid_encoded_message = "This is not a valid encoded message" + iter = KafkaProtocol._decode_message(invalid_encoded_message, 0) + self.assertRaises(ChecksumError, list, iter) + + # NOTE: The error handling in _decode_message_set_iter() is questionable. + # If it's modified, the next two tests might need to be fixed. + def test_decode_message_set_fetch_size_too_small(self): + iter = KafkaProtocol._decode_message_set_iter('a') + self.assertRaises(ConsumerFetchSizeTooSmall, list, iter) + + def test_decode_message_set_stop_iteration(self): + encoded = ('\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x10L\x9f[\xc2' + '\x00\x00\xff\xff\xff\xff\x00\x00\x00\x02v1\x00\x00\x00\x00' + '\x00\x00\x00\x00\x00\x00\x00\x10\xd5\x96\nx\x00\x00\xff' + '\xff\xff\xff\x00\x00\x00\x02v2') + iter = KafkaProtocol._decode_message_set_iter(encoded + "@#$%(Y!") + decoded = list(iter) + self.assertEqual(len(decoded), 2) + (returned_offset1, decoded_message1) = decoded[0] + self.assertEqual(returned_offset1, 0) + self.assertEqual(decoded_message1, create_message("v1")) + (returned_offset2, decoded_message2) = decoded[1] + self.assertEqual(returned_offset2, 0) + self.assertEqual(decoded_message2, create_message("v2")) + + def test_encode_produce_request(self): + requests = [ProduceRequest("topic1", 0, [create_message("a"), + create_message("b")]), + ProduceRequest("topic2", 1, [create_message("c")])] + expect = ('\x00\x00\x00\x94\x00\x00\x00\x00\x00\x00\x00\x02\x00\x07' + 'client1\x00\x02\x00\x00\x00d\x00\x00\x00\x02\x00\x06topic1' + '\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x006\x00\x00\x00' + '\x00\x00\x00\x00\x00\x00\x00\x00\x0fQ\xdf:2\x00\x00\xff\xff' + '\xff\xff\x00\x00\x00\x01a\x00\x00\x00\x00\x00\x00\x00\x00' + '\x00\x00\x00\x0f\xc8\xd6k\x88\x00\x00\xff\xff\xff\xff\x00' + '\x00\x00\x01b\x00\x06topic2\x00\x00\x00\x01\x00\x00\x00\x01' + '\x00\x00\x00\x1b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00' + '\x00\x0f\xbf\xd1[\x1e\x00\x00\xff\xff\xff\xff\x00\x00\x00' + '\x01c') + encoded = KafkaProtocol.encode_produce_request("client1", 2, requests, + 2, 100) + self.assertEqual(encoded, expect) + + def test_decode_produce_response(self): + t1 = "topic1" + t2 = "topic2" + encoded = struct.pack('>iih%dsiihqihqh%dsiihq' % (len(t1), len(t2)), + 2, 2, len(t1), t1, 2, 0, 0, 10L, 1, 1, 20L, + len(t2), t2, 1, 0, 0, 30L) + responses = list(KafkaProtocol.decode_produce_response(encoded)) + self.assertEqual(responses, + [ProduceResponse(t1, 0, 0, 10L), + ProduceResponse(t1, 1, 1, 20L), + ProduceResponse(t2, 0, 0, 30L)]) + + def test_encode_fetch_request(self): + requests = [FetchRequest("topic1", 0, 10, 1024), + FetchRequest("topic2", 1, 20, 100)] + expect = ('\x00\x00\x00Y\x00\x01\x00\x00\x00\x00\x00\x03\x00\x07' + 'client1\xff\xff\xff\xff\x00\x00\x00\x02\x00\x00\x00d\x00' + '\x00\x00\x02\x00\x06topic1\x00\x00\x00\x01\x00\x00\x00\x00' + '\x00\x00\x00\x00\x00\x00\x00\n\x00\x00\x04\x00\x00\x06' + 'topic2\x00\x00\x00\x01\x00\x00\x00\x01\x00\x00\x00\x00\x00' + '\x00\x00\x14\x00\x00\x00d') + encoded = KafkaProtocol.encode_fetch_request("client1", 3, requests, 2, + 100) + self.assertEqual(encoded, expect) + + def test_decode_fetch_response(self): + t1 = "topic1" + t2 = "topic2" + msgs = map(create_message, ["message1", "hi", "boo", "foo", "so fun!"]) + ms1 = KafkaProtocol._encode_message_set([msgs[0], msgs[1]]) + ms2 = KafkaProtocol._encode_message_set([msgs[2]]) + ms3 = KafkaProtocol._encode_message_set([msgs[3], msgs[4]]) + + encoded = struct.pack('>iih%dsiihqi%dsihqi%dsh%dsiihqi%ds' % + (len(t1), len(ms1), len(ms2), len(t2), len(ms3)), + 4, 2, len(t1), t1, 2, 0, 0, 10, len(ms1), ms1, 1, + 1, 20, len(ms2), ms2, len(t2), t2, 1, 0, 0, 30, + len(ms3), ms3) + + responses = list(KafkaProtocol.decode_fetch_response(encoded)) + def expand_messages(response): + return FetchResponse(response.topic, response.partition, + response.error, response.highwaterMark, + list(response.messages)) + + expanded_responses = map(expand_messages, responses) + expect = [FetchResponse(t1, 0, 0, 10, [OffsetAndMessage(0, msgs[0]), + OffsetAndMessage(0, msgs[1])]), + FetchResponse(t1, 1, 1, 20, [OffsetAndMessage(0, msgs[2])]), + FetchResponse(t2, 0, 0, 30, [OffsetAndMessage(0, msgs[3]), + OffsetAndMessage(0, msgs[4])])] + self.assertEqual(expanded_responses, expect) + + def test_encode_metadata_request_no_topics(self): + encoded = KafkaProtocol.encode_metadata_request("cid", 4) + self.assertEqual(encoded, '\x00\x00\x00\x11\x00\x03\x00\x00\x00\x00' + '\x00\x04\x00\x03cid\x00\x00\x00\x00') + + def test_encode_metadata_request_with_topics(self): + encoded = KafkaProtocol.encode_metadata_request("cid", 4, ["t1", "t2"]) + self.assertEqual(encoded, '\x00\x00\x00\x19\x00\x03\x00\x00\x00\x00' + '\x00\x04\x00\x03cid\x00\x00\x00\x02\x00\x02' + 't1\x00\x02t2') + + def _create_encoded_metadata_response(self, broker_data, topic_data, + topic_errors, partition_errors): + encoded = struct.pack('>ii', 3, len(broker_data)) + for node_id, broker in broker_data.iteritems(): + encoded += struct.pack('>ih%dsi' % len(broker.host), node_id, + len(broker.host), broker.host, broker.port) + + encoded += struct.pack('>i', len(topic_data)) + for topic, partitions in topic_data.iteritems(): + encoded += struct.pack('>hh%dsi' % len(topic), topic_errors[topic], + len(topic), topic, len(partitions)) + for partition, metadata in partitions.iteritems(): + encoded += struct.pack('>hiii', + partition_errors[(topic, partition)], + partition, metadata.leader, + len(metadata.replicas)) + if len(metadata.replicas) > 0: + encoded += struct.pack('>%di' % len(metadata.replicas), + *metadata.replicas) + + encoded += struct.pack('>i', len(metadata.isr)) + if len(metadata.isr) > 0: + encoded += struct.pack('>%di' % len(metadata.isr), + *metadata.isr) + + return encoded + + def test_decode_metadata_response(self): + node_brokers = { + 0: BrokerMetadata(0, "brokers1.kafka.rdio.com", 1000), + 1: BrokerMetadata(1, "brokers1.kafka.rdio.com", 1001), + 3: BrokerMetadata(3, "brokers2.kafka.rdio.com", 1000) + } + topic_partitions = { + "topic1": { + 0: PartitionMetadata("topic1", 0, 1, (0, 2), (2,)), + 1: PartitionMetadata("topic1", 1, 3, (0, 1), (0, 1)) + }, + "topic2": { + 0: PartitionMetadata("topic2", 0, 0, (), ()) + } + } + topic_errors = {"topic1": 0, "topic2": 1} + partition_errors = { + ("topic1", 0): 0, + ("topic1", 1): 1, + ("topic2", 0): 0 + } + encoded = self._create_encoded_metadata_response(node_brokers, + topic_partitions, + topic_errors, + partition_errors) + decoded = KafkaProtocol.decode_metadata_response(encoded) + self.assertEqual(decoded, (node_brokers, topic_partitions)) + + @unittest.skip("Not Implemented") + def test_encode_offset_request(self): + pass + + @unittest.skip("Not Implemented") + def test_decode_offset_response(self): + pass + + + @unittest.skip("Not Implemented") + def test_encode_offset_commit_request(self): + pass + + @unittest.skip("Not Implemented") + def test_decode_offset_commit_response(self): + pass + + @unittest.skip("Not Implemented") + def test_encode_offset_fetch_request(self): + pass + + @unittest.skip("Not Implemented") + def test_decode_offset_fetch_response(self): + pass if __name__ == '__main__': From 99b561d95f558652583f79aa4ac5dfa4c5e1b854 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 10:57:29 -0800 Subject: [PATCH 052/109] Style fix for imports --- test/test_unit.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/test/test_unit.py b/test/test_unit.py index 93d88a151..08fef9c2a 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -9,15 +9,11 @@ OffsetAndMessage, BrokerMetadata, PartitionMetadata ) from kafka.codec import ( - has_gzip, has_snappy, - gzip_encode, gzip_decode, + has_gzip, has_snappy, gzip_encode, gzip_decode, snappy_encode, snappy_decode ) from kafka.protocol import ( - create_gzip_message, - create_message, - create_snappy_message, - KafkaProtocol + create_gzip_message, create_message, create_snappy_message, KafkaProtocol ) ITERATIONS = 1000 From 81d001bfa2b6936dbefd8515204c2d51a7f299f8 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 15:18:32 -0800 Subject: [PATCH 053/109] Fix seek offset deltas We always store the offset of the next available message, so we shouldn't decrement the offset deltas when seeking by an extra 1 --- kafka/consumer.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index ff08da498..8cf27600d 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -280,12 +280,6 @@ def seek(self, offset, whence): reqs.append(OffsetRequest(self.topic, partition, -2, 1)) elif whence == 2: reqs.append(OffsetRequest(self.topic, partition, -1, 1)) - - # The API returns back the next available offset - # For eg: if the current offset is 18, the API will return - # back 19. So, if we have to seek 5 points before, we will - # end up going back to 14, instead of 13. Adjust this - deltas[partition] -= 1 else: pass From d1e4fd25c66f9fa7d955694c55e8b51c5da3a565 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 15:26:00 -0800 Subject: [PATCH 054/109] Raise a ConnectionError when a socket.error is raised when receiving data Also, log.exception() is unhelpfully noisy. Use log.error() with some error details in the message instead. --- kafka/conn.py | 18 +++++++++++------- kafka/producer.py | 6 +++--- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index ca62f52b4..ff823d238 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -35,13 +35,21 @@ def __str__(self): # Private API # ################### + def _raise_connection_error(self): + self._dirty = True + raise ConnectionError("Kafka @ {}:{} went away".format(self.host, self.port)) + def _read_bytes(self, num_bytes): bytes_left = num_bytes resp = '' log.debug("About to read %d bytes from Kafka", num_bytes) while bytes_left: - data = self._sock.recv(bytes_left) + try: + data = self._sock.recv(bytes_left) + except socket.error, e: + log.error('Unable to receive data from Kafka: %s', e) + self._raise_connection_error() if data == '': raise BufferUnderflowError("Not enough data to read this response") bytes_left -= len(data) @@ -65,10 +73,6 @@ def _consume_response(self): resp = self._read_bytes(size) return str(resp) - def _raise_connection_error(self): - self._dirty = True - raise ConnectionError("Kafka @ {}:{} went away".format(self.host, self.port)) - ################## # Public API # ################## @@ -84,8 +88,8 @@ def send(self, request_id, payload): sent = self._sock.sendall(payload) if sent is not None: self._raise_connection_error() - except socket.error: - log.exception('Unable to send payload to Kafka') + except socket.error, e: + log.error('Unable to send payload to Kafka: %s', e) self._raise_connection_error() def recv(self, request_id): diff --git a/kafka/producer.py b/kafka/producer.py index 1d4733688..eba662d27 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -67,8 +67,8 @@ def _send_upstream(topic, queue, client, batch_time, batch_size, client.send_produce_request(reqs, acks=req_acks, timeout=ack_timeout) - except Exception as exp: - log.exception("Unable to send message") + except Exception as e: + log.error("Unable to send message: %s", e) class Producer(object): @@ -145,7 +145,7 @@ def send_messages(self, partition, *msg): resp = self.client.send_produce_request([req], acks=self.req_acks, timeout=self.ack_timeout) except Exception as e: - log.exception("Unable to send messages") + log.error("Unable to send messages: %s", e) raise e return resp From 8540f1f3b6b07f9ddb28d3ade78679a0ac2d4355 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 15:29:04 -0800 Subject: [PATCH 055/109] Fix client error handling This differentiates between errors that occur when sending the request and receiving the response, and adds BufferUnderflowError handling. --- kafka/client.py | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index bd3a21406..821904c87 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -6,8 +6,10 @@ import socket import time -from kafka.common import ErrorMapping, TopicAndPartition -from kafka.common import ConnectionError, FailedPayloadsException +from kafka.common import ( + ErrorMapping, TopicAndPartition, BufferUnderflowError, ConnectionError, + FailedPayloadsException +) from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol @@ -165,14 +167,24 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): 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) if decoder_fn is None: continue - response = conn.recv(requestId) - except ConnectionError, e: # ignore BufferUnderflow for now - log.warning("Could not send request [%s] to server %s: %s" % (request, conn, e)) + try: + response = conn.recv(requestId) + except (ConnectionError, BufferUnderflowError), e: + log.warning("Could not receive response to request [%s] " + "from server %s: %s", request, conn, e) + failed = True + except ConnectionError, e: + log.warning("Could not send request [%s] to server %s: %s", + request, conn, e) + failed = True + + if failed: failed_payloads += payloads self.topics_to_brokers = {} # reset metadata continue From bbd90e12ffd83e7ed845c488e21a7155c25f5b82 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 15:49:55 -0800 Subject: [PATCH 056/109] Add a limit to fetch buffer size, and actually retry requests when fetch size is too small Note: This can cause fetching a message to exceed a given timeout, but timeouts are not guaranteed anyways, and in this case it's the client's fault for not sending a big enough buffer size rather than the kafka server. This can be bad if max_fetch_size is None (no limit) and there is some message in Kafka that is crazy huge, but that is why we should have some max_fetch_size. --- kafka/consumer.py | 95 +++++++++++++++++++++++++++++------------------ 1 file changed, 58 insertions(+), 37 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 8cf27600d..29529d6a6 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -23,6 +23,7 @@ FETCH_MAX_WAIT_TIME = 100 FETCH_MIN_BYTES = 4096 FETCH_BUFFER_SIZE_BYTES = 4096 +MAX_FETCH_BUFFER_SIZE_BYTES = FETCH_BUFFER_SIZE_BYTES * 8 ITER_TIMEOUT_SECONDS = 60 NO_MESSAGES_WAIT_TIME_SECONDS = 0.1 @@ -211,8 +212,10 @@ class SimpleConsumer(Consumer): auto_commit_every_t: default 5000. How much time (in milliseconds) to wait before commit fetch_size_bytes: number of bytes to request in a FetchRequest - buffer_size: initial number of bytes to tell kafka we have - available. This will double every time it's not enough + buffer_size: default 4K. Initial number of bytes to tell kafka we + have available. This will double as needed. + max_buffer_size: default 16K. Max number of bytes to tell kafka we have + available. None means no limit. 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. @@ -228,9 +231,15 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, auto_commit_every_t=AUTO_COMMIT_INTERVAL, fetch_size_bytes=FETCH_MIN_BYTES, buffer_size=FETCH_BUFFER_SIZE_BYTES, + max_buffer_size=MAX_FETCH_BUFFER_SIZE_BYTES, iter_timeout=None): + 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)" % + (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 @@ -353,42 +362,54 @@ def _fetch(self): # Create fetch request payloads for all the partitions requests = [] partitions = self.offsets.keys() - for partition in partitions: - requests.append(FetchRequest(self.topic, partition, - self.offsets[partition], - self.buffer_size)) - # Send request - responses = self.client.send_fetch_request( - requests, - max_wait_time=int(self.fetch_max_wait_time), - min_bytes=self.fetch_min_bytes) - - for resp in responses: - partition = resp.partition - try: - for message in resp.messages: - # Update partition offset - self.offsets[partition] = message.offset + 1 - - # Count, check and commit messages if necessary - self.count_since_commit += 1 - self._auto_commit() - - # Put the message in our queue - if self.partition_info: - self.queue.put((partition, message)) + while partitions: + for partition in partitions: + requests.append(FetchRequest(self.topic, partition, + self.offsets[partition], + self.buffer_size)) + # Send request + responses = self.client.send_fetch_request( + requests, + max_wait_time=int(self.fetch_max_wait_time), + min_bytes=self.fetch_min_bytes) + + retry_partitions = set() + for resp in responses: + partition = resp.partition + try: + for message in resp.messages: + # Update partition offset + self.offsets[partition] = message.offset + 1 + + # Count, check and commit messages if necessary + self.count_since_commit += 1 + self._auto_commit() + + # Put the message in our queue + if self.partition_info: + self.queue.put((partition, message)) + else: + self.queue.put(message) + except ConsumerFetchSizeTooSmall, e: + if (self.max_buffer_size is not None and + self.buffer_size == self.max_buffer_size): + log.error("Max fetch size %d too small", + self.max_buffer_size) + raise e + if self.max_buffer_size is None: + self.buffer_size *= 2 else: - self.queue.put(message) - except ConsumerFetchSizeTooSmall, e: - self.buffer_size *= 2 - log.warn("Fetch size too small, increase to %d (2x) and retry", - self.buffer_size) - except ConsumerNoMoreData, e: - log.debug("Iteration was ended by %r", e) - except StopIteration: - # Stop iterating through this partition - log.debug("Done iterating over partition %s" % partition) - + self.buffer_size = max([self.buffer_size * 2, + self.max_buffer_size]) + log.warn("Fetch size too small, increase to %d (2x) " + "and retry", self.buffer_size) + retry_partitions.add(partition) + except ConsumerNoMoreData, e: + log.debug("Iteration was ended by %r", e) + except StopIteration: + # Stop iterating through this partition + log.debug("Done iterating over partition %s" % partition) + partitions = retry_partitions def _mp_consume(client, group, topic, chunk, queue, start, exit, pause, size): """ From 6d2b28a59e8d8ebc86f6021c0c66973efdad8c66 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 15:52:37 -0800 Subject: [PATCH 057/109] Handle starting/stopping Kafka brokers that are already started/stopped in integration tests If some of the tests stop brokers then error out, the teardown method will try to close the same brokers and fail. This change allows it to continue. --- test/fixtures.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/test/fixtures.py b/test/fixtures.py index c771a5859..17e6672f6 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -272,8 +272,13 @@ def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot, replicas= self.tmp_dir = None self.child = None + self.running = False def open(self): + if self.running: + print("*** Kafka instance already running") + return + self.tmp_dir = tempfile.mkdtemp() print("*** Running local Kafka instance") print(" host = %s" % self.host) @@ -318,10 +323,16 @@ def open(self): self.child.start() self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id) print("*** Done!") + self.running = True def close(self): + if not self.running: + print("*** Kafka instance already stopped") + return + print("*** Stopping Kafka...") self.child.stop() self.child = None print("*** Done!") shutil.rmtree(self.tmp_dir) + self.running = False From 57218546d722cb1e47d0d00fc63e414d1ba1414c Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 17:40:07 -0800 Subject: [PATCH 058/109] Remove unnecessary brackets --- kafka/consumer.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 29529d6a6..acd4e0555 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -399,8 +399,8 @@ def _fetch(self): if self.max_buffer_size is None: self.buffer_size *= 2 else: - self.buffer_size = max([self.buffer_size * 2, - self.max_buffer_size]) + self.buffer_size = max(self.buffer_size * 2, + self.max_buffer_size) log.warn("Fetch size too small, increase to %d (2x) " "and retry", self.buffer_size) retry_partitions.add(partition) From 59f884c5aafc647114382f7bed69a484db62b5b8 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 17:47:39 -0800 Subject: [PATCH 059/109] Fix client and consumer params in integration tests --- test/test_integration.py | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/test/test_integration.py b/test/test_integration.py index a10dae243..d141c3682 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -554,7 +554,7 @@ def setUpClass(cls): cls.zk = ZookeeperFixture.instance() cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) - cls.client = KafkaClient(cls.server2.host, cls.server2.port, bufsize=8192) + cls.client = KafkaClient(cls.server2.host, cls.server2.port) @classmethod def tearDownClass(cls): # noqa @@ -583,7 +583,9 @@ def test_simple_consumer(self): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer", auto_commit=False) + consumer = SimpleConsumer(self.client, "group1", + "test_simple_consumer", auto_commit=False, + iter_timeout=0) all_messages = [] for message in consumer: all_messages.append(message) @@ -609,7 +611,9 @@ def test_simple_consumer(self): consumer.stop() def test_simple_consumer_blocking(self): - consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer_blocking", auto_commit=False) + consumer = SimpleConsumer(self.client, "group1", + "test_simple_consumer_blocking", + auto_commit=False, iter_timeout=0) # Blocking API start = datetime.now() @@ -657,7 +661,8 @@ def test_simple_consumer_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = SimpleConsumer(self.client, "group1", "test_simple_pending", auto_commit=False) + consumer = SimpleConsumer(self.client, "group1", "test_simple_pending", + auto_commit=False, iter_timeout=0) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -764,7 +769,8 @@ def test_large_messages(self): self.assertEquals(resp.offset, 10) # Consumer should still get all of them - consumer = SimpleConsumer(self.client, "group1", "test_large_messages", auto_commit=False) + consumer = SimpleConsumer(self.client, "group1", "test_large_messages", + auto_commit=False, iter_timeout=0) all_messages = messages1 + messages2 for i, message in enumerate(consumer): self.assertEquals(all_messages[i], message.message) @@ -869,7 +875,7 @@ def _kill_leader(self, topic, partition): def _count_messages(self, group, topic): client = KafkaClient(self.brokers[0].host, self.brokers[0].port) - consumer = SimpleConsumer(client, group, topic, auto_commit=False) + consumer = SimpleConsumer(client, group, topic, auto_commit=False, iter_timeout=0) all_messages = [] for message in consumer: all_messages.append(message) From d736d0b0746dc05046550f077698da45ec620157 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Fri, 3 Jan 2014 17:49:23 -0800 Subject: [PATCH 060/109] Add tests for limited and unlimited consumer max_buffer_size --- test/test_integration.py | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/test/test_integration.py b/test/test_integration.py index d141c3682..f638956b6 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -8,6 +8,7 @@ from kafka import * # noqa from kafka.common import * # noqa from kafka.codec import has_gzip, has_snappy +from kafka.consumer import FETCH_BUFFER_SIZE_BYTES, MAX_FETCH_BUFFER_SIZE_BYTES from .fixtures import ZookeeperFixture, KafkaFixture @@ -760,7 +761,7 @@ def test_large_messages(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - # Produce 10 messages that are too large (bigger than default fetch size) + # Produce 10 messages that are large (bigger than default fetch size) messages2 = [create_message(random_string(5000)) for i in range(10)] produce2 = ProduceRequest("test_large_messages", 0, messages2) @@ -776,6 +777,29 @@ def test_large_messages(self): self.assertEquals(all_messages[i], message.message) self.assertEquals(i, 19) + # Produce 1 message that is too large (bigger than max fetch size) + big_message_size = MAX_FETCH_BUFFER_SIZE_BYTES + 10 + big_message = create_message(random_string(big_message_size)) + produce3 = ProduceRequest("test_large_messages", 0, [big_message]) + for resp in self.client.send_produce_request([produce3]): + self.assertEquals(resp.error, 0) + self.assertEquals(resp.offset, 20) + + self.assertRaises(ConsumerFetchSizeTooSmall, consumer.get_message, False, 0.1) + + # Create a consumer with no fetch size limit + big_consumer = SimpleConsumer(self.client, "group1", "test_large_messages", + max_buffer_size=None, partitions=[0], + auto_commit=False, iter_timeout=0) + + # Seek to the last message + big_consumer.seek(-1, 2) + + # Consume giant message successfully + message = big_consumer.get_message(block=False, timeout=10) + self.assertIsNotNone(message) + self.assertEquals(message.message.value, big_message.value) + class TestFailover(unittest.TestCase): @classmethod From c11ff042bfffa9221220b41ca6754842b273d903 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Mon, 6 Jan 2014 14:09:16 -0800 Subject: [PATCH 061/109] Make kafka brokers per-test in failover integration tests This is better since the tests stop/start brokers, and if something goes wrong they can affect eachother. --- test/test_integration.py | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/test/test_integration.py b/test/test_integration.py index f638956b6..ccf9990fe 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -802,25 +802,23 @@ def test_large_messages(self): class TestFailover(unittest.TestCase): - @classmethod - def setUpClass(cls): + def setUp(self): - zk_chroot = random_string(10) + zk_chroot = random_string(10) replicas = 2 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)] - cls.client = KafkaClient(cls.brokers[0].host, cls.brokers[0].port) - - @classmethod - def tearDownClass(cls): - cls.client.close() - for broker in cls.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)] + self.client = KafkaClient(self.brokers[0].host, self.brokers[0].port) + + def tearDown(self): + self.client.close() + for broker in self.brokers: broker.close() - cls.zk.close() + self.zk.close() def test_switch_leader(self): From e0c45ff898aa4cb8fb4534be7d585451bafecd35 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Mon, 6 Jan 2014 14:16:11 -0800 Subject: [PATCH 062/109] Add object type and ID to message prefix in fixtures output for easier debugging --- test/fixtures.py | 36 +++++++++++++++++++++--------------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/test/fixtures.py b/test/fixtures.py index 17e6672f6..28d6519a9 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -208,9 +208,12 @@ def __init__(self, host, port): self.tmp_dir = None self.child = None + def out(self, message): + print("*** Zookeeper[%s]: %s" % (id(self), message)) + def open(self): self.tmp_dir = tempfile.mkdtemp() - print("*** Running local Zookeeper instance...") + print("*** [%s] Running local Zookeeper instance..." % id(self)) print(" host = %s" % self.host) print(" port = %s" % self.port) print(" tmp_dir = %s" % self.tmp_dir) @@ -229,16 +232,16 @@ def open(self): self.child.configure_stderr(os.path.join(self.tmp_dir, "stderr.txt")) # Party! - print("*** Starting Zookeeper...") + self.out("Starting...") self.child.start() self.child.wait_for(r"Snapshotting") - print("*** Done!") + self.out("Done!") def close(self): - print("*** Stopping Zookeeper...") + self.out("Stopping...") self.child.stop() self.child = None - print("*** Done!") + self.out("Done!") shutil.rmtree(self.tmp_dir) @@ -274,13 +277,16 @@ def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot, replicas= self.child = None self.running = False + def out(self, message): + print("*** Kafka[%s]: %s" % (id(self), message)) + def open(self): if self.running: - print("*** Kafka instance already running") + self.out("Instance already running") return self.tmp_dir = tempfile.mkdtemp() - print("*** Running local Kafka instance") + self.out("Running local instance") print(" host = %s" % self.host) print(" port = %s" % self.port) print(" broker_id = %s" % self.broker_id) @@ -308,31 +314,31 @@ def open(self): self.child.configure_stderr(os.path.join(self.tmp_dir, "stderr.txt")) # Party! - print("*** Creating Zookeeper chroot node...") + self.out("Creating Zookeeper chroot node...") proc = subprocess.Popen(kafka_run_class_args( "org.apache.zookeeper.ZooKeeperMain", "-server", "%s:%d" % (self.zk_host, self.zk_port), "create", "/%s" % self.zk_chroot, "kafka-python" )) if proc.wait() != 0: - print("*** Failed to create Zookeeper chroot node") + self.out("Failed to create Zookeeper chroot node") raise RuntimeError("Failed to create Zookeeper chroot node") - print("*** Done!") + self.out("Done!") - print("*** Starting Kafka...") + self.out("Starting...") self.child.start() self.child.wait_for(r"\[Kafka Server %d\], Started" % self.broker_id) - print("*** Done!") + self.out("Done!") self.running = True def close(self): if not self.running: - print("*** Kafka instance already stopped") + self.out("Instance already stopped") return - print("*** Stopping Kafka...") + self.out("Stopping...") self.child.stop() self.child = None - print("*** Done!") + self.out("Done!") shutil.rmtree(self.tmp_dir) self.running = False From e5a5477d346a140abd8bba9e5da2a8f9967ac911 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Tue, 7 Jan 2014 17:23:30 -0800 Subject: [PATCH 063/109] Use the same timeout when reinitializing a connection --- kafka/conn.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index ff823d238..a8d6b1fde 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -25,7 +25,8 @@ def __init__(self, host, port, timeout=10): self.port = port self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((host, port)) - self._sock.settimeout(timeout) + self.timeout = timeout + self._sock.settimeout(self.timeout) self._dirty = False def __str__(self): @@ -123,5 +124,5 @@ def reinit(self): self.close() self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) self._sock.connect((self.host, self.port)) - self._sock.settimeout(10) + self._sock.settimeout(self.timeout) self._dirty = False From b4c20acf2471a7cee23e4e2b7729a100da6c557b Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Tue, 7 Jan 2014 17:26:09 -0800 Subject: [PATCH 064/109] Handle dirty flag in conn.recv() * If the connection is dirty, reinit * If we get a BufferUnderflowError, the server could have gone away, so mark it dirty --- kafka/conn.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index a8d6b1fde..4749bdd91 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -44,7 +44,8 @@ def _read_bytes(self, num_bytes): bytes_left = num_bytes resp = '' log.debug("About to read %d bytes from Kafka", num_bytes) - + if self._dirty: + self.reinit() while bytes_left: try: data = self._sock.recv(bytes_left) @@ -52,6 +53,7 @@ def _read_bytes(self, num_bytes): log.error('Unable to receive data from Kafka: %s', e) self._raise_connection_error() if data == '': + self._dirty = True raise BufferUnderflowError("Not enough data to read this response") bytes_left -= len(data) log.debug("Read %d/%d bytes from Kafka", num_bytes - bytes_left, num_bytes) From 20776539f0277cb09f7447ffefe440d1bf231ef4 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Tue, 7 Jan 2014 17:27:21 -0800 Subject: [PATCH 065/109] Remove unnecessary method --- kafka/conn.py | 25 ++++++++----------------- 1 file changed, 8 insertions(+), 17 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 4749bdd91..b2916ce2f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -61,21 +61,6 @@ def _read_bytes(self, num_bytes): return resp - def _consume_response(self): - """ - This method handles the response header and error messages. It - then returns the response - """ - log.debug("Expecting response from Kafka") - # Read the size off of the header - resp = self._read_bytes(4) - - (size,) = struct.unpack('>i', resp) - - # Read the remainder of the response - resp = self._read_bytes(size) - return str(resp) - ################## # Public API # ################## @@ -100,8 +85,14 @@ def recv(self, request_id): Get a response from Kafka """ log.debug("Reading response %d from Kafka" % request_id) - self.data = self._consume_response() - return self.data + # Read the size off of the header + resp = self._read_bytes(4) + + (size,) = struct.unpack('>i', resp) + + # Read the remainder of the response + resp = self._read_bytes(size) + return str(resp) def copy(self): """ From f333e9154d89a28aebf021b0776f42c653cffeb3 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Tue, 7 Jan 2014 17:46:01 -0800 Subject: [PATCH 066/109] Skip snappy/gzip tests if they're not available --- test/test_unit.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/test/test_unit.py b/test/test_unit.py index 08fef9c2a..e3fd4bb5b 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -55,17 +55,16 @@ def test_submodule_namespace(self): class TestCodec(unittest.TestCase): + + @unittest.skipUnless(has_gzip(), "Gzip not available") def test_gzip(self): - if not has_gzip(): - return for i in xrange(ITERATIONS): s1 = random_string() s2 = gzip_decode(gzip_encode(s1)) self.assertEquals(s1, s2) + @unittest.skipUnless(has_snappy(), "Snappy not available") def test_snappy(self): - if not has_snappy(): - return for i in xrange(ITERATIONS): s1 = random_string() s2 = snappy_decode(snappy_encode(s1)) @@ -83,6 +82,7 @@ def test_create_message(self): self.assertEqual(msg.key, key) self.assertEqual(msg.value, payload) + @unittest.skipUnless(has_gzip(), "Snappy not available") def test_create_gzip(self): payloads = ["v1", "v2"] msg = create_gzip_message(payloads) @@ -98,6 +98,7 @@ def test_create_gzip(self): "\xff\xff\x00\x00\x00\x02v2") self.assertEqual(decoded, expect) + @unittest.skipUnless(has_snappy(), "Snappy not available") def test_create_snappy(self): payloads = ["v1", "v2"] msg = create_snappy_message(payloads) @@ -157,6 +158,7 @@ def test_decode_message_set(self): self.assertEqual(returned_offset2, 0) self.assertEqual(decoded_message2, create_message("v2")) + @unittest.skipUnless(has_gzip(), "Gzip not available") def test_decode_message_gzip(self): gzip_encoded = ('\xc0\x11\xb2\xf0\x00\x01\xff\xff\xff\xff\x00\x00\x000' '\x1f\x8b\x08\x00\xa1\xc1\xc5R\x02\xffc`\x80\x03\x01' @@ -173,6 +175,7 @@ def test_decode_message_gzip(self): self.assertEqual(returned_offset2, 0) self.assertEqual(decoded_message2, create_message("v2")) + @unittest.skipUnless(has_snappy(), "Snappy not available") def test_decode_message_snappy(self): snappy_encoded = ('\xec\x80\xa1\x95\x00\x02\xff\xff\xff\xff\x00\x00' '\x00,8\x00\x00\x19\x01@\x10L\x9f[\xc2\x00\x00\xff' From 9cbe45d2358d563bb2844eecaa545e7f6d0a6629 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Tue, 7 Jan 2014 17:54:05 -0800 Subject: [PATCH 067/109] Some cleanup and easier to read test fixture output --- test/fixtures.py | 8 ++++---- test/test_integration.py | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/test/fixtures.py b/test/fixtures.py index 28d6519a9..9e283d3c5 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -209,11 +209,11 @@ def __init__(self, host, port): self.child = None def out(self, message): - print("*** Zookeeper[%s]: %s" % (id(self), message)) + print("*** Zookeeper [%s:%d]: %s" % (self.host, self.port, message)) def open(self): self.tmp_dir = tempfile.mkdtemp() - print("*** [%s] Running local Zookeeper instance..." % id(self)) + self.out("Running local instance...") print(" host = %s" % self.host) print(" port = %s" % self.port) print(" tmp_dir = %s" % self.tmp_dir) @@ -278,7 +278,7 @@ def __init__(self, host, port, broker_id, zk_host, zk_port, zk_chroot, replicas= self.running = False def out(self, message): - print("*** Kafka[%s]: %s" % (id(self), message)) + print("*** Kafka [%s:%d]: %s" % (self.host, self.port, message)) def open(self): if self.running: @@ -286,7 +286,7 @@ def open(self): return self.tmp_dir = tempfile.mkdtemp() - self.out("Running local instance") + self.out("Running local instance...") print(" host = %s" % self.host) print(" port = %s" % self.port) print(" broker_id = %s" % self.broker_id) diff --git a/test/test_integration.py b/test/test_integration.py index ccf9990fe..eaf432d64 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -8,7 +8,7 @@ from kafka import * # noqa from kafka.common import * # noqa from kafka.codec import has_gzip, has_snappy -from kafka.consumer import FETCH_BUFFER_SIZE_BYTES, MAX_FETCH_BUFFER_SIZE_BYTES +from kafka.consumer import MAX_FETCH_BUFFER_SIZE_BYTES from .fixtures import ZookeeperFixture, KafkaFixture From 317c8480164763b484fa82d0e0273107bc861538 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 8 Jan 2014 11:30:15 -0800 Subject: [PATCH 068/109] Change BufferUnderflowError to ConnectionError in conn._read_bytes() Both errors are handled the same way when raised and caught, so this makes sense. --- kafka/client.py | 5 ++--- kafka/conn.py | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 821904c87..33c6d778f 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -3,11 +3,10 @@ from functools import partial from itertools import count import logging -import socket import time from kafka.common import ( - ErrorMapping, TopicAndPartition, BufferUnderflowError, ConnectionError, + ErrorMapping, TopicAndPartition, ConnectionError, FailedPayloadsException ) from kafka.conn import KafkaConnection @@ -175,7 +174,7 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): continue try: response = conn.recv(requestId) - except (ConnectionError, BufferUnderflowError), e: + except ConnectionError, e: log.warning("Could not receive response to request [%s] " "from server %s: %s", request, conn, e) failed = True diff --git a/kafka/conn.py b/kafka/conn.py index b2916ce2f..9a6633a3a 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -4,7 +4,6 @@ import struct from threading import local -from kafka.common import BufferUnderflowError from kafka.common import ConnectionError log = logging.getLogger("kafka") @@ -53,8 +52,8 @@ def _read_bytes(self, num_bytes): log.error('Unable to receive data from Kafka: %s', e) self._raise_connection_error() if data == '': - self._dirty = True - raise BufferUnderflowError("Not enough data to read this response") + log.error("Not enough data to read this response") + self._raise_connection_error() bytes_left -= len(data) log.debug("Read %d/%d bytes from Kafka", num_bytes - bytes_left, num_bytes) resp += data From a0c7141e2cc7399a9472a8169ea5f730f0407386 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 8 Jan 2014 11:46:10 -0800 Subject: [PATCH 069/109] Change log.error() back to log.exception() --- kafka/conn.py | 6 +++--- kafka/producer.py | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 9a6633a3a..c80f42892 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -48,8 +48,8 @@ def _read_bytes(self, num_bytes): while bytes_left: try: data = self._sock.recv(bytes_left) - except socket.error, e: - log.error('Unable to receive data from Kafka: %s', e) + except socket.error: + log.exception('Unable to receive data from Kafka') self._raise_connection_error() if data == '': log.error("Not enough data to read this response") @@ -76,7 +76,7 @@ def send(self, request_id, payload): if sent is not None: self._raise_connection_error() except socket.error, e: - log.error('Unable to send payload to Kafka: %s', e) + log.exception('Unable to send payload to Kafka') self._raise_connection_error() def recv(self, request_id): diff --git a/kafka/producer.py b/kafka/producer.py index eba662d27..a68b6c858 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -67,8 +67,8 @@ def _send_upstream(topic, queue, client, batch_time, batch_size, client.send_produce_request(reqs, acks=req_acks, timeout=ack_timeout) - except Exception as e: - log.error("Unable to send message: %s", e) + except Exception: + log.exception("Unable to send message") class Producer(object): @@ -144,9 +144,9 @@ def send_messages(self, partition, *msg): try: resp = self.client.send_produce_request([req], acks=self.req_acks, timeout=self.ack_timeout) - except Exception as e: - log.error("Unable to send messages: %s", e) - raise e + except Exception: + log.exception("Unable to send messages") + raise return resp def stop(self, timeout=1): From 55816493c76b891fca1e719eff0fe48e3f43f27f Mon Sep 17 00:00:00 2001 From: Ashish Walia Date: Tue, 7 Jan 2014 12:27:50 -0500 Subject: [PATCH 070/109] Syncing offset commit and fetch api keys with Kafka trunk code --- kafka/protocol.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/protocol.py b/kafka/protocol.py index 612acf63f..eded41c8a 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -29,8 +29,8 @@ class KafkaProtocol(object): FETCH_KEY = 1 OFFSET_KEY = 2 METADATA_KEY = 3 - OFFSET_COMMIT_KEY = 6 - OFFSET_FETCH_KEY = 7 + OFFSET_COMMIT_KEY = 8 + OFFSET_FETCH_KEY = 9 ATTRIBUTE_CODEC_MASK = 0x03 CODEC_NONE = 0x00 From daabc9f28dd9283b45e5ccd854e8fd8800419d3c Mon Sep 17 00:00:00 2001 From: Ashish Walia Date: Tue, 7 Jan 2014 12:30:52 -0500 Subject: [PATCH 071/109] Deleting client_id from offset commit and fetch response as per Kafka trunk code --- kafka/protocol.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/kafka/protocol.py b/kafka/protocol.py index eded41c8a..ac1bafd59 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -439,7 +439,6 @@ def decode_offset_commit_response(cls, data): data: bytes to decode """ ((correlation_id,), cur) = relative_unpack('>i', data, 0) - (client_id, cur) = read_short_string(data, cur) ((num_topics,), cur) = relative_unpack('>i', data, cur) for i in xrange(num_topics): @@ -490,7 +489,6 @@ def decode_offset_fetch_response(cls, data): """ ((correlation_id,), cur) = relative_unpack('>i', data, 0) - (client_id, cur) = read_short_string(data, cur) ((num_topics,), cur) = relative_unpack('>i', data, cur) for i in range(num_topics): From b9167ca1f0404918df0293a19a70b21a213dee4d Mon Sep 17 00:00:00 2001 From: Vadim Graboys Date: Mon, 13 Jan 2014 17:43:55 -0500 Subject: [PATCH 072/109] remove zero length field name in format string, to work in Python 2.6 --- kafka/conn.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index c80f42892..a6d93f8f7 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -37,7 +37,7 @@ def __str__(self): def _raise_connection_error(self): self._dirty = True - raise ConnectionError("Kafka @ {}:{} went away".format(self.host, self.port)) + raise ConnectionError("Kafka @ {0}:{1} went away".format(self.host, self.port)) def _read_bytes(self, num_bytes): bytes_left = num_bytes From 9c7b41283851735cbee5092f2923d7c8a006b89a Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Mon, 13 Jan 2014 15:30:51 -0800 Subject: [PATCH 073/109] Exception hierarchy, invalidate more md on errors --- kafka/client.py | 168 ++++++++++++++++++++------------------- kafka/common.py | 31 ++++++-- kafka/consumer.py | 2 +- kafka/producer.py | 15 ++-- test/test_integration.py | 72 ++++++++++------- 5 files changed, 166 insertions(+), 122 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 33c6d778f..7e169e884 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -1,14 +1,15 @@ import copy +import logging + from collections import defaultdict from functools import partial from itertools import count -import logging -import time -from kafka.common import ( - ErrorMapping, TopicAndPartition, ConnectionError, - FailedPayloadsException -) +from kafka.common import (ErrorMapping, TopicAndPartition, + ConnectionError, FailedPayloadsError, + BrokerResponseError, PartitionUnavailableError, + KafkaRequestError) + from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol @@ -29,8 +30,8 @@ def __init__(self, host, port, client_id=CLIENT_ID, timeout=10): } self.brokers = {} # broker_id -> BrokerMetadata self.topics_to_brokers = {} # topic_id -> broker_id - self.topic_partitions = defaultdict(list) # topic_id -> [0, 1, 2, ...] - self._load_metadata_for_topics() + self.topic_partitions = {} # topic_id -> [0, 1, 2, ...] + self.load_metadata_for_topics() # bootstrap with all metadata ################## # Private API # @@ -49,55 +50,13 @@ def _get_conn_for_broker(self, broker): def _get_leader_for_partition(self, topic, partition): key = TopicAndPartition(topic, partition) if key not in self.topics_to_brokers: - self._load_metadata_for_topics(topic) + self.load_metadata_for_topics(topic) if key not in self.topics_to_brokers: - raise Exception("Partition does not exist: %s" % str(key)) + raise KafkaRequestError("Partition does not exist: %s" % str(key)) return self.topics_to_brokers[key] - def _load_metadata_for_topics(self, *topics): - """ - Discover brokers and metadata for a set of topics. This method will - recurse in the event of a retry. - """ - request_id = self._next_id() - request = KafkaProtocol.encode_metadata_request(self.client_id, - request_id, topics) - - response = self._send_broker_unaware_request(request_id, request) - if response is None: - raise Exception("All servers failed to process request") - - (brokers, topics) = KafkaProtocol.decode_metadata_response(response) - - log.debug("Broker metadata: %s", brokers) - log.debug("Topic metadata: %s", topics) - - self.brokers = brokers - self.topics_to_brokers = {} - - for topic, partitions in topics.items(): - # Clear the list once before we add it. This removes stale entries - # and avoids duplicates - self.topic_partitions.pop(topic, None) - - if not partitions: - log.info("Partition is unassigned, delay for 1s and retry") - time.sleep(1) - self._load_metadata_for_topics(topic) - break - - for partition, meta in partitions.items(): - if meta.leader == -1: - log.info("Partition is unassigned, delay for 1s and retry") - time.sleep(1) - self._load_metadata_for_topics(topic) - else: - topic_part = TopicAndPartition(topic, partition) - self.topics_to_brokers[topic_part] = brokers[meta.leader] - self.topic_partitions[topic].append(partition) - def _next_id(self): """ Generate a new correlation id @@ -119,7 +78,7 @@ def _send_broker_unaware_request(self, requestId, request): "trying next server: %s" % (request, conn, e)) continue - return None + raise BrokerResponseError("All servers failed to process request") def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): """ @@ -150,6 +109,8 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): for payload in payloads: leader = self._get_leader_for_partition(payload.topic, payload.partition) + if leader == -1: + raise PartitionUnavailableError("Leader is unassigned for %s-%s" % payload.topic, payload.partition) payloads_by_broker[leader].append(payload) original_keys.append((payload.topic, payload.partition)) @@ -185,21 +146,51 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): if failed: failed_payloads += payloads - self.topics_to_brokers = {} # reset metadata + self.reset_all_metadata() continue for response in decoder_fn(response): acc[(response.topic, response.partition)] = response if failed_payloads: - raise FailedPayloadsException(failed_payloads) + raise FailedPayloadsError(failed_payloads) # Order the accumulated responses by the original key order return (acc[k] for k in original_keys) if acc else () + def _raise_on_response_error(self, resp): + if resp.error == ErrorMapping.NO_ERROR: + return + + if resp.error in (ErrorMapping.UNKNOWN_TOPIC_OR_PARTITON, + ErrorMapping.NOT_LEADER_FOR_PARTITION): + self.reset_topic_metadata(resp.topic) + + raise BrokerResponseError( + "Request for %s failed with errorcode=%d" % + (TopicAndPartition(resp.topic, resp.partition), resp.error)) + ################# # Public API # ################# + 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] + + def reset_all_metadata(self): + self.topics_to_brokers.clear() + self.topic_partitions.clear() + + def has_metadata_for_topic(self, topic): + return topic in self.topic_partitions def close(self): for conn in self.conns.values(): @@ -219,6 +210,36 @@ def reinit(self): for conn in self.conns.values(): conn.reinit() + def load_metadata_for_topics(self, *topics): + """ + Discover brokers and metadata for a set of topics. This function is called + lazily whenever metadata is unavailable. + """ + request_id = self._next_id() + request = KafkaProtocol.encode_metadata_request(self.client_id, + request_id, topics) + + response = self._send_broker_unaware_request(request_id, request) + + (brokers, topics) = KafkaProtocol.decode_metadata_response(response) + + log.debug("Broker metadata: %s", brokers) + log.debug("Topic metadata: %s", topics) + + self.brokers = brokers + + for topic, partitions in topics.items(): + self.reset_topic_metadata(topic) + + if not partitions: + continue + + self.topic_partitions[topic] = [] + for partition, meta in partitions.items(): + topic_part = TopicAndPartition(topic, partition) + self.topics_to_brokers[topic_part] = brokers[meta.leader] + self.topic_partitions[topic].append(partition) + def send_produce_request(self, payloads=[], acks=1, timeout=1000, fail_on_error=True, callback=None): """ @@ -256,14 +277,9 @@ def send_produce_request(self, payloads=[], acks=1, timeout=1000, out = [] for resp in resps: - # Check for errors - if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception( - "ProduceRequest for %s failed with errorcode=%d" % - (TopicAndPartition(resp.topic, resp.partition), - resp.error)) - - # Run the callback + if fail_on_error is True: + self._raise_on_response_error(resp) + if callback is not None: out.append(callback(resp)) else: @@ -289,14 +305,9 @@ def send_fetch_request(self, payloads=[], fail_on_error=True, out = [] for resp in resps: - # Check for errors - if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception( - "FetchRequest for %s failed with errorcode=%d" % - (TopicAndPartition(resp.topic, resp.partition), - resp.error)) - - # Run the callback + if fail_on_error is True: + self._raise_on_response_error(resp) + if callback is not None: out.append(callback(resp)) else: @@ -312,9 +323,8 @@ def send_offset_request(self, payloads=[], fail_on_error=True, out = [] for resp in resps: - if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("OffsetRequest failed with errorcode=%s", - resp.error) + if fail_on_error is True: + self._raise_on_response_error(resp) if callback is not None: out.append(callback(resp)) else: @@ -330,9 +340,8 @@ def send_offset_commit_request(self, group, payloads=[], out = [] for resp in resps: - if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("OffsetCommitRequest failed with " - "errorcode=%s", resp.error) + if fail_on_error is True: + self._raise_on_response_error(resp) if callback is not None: out.append(callback(resp)) @@ -350,9 +359,8 @@ def send_offset_fetch_request(self, group, payloads=[], out = [] for resp in resps: - if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("OffsetCommitRequest failed with errorcode=%s", - resp.error) + if fail_on_error is True: + self._raise_on_response_error(resp) if callback is not None: out.append(callback(resp)) else: diff --git a/kafka/common.py b/kafka/common.py index 6f0dd322b..5bd9a967d 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -69,23 +69,42 @@ class ErrorMapping(object): # Exceptions # ################# -class FailedPayloadsException(Exception): + +class KafkaError(RuntimeError): + pass + + +class KafkaRequestError(KafkaError): pass -class ConnectionError(Exception): + +class BrokerResponseError(KafkaError): + pass + + +class PartitionUnavailableError(KafkaError): pass -class BufferUnderflowError(Exception): + +class FailedPayloadsError(KafkaError): + pass + + +class ConnectionError(KafkaError): + pass + + +class BufferUnderflowError(KafkaError): pass -class ChecksumError(Exception): +class ChecksumError(KafkaError): pass -class ConsumerFetchSizeTooSmall(Exception): +class ConsumerFetchSizeTooSmall(KafkaError): pass -class ConsumerNoMoreData(Exception): +class ConsumerNoMoreData(KafkaError): pass diff --git a/kafka/consumer.py b/kafka/consumer.py index eba291247..522d6cacd 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -76,7 +76,7 @@ def __init__(self, client, group, topic, partitions=None, auto_commit=True, self.client = client self.topic = topic self.group = group - self.client._load_metadata_for_topics(topic) + self.client.load_metadata_for_topics(topic) self.offsets = {} if not partitions: diff --git a/kafka/producer.py b/kafka/producer.py index 5aead439a..6ed22ee4a 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -1,17 +1,16 @@ from __future__ import absolute_import +import logging +import time + +from Queue import Empty from collections import defaultdict from itertools import cycle from multiprocessing import Queue, Process -from Queue import Empty -import logging -import sys -import time from kafka.common import ProduceRequest -from kafka.common import FailedPayloadsException -from kafka.protocol import create_message from kafka.partitioner import HashedPartitioner +from kafka.protocol import create_message log = logging.getLogger("kafka") @@ -188,7 +187,7 @@ def __init__(self, client, topic, async=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL): self.topic = topic - client._load_metadata_for_topics(topic) + client.load_metadata_for_topics(topic) self.next_partition = cycle(client.topic_partitions[topic]) super(SimpleProducer, self).__init__(client, async, req_acks, @@ -225,7 +224,7 @@ def __init__(self, client, topic, partitioner=None, async=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL): self.topic = topic - client._load_metadata_for_topics(topic) + client.load_metadata_for_topics(topic) if not partitioner: partitioner = HashedPartitioner diff --git a/test/test_integration.py b/test/test_integration.py index eaf432d64..56974a5eb 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -12,7 +12,23 @@ from .fixtures import ZookeeperFixture, KafkaFixture -class TestKafkaClient(unittest.TestCase): +class KafkaTestCase(unittest.TestCase): + def setUp(self): + topic_name = self.id()[self.id().rindex(".")+1:] + times = 0 + while True: + times += 1 + self.client.load_metadata_for_topics(topic_name) + if self.client.has_metadata_for_topic(topic_name): + break + print "Waiting for %s topic to be created" % topic_name + time.sleep(1) + + if times > 30: + raise Exception("Unable to create topic %s" % topic_name) + + +class TestKafkaClient(KafkaTestCase): @classmethod def setUpClass(cls): # noqa cls.zk = ZookeeperFixture.instance() @@ -30,6 +46,7 @@ def tearDownClass(cls): # noqa ##################### def test_produce_many_simple(self): + produce = ProduceRequest("test_produce_many_simple", 0, messages=[ create_message("Test message %d" % i) for i in range(100) ]) @@ -331,15 +348,15 @@ def test_round_robin_partitioner(self): producer.stop() def test_hashed_partitioner(self): - producer = KeyedProducer(self.client, "test_hash_partitioner", + producer = KeyedProducer(self.client, "test_hashed_partitioner", partitioner=HashedPartitioner) producer.send(1, "one") producer.send(2, "two") producer.send(3, "three") producer.send(4, "four") - fetch1 = FetchRequest("test_hash_partitioner", 0, 0, 1024) - fetch2 = FetchRequest("test_hash_partitioner", 1, 0, 1024) + fetch1 = FetchRequest("test_hashed_partitioner", 0, 0, 1024) + fetch2 = FetchRequest("test_hashed_partitioner", 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -549,7 +566,7 @@ def test_batched_simple_producer(self): producer.stop() -class TestConsumer(unittest.TestCase): +class TestConsumer(KafkaTestCase): @classmethod def setUpClass(cls): cls.zk = ZookeeperFixture.instance() @@ -648,21 +665,21 @@ def test_simple_consumer_blocking(self): def test_simple_consumer_pending(self): # Produce 10 messages to partition 0 and 1 - produce1 = ProduceRequest("test_simple_pending", 0, messages=[ + produce1 = ProduceRequest("test_simple_consumer_pending", 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) for resp in self.client.send_produce_request([produce1]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - produce2 = ProduceRequest("test_simple_pending", 1, messages=[ + produce2 = ProduceRequest("test_simple_consumer_pending", 1, messages=[ create_message("Test message 1 %d" % i) for i in range(10) ]) for resp in self.client.send_produce_request([produce2]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = SimpleConsumer(self.client, "group1", "test_simple_pending", + consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer_pending", auto_commit=False, iter_timeout=0) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) @@ -671,7 +688,7 @@ def test_simple_consumer_pending(self): def test_multi_process_consumer(self): # Produce 100 messages to partition 0 - produce1 = ProduceRequest("test_mpconsumer", 0, messages=[ + produce1 = ProduceRequest("test_multi_process_consumer", 0, messages=[ create_message("Test message 0 %d" % i) for i in range(100) ]) @@ -680,7 +697,7 @@ def test_multi_process_consumer(self): self.assertEquals(resp.offset, 0) # Produce 100 messages to partition 1 - produce2 = ProduceRequest("test_mpconsumer", 1, messages=[ + produce2 = ProduceRequest("test_multi_process_consumer", 1, messages=[ create_message("Test message 1 %d" % i) for i in range(100) ]) @@ -689,7 +706,7 @@ def test_multi_process_consumer(self): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = MultiProcessConsumer(self.client, "grp1", "test_mpconsumer", auto_commit=False) + consumer = MultiProcessConsumer(self.client, "grp1", "test_multi_process_consumer", auto_commit=False) all_messages = [] for message in consumer: all_messages.append(message) @@ -702,11 +719,11 @@ def test_multi_process_consumer(self): start = datetime.now() messages = consumer.get_messages(block=True, timeout=5) diff = (datetime.now() - start).total_seconds() - self.assertGreaterEqual(diff, 5) + self.assertGreaterEqual(diff, 4.9) self.assertEqual(len(messages), 0) # Send 10 messages - produce = ProduceRequest("test_mpconsumer", 0, messages=[ + produce = ProduceRequest("test_multi_process_consumer", 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -729,7 +746,7 @@ def test_multi_process_consumer(self): def test_multi_proc_pending(self): # Produce 10 messages to partition 0 and 1 - produce1 = ProduceRequest("test_mppending", 0, messages=[ + produce1 = ProduceRequest("test_multi_proc_pending", 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -737,7 +754,7 @@ def test_multi_proc_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - produce2 = ProduceRequest("test_mppending", 1, messages=[ + produce2 = ProduceRequest("test_multi_proc_pending", 1, messages=[ create_message("Test message 1 %d" % i) for i in range(10) ]) @@ -745,7 +762,7 @@ def test_multi_proc_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = MultiProcessConsumer(self.client, "group1", "test_mppending", auto_commit=False) + consumer = MultiProcessConsumer(self.client, "group1", "test_multi_proc_pending", auto_commit=False) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -800,19 +817,20 @@ def test_large_messages(self): self.assertIsNotNone(message) self.assertEquals(message.message.value, big_message.value) -class TestFailover(unittest.TestCase): - def setUp(self): +class TestFailover(KafkaTestCase): - zk_chroot = random_string(10) - replicas = 2 + def setUp(self): + zk_chroot = random_string(10) + replicas = 2 partitions = 2 # mini zookeeper, 2 kafka brokers - self.zk = ZookeeperFixture.instance() - kk_args = [self.zk.host, self.zk.port, zk_chroot, replicas, partitions] + 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)] - self.client = KafkaClient(self.brokers[0].host, self.brokers[0].port) + self.client = KafkaClient(self.brokers[0].host, self.brokers[0].port) + KafkaTestCase.setUp(self) def tearDown(self): self.client.close() @@ -835,7 +853,7 @@ def test_switch_leader(self): broker = self._kill_leader(topic, partition) # expect failure, reload meta data - with self.assertRaises(FailedPayloadsException): + with self.assertRaises(FailedPayloadsError): producer.send_messages('part 1') producer.send_messages('part 2') time.sleep(1) @@ -886,17 +904,17 @@ def _send_random_messages(self, producer, n): resp = producer.send_messages(random_string(10)) if len(resp) > 0: self.assertEquals(resp[0].error, 0) - time.sleep(1) # give it some time + time.sleep(1) # give it some time def _kill_leader(self, topic, partition): leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] broker = self.brokers[leader.nodeId] broker.close() - time.sleep(1) # give it some time + time.sleep(1) # give it some time return broker def _count_messages(self, group, topic): - client = KafkaClient(self.brokers[0].host, self.brokers[0].port) + client = KafkaClient(self.brokers[0].host, self.brokers[0].port) consumer = SimpleConsumer(client, group, topic, auto_commit=False, iter_timeout=0) all_messages = [] for message in consumer: From a3c3a154606bda07cc8613587eb6e04b57316cb5 Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Mon, 13 Jan 2014 16:39:06 -0800 Subject: [PATCH 074/109] Throw KafkaUnavailableError when no brokers available --- kafka/client.py | 4 ++-- kafka/common.py | 4 ++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 7e169e884..1016051b4 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -8,7 +8,7 @@ from kafka.common import (ErrorMapping, TopicAndPartition, ConnectionError, FailedPayloadsError, BrokerResponseError, PartitionUnavailableError, - KafkaRequestError) + KafkaUnavailableError, KafkaRequestError) from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol @@ -78,7 +78,7 @@ def _send_broker_unaware_request(self, requestId, request): "trying next server: %s" % (request, conn, e)) continue - raise BrokerResponseError("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): """ diff --git a/kafka/common.py b/kafka/common.py index 5bd9a967d..c0a1a6a1a 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -78,6 +78,10 @@ class KafkaRequestError(KafkaError): pass +class KafkaUnavailableError(KafkaError): + pass + + class BrokerResponseError(KafkaError): pass From db322dae0933f0fe9b65c8218cc7471c2c6d6b7d Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Mon, 13 Jan 2014 19:12:05 -0800 Subject: [PATCH 075/109] Add a few 9s test_multi_process_consumer, make clear what setUp is doing --- test/test_integration.py | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/test/test_integration.py b/test/test_integration.py index 56974a5eb..9ad58dbb4 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -12,20 +12,24 @@ from .fixtures import ZookeeperFixture, KafkaFixture +def ensure_topic_creation(client, topic_name): + times = 0 + while True: + times += 1 + client.load_metadata_for_topics(topic_name) + if client.has_metadata_for_topic(topic_name): + break + print "Waiting for %s topic to be created" % topic_name + time.sleep(1) + + if times > 30: + raise Exception("Unable to create topic %s" % topic_name) + + class KafkaTestCase(unittest.TestCase): def setUp(self): topic_name = self.id()[self.id().rindex(".")+1:] - times = 0 - while True: - times += 1 - self.client.load_metadata_for_topics(topic_name) - if self.client.has_metadata_for_topic(topic_name): - break - print "Waiting for %s topic to be created" % topic_name - time.sleep(1) - - if times > 30: - raise Exception("Unable to create topic %s" % topic_name) + ensure_topic_creation(self.client, topic_name) class TestKafkaClient(KafkaTestCase): @@ -719,7 +723,7 @@ def test_multi_process_consumer(self): start = datetime.now() messages = consumer.get_messages(block=True, timeout=5) diff = (datetime.now() - start).total_seconds() - self.assertGreaterEqual(diff, 4.9) + self.assertGreaterEqual(diff, 4.999) self.assertEqual(len(messages), 0) # Send 10 messages @@ -830,7 +834,7 @@ def setUp(self): kk_args = [self.zk.host, self.zk.port, zk_chroot, replicas, partitions] self.brokers = [KafkaFixture.instance(i, *kk_args) for i in range(replicas)] self.client = KafkaClient(self.brokers[0].host, self.brokers[0].port) - KafkaTestCase.setUp(self) + super(TestFailover, self).setUp() def tearDown(self): self.client.close() From 2818ddfa924b470b844222cfd76cbe8cdf8dcc51 Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Tue, 14 Jan 2014 14:22:50 -0800 Subject: [PATCH 076/109] Make integration test warming more intuitive --- test/test_integration.py | 177 +++++++++++++++++++-------------------- 1 file changed, 86 insertions(+), 91 deletions(-) diff --git a/test/test_integration.py b/test/test_integration.py index 9ad58dbb4..5a2263013 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -12,6 +12,11 @@ from .fixtures import ZookeeperFixture, KafkaFixture +def random_string(l): + s = "".join(random.choice(string.letters) for i in xrange(l)) + return s + + def ensure_topic_creation(client, topic_name): times = 0 while True: @@ -28,8 +33,8 @@ def ensure_topic_creation(client, topic_name): class KafkaTestCase(unittest.TestCase): def setUp(self): - topic_name = self.id()[self.id().rindex(".")+1:] - ensure_topic_creation(self.client, topic_name) + self.topic = "%s-%s" % (self.id()[self.id().rindex(".")+1:], random_string(10)) + ensure_topic_creation(self.client, self.topic) class TestKafkaClient(KafkaTestCase): @@ -51,7 +56,7 @@ def tearDownClass(cls): # noqa def test_produce_many_simple(self): - produce = ProduceRequest("test_produce_many_simple", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Test message %d" % i) for i in range(100) ]) @@ -59,25 +64,25 @@ def test_produce_many_simple(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_simple", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 100) for resp in self.client.send_produce_request([produce]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 100) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_simple", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 200) for resp in self.client.send_produce_request([produce]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 200) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_simple", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 300) def test_produce_10k_simple(self): - produce = ProduceRequest("test_produce_10k_simple", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Test message %d" % i) for i in range(10000) ]) @@ -85,7 +90,7 @@ def test_produce_10k_simple(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_10k_simple", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 10000) def test_produce_many_gzip(self): @@ -94,13 +99,13 @@ def test_produce_many_gzip(self): message1 = create_gzip_message(["Gzipped 1 %d" % i for i in range(100)]) message2 = create_gzip_message(["Gzipped 2 %d" % i for i in range(100)]) - produce = ProduceRequest("test_produce_many_gzip", 0, messages=[message1, message2]) + produce = ProduceRequest(self.topic, 0, messages=[message1, message2]) for resp in self.client.send_produce_request([produce]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_gzip", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 200) def test_produce_many_snappy(self): @@ -109,13 +114,13 @@ def test_produce_many_snappy(self): message1 = create_snappy_message(["Snappy 1 %d" % i for i in range(100)]) message2 = create_snappy_message(["Snappy 2 %d" % i for i in range(100)]) - produce = ProduceRequest("test_produce_many_snappy", 0, messages=[message1, message2]) + produce = ProduceRequest(self.topic, 0, messages=[message1, message2]) for resp in self.client.send_produce_request([produce]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_many_snappy", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 200) def test_produce_mixed(self): @@ -125,17 +130,17 @@ def test_produce_mixed(self): message2 = create_gzip_message(["Gzipped %d" % i for i in range(100)]) message3 = create_snappy_message(["Snappy %d" % i for i in range(100)]) - produce = ProduceRequest("test_produce_mixed", 0, messages=[message1, message2, message3]) + produce = ProduceRequest(self.topic, 0, messages=[message1, message2, message3]) for resp in self.client.send_produce_request([produce]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_mixed", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 201) def test_produce_100k_gzipped(self): - req1 = ProduceRequest("test_produce_100k_gzipped", 0, messages=[ + req1 = ProduceRequest(self.topic, 0, messages=[ create_gzip_message(["Gzipped batch 1, message %d" % i for i in range(50000)]) ]) @@ -143,10 +148,10 @@ def test_produce_100k_gzipped(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_100k_gzipped", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 50000) - req2 = ProduceRequest("test_produce_100k_gzipped", 0, messages=[ + req2 = ProduceRequest(self.topic, 0, messages=[ create_gzip_message(["Gzipped batch 2, message %d" % i for i in range(50000)]) ]) @@ -154,7 +159,7 @@ def test_produce_100k_gzipped(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 50000) - (offset, ) = self.client.send_offset_request([OffsetRequest("test_produce_100k_gzipped", 0, -1, 1)]) + (offset, ) = self.client.send_offset_request([OffsetRequest(self.topic, 0, -1, 1)]) self.assertEquals(offset.offsets[0], 100000) ##################### @@ -162,18 +167,18 @@ def test_produce_100k_gzipped(self): ##################### def test_consume_none(self): - fetch = FetchRequest("test_consume_none", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch])[0] self.assertEquals(fetch_resp.error, 0) - self.assertEquals(fetch_resp.topic, "test_consume_none") + self.assertEquals(fetch_resp.topic, self.topic) self.assertEquals(fetch_resp.partition, 0) messages = list(fetch_resp.messages) self.assertEquals(len(messages), 0) def test_produce_consume(self): - produce = ProduceRequest("test_produce_consume", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Just a test message"), create_message("Message with a key", "foo"), ]) @@ -182,7 +187,7 @@ def test_produce_consume(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - fetch = FetchRequest("test_produce_consume", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch])[0] self.assertEquals(fetch_resp.error, 0) @@ -197,7 +202,7 @@ def test_produce_consume(self): self.assertEquals(messages[1].message.key, "foo") def test_produce_consume_many(self): - produce = ProduceRequest("test_produce_consume_many", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Test message %d" % i) for i in range(100) ]) @@ -206,7 +211,7 @@ def test_produce_consume_many(self): self.assertEquals(resp.offset, 0) # 1024 is not enough for 100 messages... - fetch1 = FetchRequest("test_produce_consume_many", 0, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) (fetch_resp1,) = self.client.send_fetch_request([fetch1]) @@ -216,7 +221,7 @@ def test_produce_consume_many(self): self.assertTrue(len(messages) < 100) # 10240 should be enough - fetch2 = FetchRequest("test_produce_consume_many", 0, 0, 10240) + fetch2 = FetchRequest(self.topic, 0, 0, 10240) (fetch_resp2,) = self.client.send_fetch_request([fetch2]) self.assertEquals(fetch_resp2.error, 0) @@ -229,10 +234,10 @@ def test_produce_consume_many(self): self.assertEquals(message.message.key, None) def test_produce_consume_two_partitions(self): - produce1 = ProduceRequest("test_produce_consume_two_partitions", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Partition 0 %d" % i) for i in range(10) ]) - produce2 = ProduceRequest("test_produce_consume_two_partitions", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Partition 1 %d" % i) for i in range(10) ]) @@ -240,8 +245,8 @@ def test_produce_consume_two_partitions(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - fetch1 = FetchRequest("test_produce_consume_two_partitions", 0, 0, 1024) - fetch2 = FetchRequest("test_produce_consume_two_partitions", 1, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) + fetch2 = FetchRequest(self.topic, 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) self.assertEquals(fetch_resp1.error, 0) self.assertEquals(fetch_resp1.highwaterMark, 10) @@ -266,11 +271,11 @@ def test_produce_consume_two_partitions(self): @unittest.skip('commmit offset not supported in this version') def test_commit_fetch_offsets(self): - req = OffsetCommitRequest("test_commit_fetch_offsets", 0, 42, "metadata") + req = OffsetCommitRequest(self.topic, 0, 42, "metadata") (resp,) = self.client.send_offset_commit_request("group", [req]) self.assertEquals(resp.error, 0) - req = OffsetFetchRequest("test_commit_fetch_offsets", 0) + req = OffsetFetchRequest(self.topic, 0) (resp,) = self.client.send_offset_fetch_request("group", [req]) self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 42) @@ -279,7 +284,7 @@ def test_commit_fetch_offsets(self): # Producer Tests def test_simple_producer(self): - producer = SimpleProducer(self.client, "test_simple_producer") + producer = SimpleProducer(self.client, self.topic) resp = producer.send_messages("one", "two") # Will go to partition 0 @@ -293,8 +298,8 @@ def test_simple_producer(self): self.assertEquals(resp[0].error, 0) self.assertEquals(resp[0].offset, 0) # offset of first msg - fetch1 = FetchRequest("test_simple_producer", 0, 0, 1024) - fetch2 = FetchRequest("test_simple_producer", 1, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) + fetch2 = FetchRequest(self.topic, 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) self.assertEquals(fetch_resp1.error, 0) @@ -318,15 +323,15 @@ def test_simple_producer(self): producer.stop() def test_round_robin_partitioner(self): - producer = KeyedProducer(self.client, "test_round_robin_partitioner", + producer = KeyedProducer(self.client, self.topic, partitioner=RoundRobinPartitioner) producer.send("key1", "one") producer.send("key2", "two") producer.send("key3", "three") producer.send("key4", "four") - fetch1 = FetchRequest("test_round_robin_partitioner", 0, 0, 1024) - fetch2 = FetchRequest("test_round_robin_partitioner", 1, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) + fetch2 = FetchRequest(self.topic, 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -352,15 +357,15 @@ def test_round_robin_partitioner(self): producer.stop() def test_hashed_partitioner(self): - producer = KeyedProducer(self.client, "test_hashed_partitioner", + producer = KeyedProducer(self.client, self.topic, partitioner=HashedPartitioner) producer.send(1, "one") producer.send(2, "two") producer.send(3, "three") producer.send(4, "four") - fetch1 = FetchRequest("test_hashed_partitioner", 0, 0, 1024) - fetch2 = FetchRequest("test_hashed_partitioner", 1, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) + fetch2 = FetchRequest(self.topic, 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -386,12 +391,12 @@ def test_hashed_partitioner(self): producer.stop() def test_acks_none(self): - producer = SimpleProducer(self.client, "test_acks_none", + producer = SimpleProducer(self.client, self.topic, req_acks=SimpleProducer.ACK_NOT_REQUIRED) resp = producer.send_messages("one") self.assertEquals(len(resp), 0) - fetch = FetchRequest("test_acks_none", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch]) self.assertEquals(fetch_resp[0].error, 0) @@ -405,12 +410,12 @@ def test_acks_none(self): producer.stop() def test_acks_local_write(self): - producer = SimpleProducer(self.client, "test_acks_local_write", + producer = SimpleProducer(self.client, self.topic, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE) resp = producer.send_messages("one") self.assertEquals(len(resp), 1) - fetch = FetchRequest("test_acks_local_write", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch]) self.assertEquals(fetch_resp[0].error, 0) @@ -425,12 +430,12 @@ def test_acks_local_write(self): def test_acks_cluster_commit(self): producer = SimpleProducer( - self.client, "test_acks_cluster_commit", + self.client, self.topic, req_acks=SimpleProducer.ACK_AFTER_CLUSTER_COMMIT) resp = producer.send_messages("one") self.assertEquals(len(resp), 1) - fetch = FetchRequest("test_acks_cluster_commit", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch]) self.assertEquals(fetch_resp[0].error, 0) @@ -444,16 +449,14 @@ def test_acks_cluster_commit(self): producer.stop() def test_async_simple_producer(self): - producer = SimpleProducer(self.client, "test_async_simple_producer", - async=True) - + producer = SimpleProducer(self.client, self.topic, async=True) resp = producer.send_messages("one") self.assertEquals(len(resp), 0) # Give it some time time.sleep(2) - fetch = FetchRequest("test_async_simple_producer", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch]) self.assertEquals(fetch_resp[0].error, 0) @@ -467,8 +470,7 @@ def test_async_simple_producer(self): producer.stop() def test_async_keyed_producer(self): - producer = KeyedProducer(self.client, "test_async_keyed_producer", - async=True) + producer = KeyedProducer(self.client, self.topic, async=True) resp = producer.send("key1", "one") self.assertEquals(len(resp), 0) @@ -476,7 +478,7 @@ def test_async_keyed_producer(self): # Give it some time time.sleep(2) - fetch = FetchRequest("test_async_keyed_producer", 0, 0, 1024) + fetch = FetchRequest(self.topic, 0, 0, 1024) fetch_resp = self.client.send_fetch_request([fetch]) self.assertEquals(fetch_resp[0].error, 0) @@ -490,7 +492,7 @@ def test_async_keyed_producer(self): producer.stop() def test_batched_simple_producer(self): - producer = SimpleProducer(self.client, "test_batched_simple_producer", + producer = SimpleProducer(self.client, self.topic, batch_send=True, batch_send_every_n=10, batch_send_every_t=20) @@ -505,8 +507,8 @@ def test_batched_simple_producer(self): # Give it some time time.sleep(2) - fetch1 = FetchRequest("test_batched_simple_producer", 0, 0, 1024) - fetch2 = FetchRequest("test_batched_simple_producer", 1, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) + fetch2 = FetchRequest(self.topic, 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -525,8 +527,8 @@ def test_batched_simple_producer(self): # Give it some time time.sleep(2) - fetch1 = FetchRequest("test_batched_simple_producer", 0, 0, 1024) - fetch2 = FetchRequest("test_batched_simple_producer", 1, 0, 1024) + fetch1 = FetchRequest(self.topic, 0, 0, 1024) + fetch2 = FetchRequest(self.topic, 1, 0, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -544,8 +546,8 @@ def test_batched_simple_producer(self): msgs = ["message-%d" % i for i in range(15, 17)] resp = producer.send_messages(*msgs) - fetch1 = FetchRequest("test_batched_simple_producer", 0, 5, 1024) - fetch2 = FetchRequest("test_batched_simple_producer", 1, 5, 1024) + fetch1 = FetchRequest(self.topic, 0, 5, 1024) + fetch2 = FetchRequest(self.topic, 1, 5, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -557,8 +559,8 @@ def test_batched_simple_producer(self): # Give it some time time.sleep(22) - fetch1 = FetchRequest("test_batched_simple_producer", 0, 5, 1024) - fetch2 = FetchRequest("test_batched_simple_producer", 1, 5, 1024) + fetch1 = FetchRequest(self.topic, 0, 5, 1024) + fetch2 = FetchRequest(self.topic, 1, 5, 1024) fetch_resp1, fetch_resp2 = self.client.send_fetch_request([fetch1, fetch2]) @@ -587,7 +589,7 @@ def tearDownClass(cls): # noqa def test_simple_consumer(self): # Produce 100 messages to partition 0 - produce1 = ProduceRequest("test_simple_consumer", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(100) ]) @@ -596,7 +598,7 @@ def test_simple_consumer(self): self.assertEquals(resp.offset, 0) # Produce 100 messages to partition 1 - produce2 = ProduceRequest("test_simple_consumer", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Test message 1 %d" % i) for i in range(100) ]) @@ -606,7 +608,7 @@ def test_simple_consumer(self): # Start a consumer consumer = SimpleConsumer(self.client, "group1", - "test_simple_consumer", auto_commit=False, + self.topic, auto_commit=False, iter_timeout=0) all_messages = [] for message in consumer: @@ -634,7 +636,7 @@ def test_simple_consumer(self): def test_simple_consumer_blocking(self): consumer = SimpleConsumer(self.client, "group1", - "test_simple_consumer_blocking", + self.topic, auto_commit=False, iter_timeout=0) # Blocking API @@ -645,7 +647,7 @@ def test_simple_consumer_blocking(self): self.assertEqual(len(messages), 0) # Send 10 messages - produce = ProduceRequest("test_simple_consumer_blocking", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -669,21 +671,21 @@ def test_simple_consumer_blocking(self): def test_simple_consumer_pending(self): # Produce 10 messages to partition 0 and 1 - produce1 = ProduceRequest("test_simple_consumer_pending", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) for resp in self.client.send_produce_request([produce1]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - produce2 = ProduceRequest("test_simple_consumer_pending", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Test message 1 %d" % i) for i in range(10) ]) for resp in self.client.send_produce_request([produce2]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer_pending", + consumer = SimpleConsumer(self.client, "group1", self.topic, auto_commit=False, iter_timeout=0) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) @@ -692,7 +694,7 @@ def test_simple_consumer_pending(self): def test_multi_process_consumer(self): # Produce 100 messages to partition 0 - produce1 = ProduceRequest("test_multi_process_consumer", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(100) ]) @@ -701,7 +703,7 @@ def test_multi_process_consumer(self): self.assertEquals(resp.offset, 0) # Produce 100 messages to partition 1 - produce2 = ProduceRequest("test_multi_process_consumer", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Test message 1 %d" % i) for i in range(100) ]) @@ -710,7 +712,7 @@ def test_multi_process_consumer(self): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = MultiProcessConsumer(self.client, "grp1", "test_multi_process_consumer", auto_commit=False) + consumer = MultiProcessConsumer(self.client, "grp1", self.topic, auto_commit=False) all_messages = [] for message in consumer: all_messages.append(message) @@ -727,7 +729,7 @@ def test_multi_process_consumer(self): self.assertEqual(len(messages), 0) # Send 10 messages - produce = ProduceRequest("test_multi_process_consumer", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -750,7 +752,7 @@ def test_multi_process_consumer(self): def test_multi_proc_pending(self): # Produce 10 messages to partition 0 and 1 - produce1 = ProduceRequest("test_multi_proc_pending", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -758,7 +760,7 @@ def test_multi_proc_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - produce2 = ProduceRequest("test_multi_proc_pending", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Test message 1 %d" % i) for i in range(10) ]) @@ -766,7 +768,7 @@ def test_multi_proc_pending(self): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - consumer = MultiProcessConsumer(self.client, "group1", "test_multi_proc_pending", auto_commit=False) + consumer = MultiProcessConsumer(self.client, "group1", self.topic, auto_commit=False) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -776,7 +778,7 @@ def test_multi_proc_pending(self): def test_large_messages(self): # Produce 10 "normal" size messages messages1 = [create_message(random_string(1024)) for i in range(10)] - produce1 = ProduceRequest("test_large_messages", 0, messages1) + produce1 = ProduceRequest(self.topic, 0, messages1) for resp in self.client.send_produce_request([produce1]): self.assertEquals(resp.error, 0) @@ -784,14 +786,14 @@ def test_large_messages(self): # Produce 10 messages that are large (bigger than default fetch size) messages2 = [create_message(random_string(5000)) for i in range(10)] - produce2 = ProduceRequest("test_large_messages", 0, messages2) + produce2 = ProduceRequest(self.topic, 0, messages2) for resp in self.client.send_produce_request([produce2]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 10) # Consumer should still get all of them - consumer = SimpleConsumer(self.client, "group1", "test_large_messages", + consumer = SimpleConsumer(self.client, "group1", self.topic, auto_commit=False, iter_timeout=0) all_messages = messages1 + messages2 for i, message in enumerate(consumer): @@ -801,7 +803,7 @@ def test_large_messages(self): # Produce 1 message that is too large (bigger than max fetch size) big_message_size = MAX_FETCH_BUFFER_SIZE_BYTES + 10 big_message = create_message(random_string(big_message_size)) - produce3 = ProduceRequest("test_large_messages", 0, [big_message]) + produce3 = ProduceRequest(self.topic, 0, [big_message]) for resp in self.client.send_produce_request([produce3]): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 20) @@ -809,7 +811,7 @@ def test_large_messages(self): self.assertRaises(ConsumerFetchSizeTooSmall, consumer.get_message, False, 0.1) # Create a consumer with no fetch size limit - big_consumer = SimpleConsumer(self.client, "group1", "test_large_messages", + big_consumer = SimpleConsumer(self.client, "group1", self.topic, max_buffer_size=None, partitions=[0], auto_commit=False, iter_timeout=0) @@ -843,8 +845,7 @@ def tearDown(self): self.zk.close() def test_switch_leader(self): - - key, topic, partition = random_string(5), 'test_switch_leader', 0 + key, topic, partition = random_string(5), self.topic, 0 producer = SimpleProducer(self.client, topic) for i in range(1, 4): @@ -875,8 +876,7 @@ def test_switch_leader(self): producer.stop() def test_switch_leader_async(self): - - key, topic, partition = random_string(5), 'test_switch_leader_async', 0 + key, topic, partition = random_string(5), self.topic, 0 producer = SimpleProducer(self.client, topic, async=True) for i in range(1, 4): @@ -927,11 +927,6 @@ def _count_messages(self, group, topic): client.close() return len(all_messages) - -def random_string(l): - s = "".join(random.choice(string.letters) for i in xrange(l)) - return s - if __name__ == "__main__": logging.basicConfig(level=logging.DEBUG) unittest.main() From bfece4d93900192a6343dbc2b7c07ed6cf048545 Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Wed, 15 Jan 2014 10:59:40 -0800 Subject: [PATCH 077/109] Version bump --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 8fd18289f..6c4e69e07 100644 --- a/setup.py +++ b/setup.py @@ -19,7 +19,7 @@ def run(self): setup( name="kafka-python", - version="0.8.1-1", + version="0.8.1-2", install_requires=["distribute"], tests_require=["tox"], From c7733e30dca6577f00f603e0018184a7c8b00ccf Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Wed, 15 Jan 2014 11:02:19 -0800 Subject: [PATCH 078/109] Update README.md too --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index b79882773..fcf522712 100644 --- a/README.md +++ b/README.md @@ -18,7 +18,7 @@ Copyright 2013, David Arthur under Apache License, v2.0. See `LICENSE` # Status I'm following the version numbers of Kafka, plus one number to indicate the -version of this project. The current version is 0.8.0-1. This version is under +version of this project. The current version is 0.8.1-2. This version is under development, APIs are subject to change. # Usage From c36cb618d2ba5fcba118dd8b87bf51f26074c8f1 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 15 Jan 2014 11:13:59 -0800 Subject: [PATCH 079/109] Fix offset increments: * Increment the offset before returning a message rather than when putting it in the internal queue. This prevents committing the wrong offsets. * In MultiProcessConsumer, store the offset of the next message --- kafka/consumer.py | 33 +++++++++++++++++---------------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 522d6cacd..657024f04 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -338,7 +338,19 @@ def get_message(self, block=True, timeout=0.1): with FetchContext(self, block, timeout): self._fetch() try: - return self.queue.get_nowait() + partition, message = self.queue.get_nowait() + + # Update partition offset + self.offsets[partition] = message.offset + 1 + + # Count, check and commit messages if necessary + self.count_since_commit += 1 + self._auto_commit() + + if self.partition_info: + return partition, message + else: + return message except Empty: return None @@ -380,18 +392,8 @@ def _fetch(self): partition = resp.partition try: for message in resp.messages: - # Update partition offset - self.offsets[partition] = message.offset + 1 - - # Count, check and commit messages if necessary - self.count_since_commit += 1 - self._auto_commit() - # Put the message in our queue - if self.partition_info: - self.queue.put((partition, message)) - else: - self.queue.put(message) + self.queue.put((partition, message)) except ConsumerFetchSizeTooSmall, e: if (self.max_buffer_size is not None and self.buffer_size == self.max_buffer_size): @@ -577,12 +579,11 @@ def __iter__(self): break # Count, check and commit messages if necessary - self.offsets[partition] = message.offset + self.offsets[partition] = message.offset + 1 self.start.clear() - yield message - self.count_since_commit += 1 self._auto_commit() + yield message self.start.clear() @@ -624,7 +625,7 @@ def get_messages(self, count=1, block=True, timeout=10): messages.append(message) # Count, check and commit messages if necessary - self.offsets[partition] = message.offset + self.offsets[partition] = message.offset + 1 self.count_since_commit += 1 self._auto_commit() count -= 1 From 61a0cb6f83e36172e2926c7c6813c7a56c857acc Mon Sep 17 00:00:00 2001 From: Thomas Dimson Date: Wed, 15 Jan 2014 11:29:27 -0800 Subject: [PATCH 080/109] 0.9.0 --- README.md | 5 ++--- setup.py | 3 ++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index fcf522712..9bf3a039b 100644 --- a/README.md +++ b/README.md @@ -17,9 +17,8 @@ Copyright 2013, David Arthur under Apache License, v2.0. See `LICENSE` # Status -I'm following the version numbers of Kafka, plus one number to indicate the -version of this project. The current version is 0.8.1-2. This version is under -development, APIs are subject to change. +The current version of this package is **0.9.0** and is compatible with +Kafka brokers running version **0.8.1**. # Usage diff --git a/setup.py b/setup.py index 6c4e69e07..0869fee3e 100644 --- a/setup.py +++ b/setup.py @@ -6,6 +6,7 @@ class Tox(Command): user_options = [] + def initialize_options(self): pass @@ -19,7 +20,7 @@ def run(self): setup( name="kafka-python", - version="0.8.1-2", + version="0.9.0", install_requires=["distribute"], tests_require=["tox"], From 8cc36dd7a1c7691e5c26b47cb667bc48054594a0 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 15 Jan 2014 14:28:48 -0800 Subject: [PATCH 081/109] Store fetched offsets separately. Fetch requests can be repeated if we get a ConsumerFetchSizeTooSmall or if _fetch() is called multiple times for some reason. We don't want to re-fetch messages that are already in our queue, so store the offsets of the last enqueued messages from each partition. --- kafka/consumer.py | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 657024f04..b202b230c 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -235,6 +235,12 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, buffer_size=FETCH_BUFFER_SIZE_BYTES, max_buffer_size=MAX_FETCH_BUFFER_SIZE_BYTES, iter_timeout=None): + super(SimpleConsumer, self).__init__( + client, group, topic, + partitions=partitions, + auto_commit=auto_commit, + auto_commit_every_n=auto_commit_every_n, + 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 " @@ -245,17 +251,10 @@ def __init__(self, client, group, topic, auto_commit=True, partitions=None, 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_started = defaultdict(bool) # defaults to false + self.fetch_offsets = self.offsets.copy() self.iter_timeout = iter_timeout self.queue = Queue() - super(SimpleConsumer, self).__init__( - client, group, topic, - partitions=partitions, - auto_commit=auto_commit, - auto_commit_every_n=auto_commit_every_n, - auto_commit_every_t=auto_commit_every_t) - def provide_partition_info(self): """ Indicates that partition info must be returned by the consumer @@ -301,6 +300,10 @@ def seek(self, offset, whence): else: raise ValueError("Unexpected value for `whence`, %d" % whence) + # Reset queue and fetch offsets since they are invalid + self.fetch_offsets = self.offsets.copy() + self.queue = Queue() + def get_messages(self, count=1, block=True, timeout=0.1): """ Fetch the specified number of messages @@ -375,11 +378,11 @@ def __iter__(self): def _fetch(self): # Create fetch request payloads for all the partitions requests = [] - partitions = self.offsets.keys() + partitions = self.fetch_offsets.keys() while partitions: for partition in partitions: requests.append(FetchRequest(self.topic, partition, - self.offsets[partition], + self.fetch_offsets[partition], self.buffer_size)) # Send request responses = self.client.send_fetch_request( @@ -394,6 +397,7 @@ def _fetch(self): for message in resp.messages: # Put the message in our queue self.queue.put((partition, message)) + self.fetch_offsets[partition] = message.offset + 1 except ConsumerFetchSizeTooSmall, e: if (self.max_buffer_size is not None and self.buffer_size == self.max_buffer_size): From 8b3793a649b470879d2684ad0a127c32a1348682 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 15 Jan 2014 15:54:06 -0800 Subject: [PATCH 082/109] Only use timeout if it's not None --- kafka/consumer.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index b202b230c..474e1f569 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -315,7 +315,7 @@ def get_messages(self, count=1, block=True, timeout=0.1): it will block forever. """ messages = [] - if timeout: + if timeout is not None: max_time = time.time() + timeout while count > 0 and (timeout is None or timeout > 0): @@ -328,7 +328,7 @@ def get_messages(self, count=1, block=True, timeout=0.1): if not block: # If we're not blocking, break. break - if timeout: + if timeout is not None: # If we're blocking and have a timeout, reduce it to the # appropriate value timeout = max_time - time.time() @@ -610,7 +610,7 @@ def get_messages(self, count=1, block=True, timeout=10): self.size.value = count self.pause.clear() - if timeout: + if timeout is not None: max_time = time.time() + timeout while count > 0 and (timeout is None or timeout > 0): @@ -633,7 +633,8 @@ def get_messages(self, count=1, block=True, timeout=10): self.count_since_commit += 1 self._auto_commit() count -= 1 - timeout = max_time - time.time() + if timeout is not None: + timeout = max_time - time.time() self.size.value = 0 self.start.clear() From e0f726204ab0b8b8ae5c29ae07c1aa369a5a6906 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Wed, 15 Jan 2014 16:22:41 -0800 Subject: [PATCH 083/109] Make get_messages() update and commit offsets just before returning --- kafka/consumer.py | 51 ++++++++++++++++++++++++++++++++--------------- 1 file changed, 35 insertions(+), 16 deletions(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 474e1f569..12e1af66b 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -1,6 +1,5 @@ from __future__ import absolute_import -from collections import defaultdict from itertools import izip_longest, repeat import logging import time @@ -318,10 +317,17 @@ def get_messages(self, count=1, block=True, timeout=0.1): if timeout is not None: max_time = time.time() + timeout + new_offsets = {} while count > 0 and (timeout is None or timeout > 0): - message = self.get_message(block, timeout) - if message: - messages.append(message) + result = self._get_message(block, timeout, 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. @@ -333,9 +339,17 @@ def get_messages(self, count=1, block=True, timeout=0.1): # appropriate value timeout = max_time - time.time() + # Update and commit offsets if necessary + self.offsets.update(new_offsets) + self.count_since_commit += len(messages) + self._auto_commit() return messages - def get_message(self, block=True, timeout=0.1): + def get_message(self, block=True, timeout=0.1, get_partition_info=None): + return self._get_message(block, timeout, get_partition_info) + + def _get_message(self, block=True, timeout=0.1, get_partition_info=None, + update_offset=True): if self.queue.empty(): # We're out of messages, go grab some more. with FetchContext(self, block, timeout): @@ -343,14 +357,17 @@ def get_message(self, block=True, timeout=0.1): try: partition, message = self.queue.get_nowait() - # Update partition offset - self.offsets[partition] = message.offset + 1 + if update_offset: + # Update partition offset + self.offsets[partition] = message.offset + 1 - # Count, check and commit messages if necessary - self.count_since_commit += 1 - self._auto_commit() + # Count, check and commit messages if necessary + self.count_since_commit += 1 + self._auto_commit() - if self.partition_info: + if get_partition_info is None: + get_partition_info = self.partition_info + if get_partition_info: return partition, message else: return message @@ -613,6 +630,7 @@ def get_messages(self, count=1, block=True, timeout=10): if timeout is not None: max_time = time.time() + timeout + new_offsets = {} while count > 0 and (timeout is None or timeout > 0): # Trigger consumption only if the queue is empty # By doing this, we will ensure that consumers do not @@ -627,11 +645,7 @@ def get_messages(self, count=1, block=True, timeout=10): break messages.append(message) - - # Count, check and commit messages if necessary - self.offsets[partition] = message.offset + 1 - self.count_since_commit += 1 - self._auto_commit() + new_offsets[partition] = message.offset + 1 count -= 1 if timeout is not None: timeout = max_time - time.time() @@ -640,4 +654,9 @@ def get_messages(self, count=1, block=True, timeout=10): self.start.clear() self.pause.set() + # Update and commit offsets if necessary + self.offsets.update(new_offsets) + self.count_since_commit += len(messages) + self._auto_commit() + return messages From 94f4c9b3e6dc76111d21efa170a547378db0df38 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 16 Jan 2014 13:52:28 -0800 Subject: [PATCH 084/109] Make the default connection timeout None This fixes the default behavior, which used to cause a socket timeout when waiting for 10 seconds for a message to be produced. --- kafka/conn.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index a6d93f8f7..e4aec5626 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -17,8 +17,13 @@ class KafkaConnection(local): by a call to `recv` in order to get the correct response. Eventually, we can do something in here to facilitate multiplexed requests/responses since the Kafka API includes a correlation id. + + host: the host name or IP address of a kafka broker + port: the port number the kafka broker is listening on + timeout: default None. The socket timeout for sending and receiving data. + None means no timeout, so a request can block forever. """ - def __init__(self, host, port, timeout=10): + def __init__(self, host, port, timeout=None): super(KafkaConnection, self).__init__() self.host = host self.port = port From ef24c117fee22d4f5ba9f71450974ddad598fafa Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 16 Jan 2014 14:57:24 -0800 Subject: [PATCH 085/109] Change default socket timeout to 120 seconds in both the client and connection --- kafka/client.py | 8 ++++++-- kafka/conn.py | 7 ++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 1016051b4..6bab575db 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -10,7 +10,7 @@ BrokerResponseError, PartitionUnavailableError, KafkaUnavailableError, KafkaRequestError) -from kafka.conn import KafkaConnection +from kafka.conn import KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS from kafka.protocol import KafkaProtocol log = logging.getLogger("kafka") @@ -21,7 +21,11 @@ class KafkaClient(object): CLIENT_ID = "kafka-python" ID_GEN = count() - def __init__(self, host, port, client_id=CLIENT_ID, timeout=10): + # 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, host, port, client_id=CLIENT_ID, + timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): # We need one connection to bootstrap self.client_id = client_id self.timeout = timeout diff --git a/kafka/conn.py b/kafka/conn.py index e4aec5626..2877c74e4 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -8,6 +8,7 @@ log = logging.getLogger("kafka") +DEFAULT_SOCKET_TIMEOUT_SECONDS = 120 class KafkaConnection(local): """ @@ -20,10 +21,10 @@ class KafkaConnection(local): host: the host name or IP address of a kafka broker port: the port number the kafka broker is listening on - timeout: default None. The socket timeout for sending and receiving data. - None means no timeout, so a request can block forever. + timeout: default 120. The socket timeout for sending and receiving data + in seconds. None means no timeout, so a request can block forever. """ - def __init__(self, host, port, timeout=None): + def __init__(self, host, port, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): super(KafkaConnection, self).__init__() self.host = host self.port = port From 6189bd46951580f1c0e280d289ab90dfc7d70b37 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Sat, 18 Jan 2014 09:27:08 -0500 Subject: [PATCH 086/109] added mock --- kafka/client.py | 7 +-- setup.py | 3 +- test/test_unit.py | 135 +++++++++++++++++++++++++++++++++++++++++++--- 3 files changed, 134 insertions(+), 11 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 155f65883..e6b3ca9fd 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -243,9 +243,10 @@ def load_metadata_for_topics(self, *topics): self.topic_partitions[topic] = [] for partition, meta in partitions.items(): - topic_part = TopicAndPartition(topic, partition) - self.topics_to_brokers[topic_part] = brokers[meta.leader] - self.topic_partitions[topic].append(partition) + if meta.leader != -1: + topic_part = TopicAndPartition(topic, partition) + self.topics_to_brokers[topic_part] = brokers[meta.leader] + self.topic_partitions[topic].append(partition) def send_produce_request(self, payloads=[], acks=1, timeout=1000, fail_on_error=True, callback=None): diff --git a/setup.py b/setup.py index 0869fee3e..fe951e7c1 100644 --- a/setup.py +++ b/setup.py @@ -1,4 +1,3 @@ -import os.path import sys from setuptools import setup, Command @@ -23,7 +22,7 @@ def run(self): version="0.9.0", install_requires=["distribute"], - tests_require=["tox"], + tests_require=["tox", "mock"], cmdclass={"test": Tox}, packages=["kafka"], diff --git a/test/test_unit.py b/test/test_unit.py index e3fd4bb5b..0d8f16982 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -6,7 +6,8 @@ from kafka.common import ( ProduceRequest, FetchRequest, Message, ChecksumError, ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, - OffsetAndMessage, BrokerMetadata, PartitionMetadata + OffsetAndMessage, BrokerMetadata, PartitionMetadata, + TopicAndPartition ) from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, @@ -16,6 +17,10 @@ create_gzip_message, create_message, create_snappy_message, KafkaProtocol ) +from kafka.client import KafkaClient + +from mock import patch + ITERATIONS = 1000 STRLEN = 100 @@ -87,8 +92,9 @@ def test_create_gzip(self): payloads = ["v1", "v2"] msg = create_gzip_message(payloads) self.assertEqual(msg.magic, 0) - self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK & - KafkaProtocol.CODEC_GZIP) + self.assertEqual( + msg.attributes, + KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_GZIP) self.assertEqual(msg.key, None) # Need to decode to check since gzipped payload is non-deterministic decoded = gzip_decode(msg.value) @@ -103,8 +109,9 @@ def test_create_snappy(self): payloads = ["v1", "v2"] msg = create_snappy_message(payloads) self.assertEqual(msg.magic, 0) - self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK & - KafkaProtocol.CODEC_SNAPPY) + self.assertEqual( + msg.attributes, + KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_SNAPPY) self.assertEqual(msg.key, None) expect = ("8\x00\x00\x19\x01@\x10L\x9f[\xc2\x00\x00\xff\xff\xff\xff" "\x00\x00\x00\x02v1\x19\x1bD\x00\x10\xd5\x96\nx\x00\x00\xff" @@ -275,6 +282,7 @@ def test_decode_fetch_response(self): len(ms3), ms3) responses = list(KafkaProtocol.decode_fetch_response(encoded)) + def expand_messages(response): return FetchResponse(response.topic, response.partition, response.error, response.highwaterMark, @@ -362,7 +370,6 @@ def test_encode_offset_request(self): def test_decode_offset_response(self): pass - @unittest.skip("Not Implemented") def test_encode_offset_commit_request(self): pass @@ -380,5 +387,121 @@ def test_decode_offset_fetch_response(self): pass +class TestClient(unittest.TestCase): + + #@unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_client_load_metadata(self, protocol, conn): + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(1, 'broker_1', 4567) + brokers[1] = BrokerMetadata(2, 'broker_2', 5678) + + topics = {} + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2]) + } + topics['topic_2'] = { + 0: PartitionMetadata('topic_2', 0, 0, [0, 1], [0, 1]), + 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(host='broker_1', port=4567) + self.assertItemsEqual({ + TopicAndPartition('topic_1', 0): brokers[0], + TopicAndPartition('topic_2', 0): brokers[0], + TopicAndPartition('topic_2', 1): brokers[1]}, + client.topics_to_brokers) + + # @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_client_load_metadata_unassigned_partitions(self, protocol, conn): + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(0, 'broker_1', 4567) + brokers[1] = BrokerMetadata(1, 'broker_2', 5678) + + topics = {} + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, -1, [], []) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(host='broker_1', port=4567) + + self.assertItemsEqual({}, client.topics_to_brokers) + self.assertRaises( + Exception, + client._get_leader_for_partition, + 'topic_1', 0) + + # calling _get_leader_for_partition (from any broker aware request) + # will try loading metadata again for the same topic + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) + } + leader = client._get_leader_for_partition('topic_1', 0) + + self.assertEqual(brokers[0], leader) + self.assertItemsEqual({ + TopicAndPartition('topic_1', 0): brokers[0]}, + client.topics_to_brokers) + + #@unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_client_load_metadata_noleader_partitions(self, protocol, conn): + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(0, 'broker_1', 4567) + brokers[1] = BrokerMetadata(1, 'broker_2', 5678) + + topics = {} + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, -1, [], []) + } + topics['topic_2'] = { + 0: PartitionMetadata('topic_2', 0, 0, [0, 1], []), + 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(host='broker_1', port=4567) + self.assertItemsEqual( + { + TopicAndPartition('topic_2', 0): brokers[0], + TopicAndPartition('topic_2', 1): brokers[1] + }, + client.topics_to_brokers) + self.assertRaises( + Exception, + client._get_leader_for_partition, + 'topic_1', 0) + + # calling _get_leader_for_partition (from any broker aware request) + # will try loading metadata again for the same topic + topics['topic_1'] = { + 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) + } + leader = client._get_leader_for_partition('topic_1', 0) + + self.assertEqual(brokers[0], leader) + self.assertItemsEqual( + { + TopicAndPartition('topic_1', 0): brokers[0], + TopicAndPartition('topic_2', 0): brokers[0], + TopicAndPartition('topic_2', 1): brokers[1] + }, + client.topics_to_brokers) + if __name__ == '__main__': unittest.main() From 9e2778efbb01753843c50d245378fb6a3d3ba989 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Sun, 19 Jan 2014 20:42:32 -0500 Subject: [PATCH 087/109] removed commented out lines --- test/test_unit.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/test/test_unit.py b/test/test_unit.py index 0d8f16982..602f5f57d 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -389,7 +389,6 @@ def test_decode_offset_fetch_response(self): class TestClient(unittest.TestCase): - #@unittest.skip('requires disabling recursion on _load_metadata_for_topics') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_client_load_metadata(self, protocol, conn): @@ -417,7 +416,6 @@ def test_client_load_metadata(self, protocol, conn): TopicAndPartition('topic_2', 1): brokers[1]}, client.topics_to_brokers) - # @unittest.skip('requires disabling recursion on _load_metadata_for_topics') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_client_load_metadata_unassigned_partitions(self, protocol, conn): @@ -454,7 +452,6 @@ def test_client_load_metadata_unassigned_partitions(self, protocol, conn): TopicAndPartition('topic_1', 0): brokers[0]}, client.topics_to_brokers) - #@unittest.skip('requires disabling recursion on _load_metadata_for_topics') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_client_load_metadata_noleader_partitions(self, protocol, conn): From bcd5539cb61f948a8ee51efc965c038d6604be9d Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Mon, 20 Jan 2014 11:41:44 -0800 Subject: [PATCH 088/109] Add doc string for SimpleConsumer._get_message() --- kafka/consumer.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/kafka/consumer.py b/kafka/consumer.py index 12e1af66b..8d2c2b093 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -350,6 +350,12 @@ def get_message(self, block=True, timeout=0.1, get_partition_info=None): def _get_message(self, block=True, timeout=0.1, get_partition_info=None, update_offset=True): + """ + If no messages can be fetched, returns None. + If get_partition_info is None, it defaults to self.partition_info + If get_partition_info is True, returns (partition, message) + If get_partition_info is False, returns message + """ if self.queue.empty(): # We're out of messages, go grab some more. with FetchContext(self, block, timeout): From a46898d01515989f01cc6d39107fd6beecc508f0 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Thu, 23 Jan 2014 19:34:09 -0800 Subject: [PATCH 089/109] Make producers take a topic argument at send rather than init time This allows a single producer to be used to send to multiple topics. See https://github.com/mumrah/kafka-python/issues/110 --- README.md | 26 ++++++------- kafka/producer.py | 74 +++++++++++++++++++----------------- test/test_integration.py | 82 ++++++++++++++++++++-------------------- 3 files changed, 94 insertions(+), 88 deletions(-) diff --git a/README.md b/README.md index 9bf3a039b..fbccc1ec4 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,7 @@ Copyright 2013, David Arthur under Apache License, v2.0. See `LICENSE` # Status -The current version of this package is **0.9.0** and is compatible with +The current version of this package is **0.9.0** and is compatible with Kafka brokers running version **0.8.1**. # Usage @@ -32,24 +32,24 @@ from kafka.producer import SimpleProducer, KeyedProducer kafka = KafkaClient("localhost", 9092) # To send messages synchronously -producer = SimpleProducer(kafka, "my-topic") -producer.send_messages("some message") -producer.send_messages("this method", "is variadic") +producer = SimpleProducer(kafka) +producer.send_messages("my-topic", "some message") +producer.send_messages("my-topic", "this method", "is variadic") # To send messages asynchronously -producer = SimpleProducer(kafka, "my-topic", async=True) -producer.send_messages("async message") +producer = SimpleProducer(kafka, async=True) +producer.send_messages("my-topic", "async message") # To wait for acknowledgements # ACK_AFTER_LOCAL_WRITE : server will wait till the data is written to # a local log before sending response # ACK_AFTER_CLUSTER_COMMIT : server will block until the message is committed # by all in sync replicas before sending a response -producer = SimpleProducer(kafka, "my-topic", async=False, +producer = SimpleProducer(kafka, async=False, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE, ack_timeout=2000) -response = producer.send_messages("async message") +response = producer.send_messages("my-topic", "async message") if response: print(response[0].error) @@ -62,7 +62,7 @@ if response: # 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, "my-topic", batch_send=True, +producer = SimpleProducer(kafka, batch_send=True, batch_send_every_n=20, batch_send_every_t=60) @@ -83,11 +83,11 @@ from kafka.partitioner import HashedPartitioner, RoundRobinPartitioner kafka = KafkaClient("localhost", 9092) # HashedPartitioner is default -producer = KeyedProducer(kafka, "my-topic") -producer.send("key1", "some message") -producer.send("key2", "this methode") +producer = KeyedProducer(kafka) +producer.send("my-topic", "key1", "some message") +producer.send("my-topic", "key2", "this methode") -producer = KeyedProducer(kafka, "my-topic", partitioner=RoundRobinPartitioner) +producer = KeyedProducer(kafka, partitioner=RoundRobinPartitioner) ``` ## Multiprocess consumer diff --git a/kafka/producer.py b/kafka/producer.py index 6b624f2d6..4279e6111 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -20,7 +20,7 @@ STOP_ASYNC_PRODUCER = -1 -def _send_upstream(topic, queue, client, batch_time, batch_size, +def _send_upstream(queue, client, batch_time, batch_size, req_acks, ack_timeout): """ Listen on the queue for a specified number of messages or till @@ -44,23 +44,24 @@ def _send_upstream(topic, queue, client, batch_time, batch_size, # timeout is reached while count > 0 and timeout >= 0: try: - partition, msg = queue.get(timeout=timeout) + topic, partition, msg = queue.get(timeout=timeout) + except Empty: break # Check if the controller has requested us to stop - if partition == STOP_ASYNC_PRODUCER: + if topic == STOP_ASYNC_PRODUCER: stop = True break # Adjust the timeout to match the remaining period count -= 1 timeout = send_at - time.time() - msgset[partition].append(msg) + msgset[(topic, partition)].append(msg) # Send collected requests upstream reqs = [] - for partition, messages in msgset.items(): + for (topic, partition), messages in msgset.items(): req = ProduceRequest(topic, partition, messages) reqs.append(req) @@ -78,7 +79,6 @@ class Producer(object): Params: client - The Kafka client instance to use - topic - The topic for sending messages to async - If set to 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 @@ -119,8 +119,7 @@ def __init__(self, client, async=False, if self.async: self.queue = Queue() # Messages are sent through this queue self.proc = Process(target=_send_upstream, - args=(self.topic, - self.queue, + args=(self.queue, self.client.copy(), batch_send_every_t, batch_send_every_n, @@ -131,17 +130,17 @@ def __init__(self, client, async=False, self.proc.daemon = True self.proc.start() - def send_messages(self, partition, *msg): + def send_messages(self, topic, partition, *msg): """ Helper method to send produce requests """ if self.async: for m in msg: - self.queue.put((partition, create_message(m))) + self.queue.put((topic, partition, create_message(m))) resp = [] else: messages = [create_message(m) for m in msg] - req = ProduceRequest(self.topic, partition, messages) + req = ProduceRequest(topic, partition, messages) try: resp = self.client.send_produce_request([req], acks=self.req_acks, timeout=self.ack_timeout) @@ -156,7 +155,7 @@ def stop(self, timeout=1): forcefully cleaning up. """ if self.async: - self.queue.put((STOP_ASYNC_PRODUCER, None)) + self.queue.put((STOP_ASYNC_PRODUCER, None, None)) self.proc.join(timeout) if self.proc.is_alive(): @@ -169,7 +168,6 @@ class SimpleProducer(Producer): Params: client - The Kafka client instance to use - topic - The topic for sending messages to 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 @@ -180,27 +178,31 @@ class SimpleProducer(Producer): 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 """ - def __init__(self, client, topic, async=False, + def __init__(self, client, async=False, req_acks=Producer.ACK_AFTER_LOCAL_WRITE, ack_timeout=Producer.DEFAULT_ACK_TIMEOUT, batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL): - self.topic = topic - client.load_metadata_for_topics(topic) - self.next_partition = cycle(client.topic_partitions[topic]) - + self.partition_cycles = {} super(SimpleProducer, self).__init__(client, async, req_acks, ack_timeout, batch_send, batch_send_every_n, batch_send_every_t) - def send_messages(self, *msg): - partition = self.next_partition.next() - return super(SimpleProducer, self).send_messages(partition, *msg) + def _next_partition(self, topic): + if topic not in self.partition_cycles: + if topic not in self.client.topic_partitions: + self.client.load_metadata_for_topics(topic) + self.partition_cycles[topic] = cycle(self.client.topic_partitions[topic]) + return self.partition_cycles[topic].next() + + def send_messages(self, topic, *msg): + partition = self._next_partition(topic) + return super(SimpleProducer, self).send_messages(topic, partition, *msg) def __repr__(self): - return '' % (self.topic, self.async) + return '' % self.async class KeyedProducer(Producer): @@ -209,7 +211,6 @@ class KeyedProducer(Producer): Args: client - The kafka client instance - topic - The kafka topic to send messages to 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 @@ -220,29 +221,34 @@ class KeyedProducer(Producer): 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 """ - def __init__(self, client, topic, partitioner=None, async=False, + def __init__(self, client, partitioner=None, async=False, req_acks=Producer.ACK_AFTER_LOCAL_WRITE, ack_timeout=Producer.DEFAULT_ACK_TIMEOUT, batch_send=False, batch_send_every_n=BATCH_SEND_MSG_COUNT, batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL): - self.topic = topic - client.load_metadata_for_topics(topic) - if not partitioner: partitioner = HashedPartitioner - - self.partitioner = partitioner(client.topic_partitions[topic]) + self.partitioner_class = partitioner + self.partitioners = {} super(KeyedProducer, self).__init__(client, async, req_acks, ack_timeout, batch_send, batch_send_every_n, batch_send_every_t) - def send(self, key, msg): - partitions = self.client.topic_partitions[self.topic] - partition = self.partitioner.partition(key, partitions) - return self.send_messages(partition, msg) + def _next_partition(self, topic, key): + if topic not in self.partitioners: + if topic not in self.client.topic_partitions: + self.client.load_metadata_for_topics(topic) + self.partitioners[topic] = \ + self.partitioner_class(self.client.topic_partitions[topic]) + partitioner = self.partitioners[topic] + return partitioner.partition(key, self.client.topic_partitions[topic]) + + def send(self, topic, key, msg): + partition = self._next_partition(topic, key) + return self.send_messages(topic, partition, msg) def __repr__(self): - return '' % (self.topic, self.async) + return '' % self.async diff --git a/test/test_integration.py b/test/test_integration.py index 5a2263013..d0da523eb 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -284,8 +284,8 @@ def test_commit_fetch_offsets(self): # Producer Tests def test_simple_producer(self): - producer = SimpleProducer(self.client, self.topic) - resp = producer.send_messages("one", "two") + producer = SimpleProducer(self.client) + resp = producer.send_messages(self.topic, "one", "two") # Will go to partition 0 self.assertEquals(len(resp), 1) @@ -293,7 +293,7 @@ def test_simple_producer(self): self.assertEquals(resp[0].offset, 0) # offset of first msg # Will go to partition 1 - resp = producer.send_messages("three") + resp = producer.send_messages(self.topic, "three") self.assertEquals(len(resp), 1) self.assertEquals(resp[0].error, 0) self.assertEquals(resp[0].offset, 0) # offset of first msg @@ -315,7 +315,7 @@ def test_simple_producer(self): self.assertEquals(messages[0].message.value, "three") # Will go to partition 0 - resp = producer.send_messages("four", "five") + resp = producer.send_messages(self.topic, "four", "five") self.assertEquals(len(resp), 1) self.assertEquals(resp[0].error, 0) self.assertEquals(resp[0].offset, 2) # offset of first msg @@ -323,12 +323,12 @@ def test_simple_producer(self): producer.stop() def test_round_robin_partitioner(self): - producer = KeyedProducer(self.client, self.topic, + producer = KeyedProducer(self.client, partitioner=RoundRobinPartitioner) - producer.send("key1", "one") - producer.send("key2", "two") - producer.send("key3", "three") - producer.send("key4", "four") + producer.send(self.topic, "key1", "one") + producer.send(self.topic, "key2", "two") + producer.send(self.topic, "key3", "three") + producer.send(self.topic, "key4", "four") fetch1 = FetchRequest(self.topic, 0, 0, 1024) fetch2 = FetchRequest(self.topic, 1, 0, 1024) @@ -357,12 +357,12 @@ def test_round_robin_partitioner(self): producer.stop() def test_hashed_partitioner(self): - producer = KeyedProducer(self.client, self.topic, + producer = KeyedProducer(self.client, partitioner=HashedPartitioner) - producer.send(1, "one") - producer.send(2, "two") - producer.send(3, "three") - producer.send(4, "four") + producer.send(self.topic, 1, "one") + producer.send(self.topic, 2, "two") + producer.send(self.topic, 3, "three") + producer.send(self.topic, 4, "four") fetch1 = FetchRequest(self.topic, 0, 0, 1024) fetch2 = FetchRequest(self.topic, 1, 0, 1024) @@ -391,9 +391,9 @@ def test_hashed_partitioner(self): producer.stop() def test_acks_none(self): - producer = SimpleProducer(self.client, self.topic, + producer = SimpleProducer(self.client, req_acks=SimpleProducer.ACK_NOT_REQUIRED) - resp = producer.send_messages("one") + resp = producer.send_messages(self.topic, "one") self.assertEquals(len(resp), 0) fetch = FetchRequest(self.topic, 0, 0, 1024) @@ -410,9 +410,9 @@ def test_acks_none(self): producer.stop() def test_acks_local_write(self): - producer = SimpleProducer(self.client, self.topic, + producer = SimpleProducer(self.client, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE) - resp = producer.send_messages("one") + resp = producer.send_messages(self.topic, "one") self.assertEquals(len(resp), 1) fetch = FetchRequest(self.topic, 0, 0, 1024) @@ -430,9 +430,9 @@ def test_acks_local_write(self): def test_acks_cluster_commit(self): producer = SimpleProducer( - self.client, self.topic, + self.client, req_acks=SimpleProducer.ACK_AFTER_CLUSTER_COMMIT) - resp = producer.send_messages("one") + resp = producer.send_messages(self.topic, "one") self.assertEquals(len(resp), 1) fetch = FetchRequest(self.topic, 0, 0, 1024) @@ -449,8 +449,8 @@ def test_acks_cluster_commit(self): producer.stop() def test_async_simple_producer(self): - producer = SimpleProducer(self.client, self.topic, async=True) - resp = producer.send_messages("one") + producer = SimpleProducer(self.client, async=True) + resp = producer.send_messages(self.topic, "one") self.assertEquals(len(resp), 0) # Give it some time @@ -470,9 +470,9 @@ def test_async_simple_producer(self): producer.stop() def test_async_keyed_producer(self): - producer = KeyedProducer(self.client, self.topic, async=True) + producer = KeyedProducer(self.client, async=True) - resp = producer.send("key1", "one") + resp = producer.send(self.topic, "key1", "one") self.assertEquals(len(resp), 0) # Give it some time @@ -492,14 +492,14 @@ def test_async_keyed_producer(self): producer.stop() def test_batched_simple_producer(self): - producer = SimpleProducer(self.client, self.topic, + producer = SimpleProducer(self.client, batch_send=True, batch_send_every_n=10, batch_send_every_t=20) # Send 5 messages and do a fetch msgs = ["message-%d" % i for i in range(0, 5)] - resp = producer.send_messages(*msgs) + resp = producer.send_messages(self.topic, *msgs) # Batch mode is async. No ack self.assertEquals(len(resp), 0) @@ -522,7 +522,7 @@ def test_batched_simple_producer(self): # Send 5 more messages, wait for 2 seconds and do a fetch msgs = ["message-%d" % i for i in range(5, 10)] - resp = producer.send_messages(*msgs) + resp = producer.send_messages(self.topic, *msgs) # Give it some time time.sleep(2) @@ -542,9 +542,9 @@ def test_batched_simple_producer(self): # Send 7 messages and wait for 20 seconds msgs = ["message-%d" % i for i in range(10, 15)] - resp = producer.send_messages(*msgs) + resp = producer.send_messages(self.topic, *msgs) msgs = ["message-%d" % i for i in range(15, 17)] - resp = producer.send_messages(*msgs) + resp = producer.send_messages(self.topic, *msgs) fetch1 = FetchRequest(self.topic, 0, 5, 1024) fetch2 = FetchRequest(self.topic, 1, 5, 1024) @@ -846,25 +846,25 @@ def tearDown(self): def test_switch_leader(self): key, topic, partition = random_string(5), self.topic, 0 - producer = SimpleProducer(self.client, topic) + producer = SimpleProducer(self.client) for i in range(1, 4): # XXX unfortunately, the conns dict needs to be warmed for this to work # XXX unfortunately, for warming to work, we need at least as many partitions as brokers - self._send_random_messages(producer, 10) + self._send_random_messages(producer, self.topic, 10) # kil leader for partition 0 broker = self._kill_leader(topic, partition) # expect failure, reload meta data with self.assertRaises(FailedPayloadsError): - producer.send_messages('part 1') - producer.send_messages('part 2') + producer.send_messages(self.topic, 'part 1') + producer.send_messages(self.topic, 'part 2') time.sleep(1) # send to new leader - self._send_random_messages(producer, 10) + self._send_random_messages(producer, self.topic, 10) broker.open() time.sleep(3) @@ -877,22 +877,22 @@ def test_switch_leader(self): def test_switch_leader_async(self): key, topic, partition = random_string(5), self.topic, 0 - producer = SimpleProducer(self.client, topic, async=True) + producer = SimpleProducer(self.client, async=True) for i in range(1, 4): - self._send_random_messages(producer, 10) + self._send_random_messages(producer, self.topic, 10) # kil leader for partition 0 broker = self._kill_leader(topic, partition) # expect failure, reload meta data - producer.send_messages('part 1') - producer.send_messages('part 2') + producer.send_messages(self.topic, 'part 1') + producer.send_messages(self.topic, 'part 2') time.sleep(1) # send to new leader - self._send_random_messages(producer, 10) + self._send_random_messages(producer, self.topic, 10) broker.open() time.sleep(3) @@ -903,9 +903,9 @@ def test_switch_leader_async(self): producer.stop() - def _send_random_messages(self, producer, n): + def _send_random_messages(self, producer, topic, n): for j in range(n): - resp = producer.send_messages(random_string(10)) + resp = producer.send_messages(topic, random_string(10)) if len(resp) > 0: self.assertEquals(resp[0].error, 0) time.sleep(1) # give it some time From f6df696e0ab11ec931283dcca8c518cd54d57687 Mon Sep 17 00:00:00 2001 From: Omar Ghishan Date: Mon, 27 Jan 2014 11:16:53 -0800 Subject: [PATCH 090/109] Use TopicAndPartition when producing async messages --- kafka/producer.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/kafka/producer.py b/kafka/producer.py index 4279e6111..12a293401 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -8,7 +8,7 @@ from itertools import cycle from multiprocessing import Queue, Process -from kafka.common import ProduceRequest +from kafka.common import ProduceRequest, TopicAndPartition from kafka.partitioner import HashedPartitioner from kafka.protocol import create_message @@ -44,25 +44,27 @@ def _send_upstream(queue, client, batch_time, batch_size, # timeout is reached while count > 0 and timeout >= 0: try: - topic, partition, msg = queue.get(timeout=timeout) + topic_partition, msg = queue.get(timeout=timeout) except Empty: break # Check if the controller has requested us to stop - if topic == STOP_ASYNC_PRODUCER: + if topic_partition == STOP_ASYNC_PRODUCER: stop = True break # 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) # Send collected requests upstream reqs = [] - for (topic, partition), messages in msgset.items(): - req = ProduceRequest(topic, partition, messages) + for topic_partition, messages in msgset.items(): + req = ProduceRequest(topic_partition.topic, + topic_partition.partition, + messages) reqs.append(req) try: @@ -136,7 +138,8 @@ def send_messages(self, topic, partition, *msg): """ if self.async: for m in msg: - self.queue.put((topic, partition, create_message(m))) + self.queue.put((TopicAndPartition(topic, partition), + create_message(m))) resp = [] else: messages = [create_message(m) for m in msg] @@ -155,7 +158,7 @@ def stop(self, timeout=1): forcefully cleaning up. """ if self.async: - self.queue.put((STOP_ASYNC_PRODUCER, None, None)) + self.queue.put((STOP_ASYNC_PRODUCER, None)) self.proc.join(timeout) if self.proc.is_alive(): From 8bcf0f0940a94ddb2ee44a6edb333ca0d8595913 Mon Sep 17 00:00:00 2001 From: Marc Labbe Date: Fri, 31 Jan 2014 20:06:30 -0500 Subject: [PATCH 091/109] Handle cases for partition with leader=-1 (not defined) --- kafka/client.py | 18 +++++--- kafka/common.py | 4 -- test/test_unit.py | 113 ++++++++++++++++++++++++++-------------------- 3 files changed, 77 insertions(+), 58 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index e6b3ca9fd..a76bf479d 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -8,7 +8,7 @@ from kafka.common import (ErrorMapping, TopicAndPartition, ConnectionError, FailedPayloadsError, BrokerResponseError, PartitionUnavailableError, - KafkaUnavailableError, KafkaRequestError) + KafkaUnavailableError) from kafka.conn import KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS from kafka.protocol import KafkaProtocol @@ -53,11 +53,13 @@ def _get_conn_for_broker(self, broker): def _get_leader_for_partition(self, topic, partition): key = TopicAndPartition(topic, partition) - if key not in self.topics_to_brokers: + # reload metadata whether the partition is not available + # or has not leader (broker is None) + if self.topics_to_brokers.get(key) is None: self.load_metadata_for_topics(topic) if key not in self.topics_to_brokers: - raise KafkaRequestError("Partition does not exist: %s" % str(key)) + raise PartitionUnavailableError("No leader for %s" % str(key)) return self.topics_to_brokers[key] @@ -239,14 +241,18 @@ def load_metadata_for_topics(self, *topics): self.reset_topic_metadata(topic) if not partitions: + log.info('No partitions for %s', topic) continue self.topic_partitions[topic] = [] for partition, meta in partitions.items(): - if meta.leader != -1: - topic_part = TopicAndPartition(topic, partition) + self.topic_partitions[topic].append(partition) + topic_part = TopicAndPartition(topic, partition) + if meta.leader == -1: + log.info('No leader for topic %s partition %d', topic, partition) + self.topics_to_brokers[topic_part] = None + else: self.topics_to_brokers[topic_part] = brokers[meta.leader] - self.topic_partitions[topic].append(partition) def send_produce_request(self, payloads=[], acks=1, timeout=1000, fail_on_error=True, callback=None): diff --git a/kafka/common.py b/kafka/common.py index c0a1a6a1a..ec0b89b99 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -74,10 +74,6 @@ class KafkaError(RuntimeError): pass -class KafkaRequestError(KafkaError): - pass - - class KafkaUnavailableError(KafkaError): pass diff --git a/test/test_unit.py b/test/test_unit.py index 602f5f57d..b8af24271 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -7,7 +7,7 @@ ProduceRequest, FetchRequest, Message, ChecksumError, ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, OffsetAndMessage, BrokerMetadata, PartitionMetadata, - TopicAndPartition + TopicAndPartition, PartitionUnavailableError ) from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, @@ -55,7 +55,6 @@ def test_submodule_namespace(self): from kafka import KafkaClient as KafkaClient2 self.assertEquals(KafkaClient2.__name__, "KafkaClient") - from kafka.codec import snappy_encode self.assertEquals(snappy_encode.__name__, "snappy_encode") @@ -391,7 +390,8 @@ class TestClient(unittest.TestCase): @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') - def test_client_load_metadata(self, protocol, conn): + def test_load_metadata(self, protocol, conn): + "Load metadata for all topics" conn.recv.return_value = 'response' # anything but None @@ -403,22 +403,33 @@ def test_client_load_metadata(self, protocol, conn): topics['topic_1'] = { 0: PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2]) } - topics['topic_2'] = { - 0: PartitionMetadata('topic_2', 0, 0, [0, 1], [0, 1]), - 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) + topics['topic_noleader'] = { + 0: PartitionMetadata('topic_noleader', 0, -1, [], []), + 1: PartitionMetadata('topic_noleader', 1, -1, [], []) + } + topics['topic_no_partitions'] = {} + topics['topic_3'] = { + 0: PartitionMetadata('topic_3', 0, 0, [0, 1], [0, 1]), + 1: PartitionMetadata('topic_3', 1, 1, [1, 0], [1, 0]), + 2: PartitionMetadata('topic_3', 2, 0, [0, 1], [0, 1]) } protocol.decode_metadata_response.return_value = (brokers, topics) + # client loads metadata at init client = KafkaClient(host='broker_1', port=4567) self.assertItemsEqual({ TopicAndPartition('topic_1', 0): brokers[0], - TopicAndPartition('topic_2', 0): brokers[0], - TopicAndPartition('topic_2', 1): 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]}, client.topics_to_brokers) @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') - def test_client_load_metadata_unassigned_partitions(self, protocol, conn): + def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn): + "Get leader for partitions reload metadata if it is not available" conn.recv.return_value = 'response' # anything but None @@ -426,35 +437,32 @@ def test_client_load_metadata_unassigned_partitions(self, protocol, conn): brokers[0] = BrokerMetadata(0, 'broker_1', 4567) brokers[1] = BrokerMetadata(1, 'broker_2', 5678) - topics = {} - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, -1, [], []) - } + topics = {'topic_no_partitions': {}} protocol.decode_metadata_response.return_value = (brokers, topics) client = KafkaClient(host='broker_1', port=4567) + # topic metadata is loaded but empty self.assertItemsEqual({}, client.topics_to_brokers) - self.assertRaises( - Exception, - client._get_leader_for_partition, - 'topic_1', 0) + + topics['topic_no_partitions'] = { + 0: PartitionMetadata('topic_no_partitions', 0, 0, [0, 1], [0, 1]) + } + protocol.decode_metadata_response.return_value = (brokers, topics) # calling _get_leader_for_partition (from any broker aware request) # will try loading metadata again for the same topic - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) - } - leader = client._get_leader_for_partition('topic_1', 0) + leader = client._get_leader_for_partition('topic_no_partitions', 0) self.assertEqual(brokers[0], leader) self.assertItemsEqual({ - TopicAndPartition('topic_1', 0): brokers[0]}, + TopicAndPartition('topic_no_partitions', 0): brokers[0]}, client.topics_to_brokers) @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') - def test_client_load_metadata_noleader_partitions(self, protocol, conn): + def test_get_leader_for_unassigned_partitions(self, protocol, conn): + "Get leader raises if no partitions is defined for a topic" conn.recv.return_value = 'response' # anything but None @@ -462,43 +470,52 @@ def test_client_load_metadata_noleader_partitions(self, protocol, conn): brokers[0] = BrokerMetadata(0, 'broker_1', 4567) brokers[1] = BrokerMetadata(1, 'broker_2', 5678) - topics = {} - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, -1, [], []) - } - topics['topic_2'] = { - 0: PartitionMetadata('topic_2', 0, 0, [0, 1], []), - 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) - } + topics = {'topic_no_partitions': {}} protocol.decode_metadata_response.return_value = (brokers, topics) client = KafkaClient(host='broker_1', port=4567) - self.assertItemsEqual( - { - TopicAndPartition('topic_2', 0): brokers[0], - TopicAndPartition('topic_2', 1): brokers[1] - }, - client.topics_to_brokers) + + self.assertItemsEqual({}, client.topics_to_brokers) self.assertRaises( - Exception, + PartitionUnavailableError, client._get_leader_for_partition, - 'topic_1', 0) + 'topic_no_partitions', 0) - # calling _get_leader_for_partition (from any broker aware request) - # will try loading metadata again for the same topic - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_get_leader_returns_none_when_noleader(self, protocol, conn): + "Getting leader for partitions returns None when the partiion has no leader" + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(0, 'broker_1', 4567) + brokers[1] = BrokerMetadata(1, 'broker_2', 5678) + + topics = {} + topics['topic_noleader'] = { + 0: PartitionMetadata('topic_noleader', 0, -1, [], []), + 1: PartitionMetadata('topic_noleader', 1, -1, [], []) } - leader = client._get_leader_for_partition('topic_1', 0) + protocol.decode_metadata_response.return_value = (brokers, topics) - self.assertEqual(brokers[0], leader) + client = KafkaClient(host='broker_1', port=4567) self.assertItemsEqual( { - TopicAndPartition('topic_1', 0): brokers[0], - TopicAndPartition('topic_2', 0): brokers[0], - TopicAndPartition('topic_2', 1): brokers[1] + TopicAndPartition('topic_noleader', 0): None, + TopicAndPartition('topic_noleader', 1): None }, client.topics_to_brokers) + self.assertIsNone(client._get_leader_for_partition('topic_noleader', 0)) + self.assertIsNone(client._get_leader_for_partition('topic_noleader', 1)) + + topics['topic_noleader'] = { + 0: PartitionMetadata('topic_noleader', 0, 0, [0, 1], [0, 1]), + 1: PartitionMetadata('topic_noleader', 1, 1, [1, 0], [1, 0]) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + self.assertEqual(brokers[0], client._get_leader_for_partition('topic_noleader', 0)) + self.assertEqual(brokers[1], client._get_leader_for_partition('topic_noleader', 1)) if __name__ == '__main__': unittest.main() From b253166bec5a7d836767523b1ad5275eeed6b83f Mon Sep 17 00:00:00 2001 From: mrtheb Date: Fri, 31 Jan 2014 22:08:08 -0500 Subject: [PATCH 092/109] check for broker None in send_broker_aware_request (added test for it) --- kafka/client.py | 19 ++++++++++++++----- test/test_unit.py | 28 ++++++++++++++++++++++++++++ 2 files changed, 42 insertions(+), 5 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index a76bf479d..fbbff255a 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -52,14 +52,22 @@ def _get_conn_for_broker(self, broker): return self.conns[(broker.host, broker.port)] def _get_leader_for_partition(self, topic, partition): + """ + Returns the leader for a partition or None if the partition exists + but has no leader. + + PartitionUnavailableError will be raised if the topic or partition + is not part of the metadata. + """ + key = TopicAndPartition(topic, partition) # reload metadata whether the partition is not available - # or has not leader (broker is None) + # or has no leader (broker is None) if self.topics_to_brokers.get(key) is None: self.load_metadata_for_topics(topic) if key not in self.topics_to_brokers: - raise PartitionUnavailableError("No leader for %s" % str(key)) + raise PartitionUnavailableError("%s not available" % str(key)) return self.topics_to_brokers[key] @@ -115,8 +123,9 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): for payload in payloads: leader = self._get_leader_for_partition(payload.topic, payload.partition) - if leader == -1: - raise PartitionUnavailableError("Leader is unassigned for %s-%s" % payload.topic, payload.partition) + if leader is None: + raise PartitionUnavailableError( + "No leader for topic %s partition %s" % (payload.topic, payload.partition)) payloads_by_broker[leader].append(payload) original_keys.append((payload.topic, payload.partition)) @@ -249,7 +258,7 @@ def load_metadata_for_topics(self, *topics): self.topic_partitions[topic].append(partition) topic_part = TopicAndPartition(topic, partition) if meta.leader == -1: - log.info('No leader for topic %s partition %d', topic, partition) + log.info('No leader for topic %s partition %s', topic, partition) self.topics_to_brokers[topic_part] = None else: self.topics_to_brokers[topic_part] = brokers[meta.leader] diff --git a/test/test_unit.py b/test/test_unit.py index b8af24271..f0edd161e 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -517,5 +517,33 @@ def test_get_leader_returns_none_when_noleader(self, protocol, conn): self.assertEqual(brokers[0], client._get_leader_for_partition('topic_noleader', 0)) self.assertEqual(brokers[1], client._get_leader_for_partition('topic_noleader', 1)) + @patch('kafka.client.KafkaConnection') + @patch('kafka.client.KafkaProtocol') + def test_send_produce_request_raises_when_noleader(self, protocol, conn): + "Getting leader for partitions returns None when the partiion has no leader" + + conn.recv.return_value = 'response' # anything but None + + brokers = {} + brokers[0] = BrokerMetadata(0, 'broker_1', 4567) + brokers[1] = BrokerMetadata(1, 'broker_2', 5678) + + topics = {} + topics['topic_noleader'] = { + 0: PartitionMetadata('topic_noleader', 0, -1, [], []), + 1: PartitionMetadata('topic_noleader', 1, -1, [], []) + } + protocol.decode_metadata_response.return_value = (brokers, topics) + + client = KafkaClient(host='broker_1', port=4567) + + requests = [ProduceRequest( + "topic_noleader", 0, + [create_message("a"), create_message("b")])] + + self.assertRaises( + PartitionUnavailableError, + client.send_produce_request, requests) + if __name__ == '__main__': unittest.main() From a2191e5be5d5fcd212582580c163f4533cca6c73 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Sun, 9 Feb 2014 13:44:47 -0500 Subject: [PATCH 093/109] Support list (or comma-separated) of hosts (replaces host and port arguments) --- kafka/client.py | 13 +++++++------ kafka/conn.py | 5 ++++- test/test_integration.py | 26 +++++++++++++------------- test/test_unit.py | 40 ++++++++++++++++++++++++++++++++-------- 4 files changed, 56 insertions(+), 28 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 33c4419f7..96cc1dfbd 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -10,7 +10,7 @@ BrokerResponseError, PartitionUnavailableError, KafkaUnavailableError, KafkaRequestError) -from kafka.conn import KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS +from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS from kafka.protocol import KafkaProtocol log = logging.getLogger("kafka") @@ -24,14 +24,15 @@ class KafkaClient(object): # 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, host, port, client_id=CLIENT_ID, + 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.timeout = timeout - self.conns = { # (host, port) -> KafkaConnection - (host, port): KafkaConnection(host, port, timeout=timeout) - } + self.hosts = collect_hosts(hosts) + + # create connections only when we need them + self.conns = {} self.brokers = {} # broker_id -> BrokerMetadata self.topics_to_brokers = {} # topic_id -> broker_id self.topic_partitions = {} # topic_id -> [0, 1, 2, ...] @@ -46,7 +47,7 @@ def _get_conn(self, host, port): host_key = (host, port) if host_key not in self.conns: - self.conns[host_key] = KafkaConnection(host, port, self.bufsize) + self.conns[host_key] = KafkaConnection(host, port) return self.conns[host_key] diff --git a/kafka/conn.py b/kafka/conn.py index de2d385fa..20f22dcb1 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -17,8 +17,11 @@ def collect_hosts(hosts, randomize=True): randomize the returned list. """ + if isinstance(hosts, str): + hosts = hosts.split(',') + result = [] - for host_port in hosts.split(","): + for host_port in hosts: res = host_port.split(':') host = res[0] diff --git a/test/test_integration.py b/test/test_integration.py index 000f44aac..3d6ccf60b 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -33,7 +33,7 @@ def ensure_topic_creation(client, topic_name): class KafkaTestCase(unittest.TestCase): def setUp(self): - self.topic = "%s-%s" % (self.id()[self.id().rindex(".")+1:], random_string(10)) + self.topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10)) ensure_topic_creation(self.client, self.topic) @@ -578,7 +578,7 @@ def setUpClass(cls): cls.zk = ZookeeperFixture.instance() cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) - cls.client = KafkaClient('%s:%d' % (cls.server2.host, cls.server2.port), bufsize=8192) + cls.client = KafkaClient('%s:%d' % (cls.server2.host, cls.server2.port)) @classmethod def tearDownClass(cls): # noqa @@ -800,7 +800,6 @@ def test_large_messages(self): self.assertEquals(all_messages[i], message.message) self.assertEquals(i, 19) - # Produce 1 message that is too large (bigger than max fetch size) big_message_size = MAX_FETCH_BUFFER_SIZE_BYTES + 10 big_message = create_message(random_string(big_message_size)) @@ -827,25 +826,26 @@ def test_large_messages(self): class TestFailover(KafkaTestCase): - def setUp(self): + @classmethod + def setUpClass(cls): # noqa zk_chroot = random_string(10) replicas = 2 partitions = 2 # mini zookeeper, 2 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)] + 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)] - hosts = ','.join(['%s:%d' % (b.host, b.port) for b in cls.brokers]) + hosts = ['%s:%d' % (b.host, b.port) for b in cls.brokers] cls.client = KafkaClient(hosts) - super(TestFailover, self).setUp() - def tearDown(self): - self.client.close() - for broker in self.brokers: + @classmethod + def tearDownClass(cls): + cls.client.close() + for broker in cls.brokers: broker.close() - self.zk.close() + cls.zk.close() def test_switch_leader(self): key, topic, partition = random_string(5), self.topic, 0 diff --git a/test/test_unit.py b/test/test_unit.py index 4c78c1bbf..624fe39ed 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -5,11 +5,13 @@ from mock import patch +from kafka import KafkaClient from kafka.common import ( ProduceRequest, FetchRequest, Message, ChecksumError, ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, OffsetAndMessage, BrokerMetadata, PartitionMetadata ) +from kafka.common import KafkaUnavailableError from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, snappy_encode, snappy_decode @@ -384,6 +386,26 @@ def test_decode_offset_fetch_response(self): class TestKafkaClient(unittest.TestCase): + def test_init_with_list(self): + + with patch.object(KafkaClient, 'load_metadata_for_topics'): + client = KafkaClient( + hosts=['kafka01:9092', 'kafka02:9092', 'kafka03:9092']) + + self.assertItemsEqual( + [('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)], + client.hosts) + + def test_init_with_csv(self): + + with patch.object(KafkaClient, 'load_metadata_for_topics'): + client = KafkaClient( + hosts='kafka01:9092,kafka02:9092,kafka03:9092') + + self.assertItemsEqual( + [('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)], + client.hosts) + def test_send_broker_unaware_request_fail(self): 'Tests that call fails when all hosts are unavailable' @@ -402,14 +424,16 @@ def mock_get_conn(host, port): return mocked_conns[(host, port)] # patch to avoid making requests before we want it - with patch.object(KafkaClient, '_load_metadata_for_topics'), \ + with patch.object(KafkaClient, 'load_metadata_for_topics'), \ patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): - client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + client = KafkaClient(hosts=['kafka01:9092','kafka02:9092']) - resp = client._send_broker_unaware_request(1, 'fake request') - self.assertIsNone(resp) + self.assertRaises( + KafkaUnavailableError, + client._send_broker_unaware_request, + 1, 'fake request') for key, conn in mocked_conns.iteritems(): conn.send.assert_called_with(1, 'fake request') @@ -434,7 +458,7 @@ def mock_get_conn(host, port): return mocked_conns[(host, port)] # patch to avoid making requests before we want it - with patch.object(KafkaClient, '_load_metadata_for_topics'), \ + with patch.object(KafkaClient, 'load_metadata_for_topics'), \ patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): client = KafkaClient(hosts='kafka01:9092,kafka02:9092') @@ -444,7 +468,7 @@ def mock_get_conn(host, port): self.assertEqual('valid response', resp) mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) - @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @unittest.skip('requires disabling recursion on load_metadata_for_topics') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_client_load_metadata(self, protocol, conn): @@ -474,7 +498,7 @@ def test_client_load_metadata(self, protocol, conn): }, client.topics_to_brokers) - @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @unittest.skip('requires disabling recursion on load_metadata_for_topics') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_client_load_metadata_unassigned_partitions(self, protocol, conn): @@ -513,7 +537,7 @@ def test_client_load_metadata_unassigned_partitions(self, protocol, conn): }, client.topics_to_brokers) - @unittest.skip('requires disabling recursion on _load_metadata_for_topics') + @unittest.skip('requires disabling recursion on load_metadata_for_topics') @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_client_load_metadata_noleader_partitions(self, protocol, conn): From 5e5d7097a70e79d90d573aef6a0adba40ada5f03 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Sat, 15 Feb 2014 11:00:35 -0500 Subject: [PATCH 094/109] Changes based on comments by @rdiomar, plus added LeaderUnavailableError for clarity --- kafka/client.py | 11 +++++++---- kafka/common.py | 4 ++++ test/test_unit.py | 8 ++++---- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index fbbff255a..c3606e45a 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -8,6 +8,7 @@ from kafka.common import (ErrorMapping, TopicAndPartition, ConnectionError, FailedPayloadsError, BrokerResponseError, PartitionUnavailableError, + LeaderUnavailableError, KafkaUnavailableError) from kafka.conn import KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS @@ -124,8 +125,10 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): leader = self._get_leader_for_partition(payload.topic, payload.partition) if leader is None: - raise PartitionUnavailableError( - "No leader for topic %s partition %s" % (payload.topic, payload.partition)) + raise LeaderUnavailableError( + "Leader not available for topic %s partition %s" % + (payload.topic, payload.partition)) + payloads_by_broker[leader].append(payload) original_keys.append((payload.topic, payload.partition)) @@ -250,7 +253,7 @@ def load_metadata_for_topics(self, *topics): self.reset_topic_metadata(topic) if not partitions: - log.info('No partitions for %s', topic) + log.warning('No partitions for %s', topic) continue self.topic_partitions[topic] = [] @@ -258,7 +261,7 @@ def load_metadata_for_topics(self, *topics): self.topic_partitions[topic].append(partition) topic_part = TopicAndPartition(topic, partition) if meta.leader == -1: - log.info('No leader for topic %s partition %s', topic, partition) + log.warning('No leader for topic %s partition %s', topic, partition) self.topics_to_brokers[topic_part] = None else: self.topics_to_brokers[topic_part] = brokers[meta.leader] diff --git a/kafka/common.py b/kafka/common.py index ec0b89b99..b4fe5c745 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -82,6 +82,10 @@ class BrokerResponseError(KafkaError): pass +class LeaderUnavailableError(KafkaError): + pass + + class PartitionUnavailableError(KafkaError): pass diff --git a/test/test_unit.py b/test/test_unit.py index f0edd161e..1439d8b5b 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -7,7 +7,7 @@ ProduceRequest, FetchRequest, Message, ChecksumError, ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, OffsetAndMessage, BrokerMetadata, PartitionMetadata, - TopicAndPartition, PartitionUnavailableError + TopicAndPartition, LeaderUnavailableError, PartitionUnavailableError ) from kafka.codec import ( has_gzip, has_snappy, gzip_encode, gzip_decode, @@ -386,7 +386,7 @@ def test_decode_offset_fetch_response(self): pass -class TestClient(unittest.TestCase): +class TestKafkaClient(unittest.TestCase): @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') @@ -520,7 +520,7 @@ def test_get_leader_returns_none_when_noleader(self, protocol, conn): @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_noleader(self, protocol, conn): - "Getting leader for partitions returns None when the partiion has no leader" + "Send producer request raises LeaderUnavailableError if leader is not available" conn.recv.return_value = 'response' # anything but None @@ -542,7 +542,7 @@ def test_send_produce_request_raises_when_noleader(self, protocol, conn): [create_message("a"), create_message("b")])] self.assertRaises( - PartitionUnavailableError, + LeaderUnavailableError, client.send_produce_request, requests) if __name__ == '__main__': From 51910f981843dfa967d24659cdb46117210c832d Mon Sep 17 00:00:00 2001 From: mrtheb Date: Sat, 15 Feb 2014 13:43:56 -0500 Subject: [PATCH 095/109] clean up after comments from @rdiomar --- kafka/conn.py | 8 ++- test/test_unit.py | 136 ++-------------------------------------------- 2 files changed, 11 insertions(+), 133 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 20f22dcb1..749cc02fc 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -10,6 +10,8 @@ log = logging.getLogger("kafka") DEFAULT_SOCKET_TIMEOUT_SECONDS = 120 +DEFAULT_KAFKA_PORT = 9092 + def collect_hosts(hosts, randomize=True): """ @@ -18,14 +20,14 @@ def collect_hosts(hosts, randomize=True): """ if isinstance(hosts, str): - hosts = hosts.split(',') + hosts = hosts.strip().split(',') result = [] for host_port in hosts: res = host_port.split(':') host = res[0] - port = int(res[1]) if len(res) > 1 else 9092 + port = int(res[1]) if len(res) > 1 else DEFAULT_KAFKA_PORT result.append((host.strip(), port)) if randomize: @@ -105,7 +107,7 @@ def send(self, request_id, payload): sent = self._sock.sendall(payload) if sent is not None: self._raise_connection_error() - except socket.error, e: + except socket.error: log.exception('Unable to send payload to Kafka') self._raise_connection_error() diff --git a/test/test_unit.py b/test/test_unit.py index 624fe39ed..3239e6a77 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -3,7 +3,8 @@ import struct import unittest -from mock import patch +from mock import MagicMock, patch + from kafka import KafkaClient from kafka.common import ( @@ -366,7 +367,6 @@ def test_encode_offset_request(self): def test_decode_offset_response(self): pass - @unittest.skip("Not Implemented") def test_encode_offset_commit_request(self): pass @@ -409,26 +409,22 @@ def test_init_with_csv(self): def test_send_broker_unaware_request_fail(self): 'Tests that call fails when all hosts are unavailable' - from mock import MagicMock - mocked_conns = { ('kafka01', 9092): MagicMock(), ('kafka02', 9092): MagicMock() } - # inject conns + # inject KafkaConnection side effects mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)") mocked_conns[('kafka02', 9092)].send.side_effect = RuntimeError("Kafka02 went away (unittest)") def mock_get_conn(host, port): - print 'mock_get_conn: %s:%d=%s' % (host, port, mocked_conns[(host, port)]) return mocked_conns[(host, port)] # patch to avoid making requests before we want it with patch.object(KafkaClient, 'load_metadata_for_topics'), \ patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): - client = KafkaClient(hosts=['kafka01:9092','kafka02:9092']) - + client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) self.assertRaises( KafkaUnavailableError, @@ -439,22 +435,19 @@ def mock_get_conn(host, port): conn.send.assert_called_with(1, 'fake request') def test_send_broker_unaware_request(self): - 'Tests that call fails when one of the host is available' - - from mock import MagicMock + 'Tests that call works when at least one of the host is available' mocked_conns = { ('kafka01', 9092): MagicMock(), ('kafka02', 9092): MagicMock(), ('kafka03', 9092): MagicMock() } - # inject conns + # inject KafkaConnection side effects mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)") mocked_conns[('kafka02', 9092)].recv.return_value = 'valid response' mocked_conns[('kafka03', 9092)].send.side_effect = RuntimeError("kafka03 went away (unittest)") def mock_get_conn(host, port): - print 'mock_get_conn: %s:%d=%s' % (host, port, mocked_conns[(host, port)]) return mocked_conns[(host, port)] # patch to avoid making requests before we want it @@ -468,123 +461,6 @@ def mock_get_conn(host, port): self.assertEqual('valid response', resp) mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) - @unittest.skip('requires disabling recursion on load_metadata_for_topics') - @patch('kafka.client.KafkaConnection') - @patch('kafka.client.KafkaProtocol') - def test_client_load_metadata(self, protocol, conn): - - conn.recv.return_value = 'response' # anything but None - - brokers = {} - brokers[0] = BrokerMetadata(1, 'broker_1', 4567) - brokers[1] = BrokerMetadata(2, 'broker_2', 5678) - - topics = {} - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2]) - } - topics['topic_2'] = { - 0: PartitionMetadata('topic_2', 0, 0, [0, 1], [0, 1]), - 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) - } - protocol.decode_metadata_response.return_value = (brokers, topics) - - client = KafkaClient(hosts='broker_1:4567') - self.assertItemsEqual( - { - TopicAndPartition('topic_1', 0): brokers[0], - TopicAndPartition('topic_2', 0): brokers[0], - TopicAndPartition('topic_2', 1): brokers[1] - }, - client.topics_to_brokers) - - @unittest.skip('requires disabling recursion on load_metadata_for_topics') - @patch('kafka.client.KafkaConnection') - @patch('kafka.client.KafkaProtocol') - def test_client_load_metadata_unassigned_partitions(self, protocol, conn): - - conn.recv.return_value = 'response' # anything but None - - brokers = {} - brokers[0] = BrokerMetadata(0, 'broker_1', 4567) - brokers[1] = BrokerMetadata(1, 'broker_2', 5678) - - topics = {} - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, -1, [], []) - } - protocol.decode_metadata_response.return_value = (brokers, topics) - - client = KafkaClient(hosts='broker_1:4567') - - self.assertItemsEqual({}, client.topics_to_brokers) - self.assertRaises( - Exception, - client._get_leader_for_partition, - 'topic_1', 0) - - # calling _get_leader_for_partition (from any broker aware request) - # will try loading metadata again for the same topic - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) - } - leader = client._get_leader_for_partition('topic_1', 0) - - self.assertEqual(brokers[0], leader) - self.assertItemsEqual( - { - TopicAndPartition('topic_1', 0): brokers[0], - }, - client.topics_to_brokers) - - @unittest.skip('requires disabling recursion on load_metadata_for_topics') - @patch('kafka.client.KafkaConnection') - @patch('kafka.client.KafkaProtocol') - def test_client_load_metadata_noleader_partitions(self, protocol, conn): - - conn.recv.return_value = 'response' # anything but None - - brokers = {} - brokers[0] = BrokerMetadata(0, 'broker_1', 4567) - brokers[1] = BrokerMetadata(1, 'broker_2', 5678) - - topics = {} - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, -1, [], []) - } - topics['topic_2'] = { - 0: PartitionMetadata('topic_2', 0, 0, [0, 1], []), - 1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0]) - } - protocol.decode_metadata_response.return_value = (brokers, topics) - - client = KafkaClient(hosts='broker_1:4567') - self.assertItemsEqual( - { - TopicAndPartition('topic_2', 0): brokers[0], - TopicAndPartition('topic_2', 1): brokers[1] - }, - client.topics_to_brokers) - self.assertRaises( - Exception, - client._get_leader_for_partition, - 'topic_1', 0) - - # calling _get_leader_for_partition (from any broker aware request) - # will try loading metadata again for the same topic - topics['topic_1'] = { - 0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1]) - } - leader = client._get_leader_for_partition('topic_1', 0) - - self.assertEqual(brokers[0], leader) - self.assertItemsEqual( - { - TopicAndPartition('topic_1', 0): brokers[0], - TopicAndPartition('topic_2', 0): brokers[0], - TopicAndPartition('topic_2', 1): brokers[1] - }, - client.topics_to_brokers) if __name__ == '__main__': unittest.main() From 828133cff064f4f8fba753183ac21619355ac005 Mon Sep 17 00:00:00 2001 From: mrtheb Date: Sat, 15 Feb 2014 22:05:11 -0500 Subject: [PATCH 096/109] fixing _get_leader_for_partition unittests --- test/test_unit.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/test/test_unit.py b/test/test_unit.py index 1439d8b5b..0b2b3397f 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -417,8 +417,8 @@ def test_load_metadata(self, protocol, conn): # client loads metadata at init client = KafkaClient(host='broker_1', port=4567) - self.assertItemsEqual({ - TopicAndPartition('topic_1', 0): brokers[0], + self.assertDictEqual({ + TopicAndPartition('topic_1', 0): brokers[1], TopicAndPartition('topic_noleader', 0): None, TopicAndPartition('topic_noleader', 1): None, TopicAndPartition('topic_3', 0): brokers[0], @@ -443,7 +443,7 @@ def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn): client = KafkaClient(host='broker_1', port=4567) # topic metadata is loaded but empty - self.assertItemsEqual({}, client.topics_to_brokers) + self.assertDictEqual({}, client.topics_to_brokers) topics['topic_no_partitions'] = { 0: PartitionMetadata('topic_no_partitions', 0, 0, [0, 1], [0, 1]) @@ -455,7 +455,7 @@ def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn): leader = client._get_leader_for_partition('topic_no_partitions', 0) self.assertEqual(brokers[0], leader) - self.assertItemsEqual({ + self.assertDictEqual({ TopicAndPartition('topic_no_partitions', 0): brokers[0]}, client.topics_to_brokers) @@ -475,7 +475,7 @@ def test_get_leader_for_unassigned_partitions(self, protocol, conn): client = KafkaClient(host='broker_1', port=4567) - self.assertItemsEqual({}, client.topics_to_brokers) + self.assertDictEqual({}, client.topics_to_brokers) self.assertRaises( PartitionUnavailableError, client._get_leader_for_partition, @@ -500,7 +500,7 @@ def test_get_leader_returns_none_when_noleader(self, protocol, conn): protocol.decode_metadata_response.return_value = (brokers, topics) client = KafkaClient(host='broker_1', port=4567) - self.assertItemsEqual( + self.assertDictEqual( { TopicAndPartition('topic_noleader', 0): None, TopicAndPartition('topic_noleader', 1): None From 96c9ce0c4a2ab515c581062117d5a6221f1f2036 Mon Sep 17 00:00:00 2001 From: Greg Bowyer Date: Wed, 19 Feb 2014 14:42:40 -0800 Subject: [PATCH 097/109] Make it possible to read and write xerial snappy Fixes mumrah/kafka-python#126 TL;DR ===== This makes it possible to read and write snappy compressed streams that are compatible with the java and scala kafka clients (the xerial blocking format)) Xerial Details ============== Kafka supports transparent compression of data (both in transit and at rest) of messages, one of the allowable compression algorithms is Google's snappy, an algorithm which has excellent performance at the cost of efficiency. The specific implementation of snappy used in kafka is the xerial-snappy implementation, this is a readily available java library for snappy. As part of this implementation, there is a specialised blocking format that is somewhat none standard in the snappy world. Xerial Format ------------- The blocking mode of the xerial snappy library is fairly simple, using a magic header to identify itself and then a size + block scheme, unless otherwise noted all items in xerials blocking format are assumed to be big-endian. A block size (```xerial_blocksize``` in implementation) controls how frequent the blocking occurs 32k is the default in the xerial library, this blocking controls the size of the uncompressed chunks that will be fed to snappy to be compressed. The format winds up being | Header | Block1 len | Block1 data | Blockn len | Blockn data | | ----------- | ---------- | ------------ | ---------- | ------------ | | 16 bytes | BE int32 | snappy bytes | BE int32 | snappy bytes | It is important to not that the blocksize is the amount of uncompressed data presented to snappy at each block, whereas the blocklen is the number of bytes that will be present in the stream, that is the length will always be <= blocksize. Xerial blocking header ---------------------- Marker | Magic String | Null / Pad | Version | Compat ------ | ------------ | ---------- | -------- | -------- byte | c-string | byte | int32 | int32 ------ | ------------ | ---------- | -------- | -------- -126 | 'SNAPPY' | \0 | variable | variable The pad appears to be to ensure that SNAPPY is a valid cstring, and to align the header on a word boundary. The version is the version of this format as written by xerial, in the wild this is currently 1 as such we only support v1. Compat is there to claim the minimum supported version that can read a xerial block stream, presently in the wild this is 1. Implementation specific details =============================== The implementation presented here follows the Xerial implementation as of its v1 blocking format, no attempts are made to check for future versions. Since none-xerial aware clients might have persisted snappy compressed messages to kafka brokers we allow clients to turn on xerial compatibility for message sending, and perform header sniffing to detect xerial vs plain snappy payloads. --- kafka/codec.py | 98 +++++++++++++++++++++++++++++++++++++++++++++-- test/test_unit.py | 43 +++++++++++++++++++++ 2 files changed, 138 insertions(+), 3 deletions(-) diff --git a/kafka/codec.py b/kafka/codec.py index eb5d03cf9..206ddb491 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -1,5 +1,9 @@ from cStringIO import StringIO import gzip +import struct + +_XERIAL_V1_HEADER = (-126, 'S', 'N', 'A', 'P', 'P', 'Y', 0, 1, 1) +_XERIAL_V1_FORMAT = 'bccccccBii' try: import snappy @@ -36,13 +40,101 @@ def gzip_decode(payload): return result -def snappy_encode(payload): +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 format winds up being + +-------------+------------+--------------+------------+--------------+ + | Header | Block1 len | Block1 data | Blockn len | Blockn data | + |-------------+------------+--------------+------------+--------------| + | 16 bytes | BE int32 | snappy bytes | BE int32 | snappy bytes | + +-------------+------------+--------------+------------+--------------+ + + It is important to not that the blocksize is the amount of uncompressed + data presented to snappy at each block, whereas the blocklen is the + number of bytes that will be present in the stream, that is the + length will always be <= blocksize. + """ + if not _has_snappy: raise NotImplementedError("Snappy codec is not available") - return snappy.compress(payload) + + if xerial_compatible: + def _chunker(): + for i in xrange(0, len(payload), xerial_blocksize): + yield payload[i:i+xerial_blocksize] + + out = StringIO() + + header = ''.join([struct.pack('!' + fmt, dat) for fmt, dat + in zip(_XERIAL_V1_FORMAT, _XERIAL_V1_HEADER)]) + + out.write(header) + for chunk in _chunker(): + block = snappy.compress(chunk) + block_size = len(block) + out.write(struct.pack('!i', block_size)) + out.write(block) + + out.seek(0) + return out.read() + + else: + return snappy.compress(payload) + + +def _detect_xerial_stream(payload): + """Detects if the data given might have been encoded with the blocking mode + of the xerial snappy library. + + This mode writes a magic header of the format: + +--------+--------------+------------+---------+--------+ + | Marker | Magic String | Null / Pad | Version | Compat | + |--------+--------------+------------+---------+--------| + | byte | c-string | byte | int32 | int32 | + |--------+--------------+------------+---------+--------| + | -126 | 'SNAPPY' | \0 | | | + +--------+--------------+------------+---------+--------+ + + The pad appears to be to ensure that SNAPPY is a valid cstring + The version is the version of this format as written by xerial, + in the wild this is currently 1 as such we only support v1. + + Compat is there to claim the miniumum supported version that + can read a xerial block stream, presently in the wild this is + 1. + """ + + if len(payload) > 16: + header = header = struct.unpack('!' + _XERIAL_V1_FORMAT, bytes(payload)[:16]) + return header == _XERIAL_V1_HEADER + return False def snappy_decode(payload): if not _has_snappy: raise NotImplementedError("Snappy codec is not available") - return snappy.decompress(payload) + + if _detect_xerial_stream(payload): + # TODO ? Should become a fileobj ? + out = StringIO() + byt = buffer(payload[16:]) + length = len(byt) + cursor = 0 + + while cursor < length: + block_size = struct.unpack_from('!i', byt[cursor:])[0] + # Skip the block size + cursor += 4 + end = cursor + block_size + out.write(snappy.decompress(byt[cursor:end])) + cursor = end + + out.seek(0) + return out.read() + else: + return snappy.decompress(payload) diff --git a/test/test_unit.py b/test/test_unit.py index e3fd4bb5b..b5f01182f 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -70,6 +70,49 @@ def test_snappy(self): s2 = snappy_decode(snappy_encode(s1)) self.assertEquals(s1, s2) + @unittest.skipUnless(has_snappy(), "Snappy not available") + def test_snappy_detect_xerial(self): + import kafka as kafka1 + _detect_xerial_stream = kafka1.codec._detect_xerial_stream + + header = b'\x82SNAPPY\x00\x00\x00\x00\x01\x00\x00\x00\x01Some extra bytes' + false_header = b'\x01SNAPPY\x00\x00\x00\x01\x00\x00\x00\x01' + random_snappy = snappy_encode('SNAPPY' * 50) + short_data = b'\x01\x02\x03\x04' + + self.assertTrue(_detect_xerial_stream(header)) + self.assertFalse(_detect_xerial_stream(b'')) + self.assertFalse(_detect_xerial_stream(b'\x00')) + self.assertFalse(_detect_xerial_stream(false_header)) + self.assertFalse(_detect_xerial_stream(random_snappy)) + self.assertFalse(_detect_xerial_stream(short_data)) + + @unittest.skipUnless(has_snappy(), "Snappy not available") + def test_snappy_decode_xerial(self): + header = b'\x82SNAPPY\x00\x00\x00\x00\x01\x00\x00\x00\x01' + random_snappy = snappy_encode('SNAPPY' * 50) + block_len = len(random_snappy) + random_snappy2 = snappy_encode('XERIAL' * 50) + block_len2 = len(random_snappy2) + + to_test = header \ + + struct.pack('!i', block_len) + random_snappy \ + + struct.pack('!i', block_len2) + random_snappy2 \ + + self.assertEquals(snappy_decode(to_test), ('SNAPPY' * 50) + ('XERIAL' * 50)) + + @unittest.skipUnless(has_snappy(), "Snappy not available") + def test_snappy_encode_xerial(self): + to_ensure = b'\x82SNAPPY\x00\x00\x00\x00\x01\x00\x00\x00\x01' + \ + '\x00\x00\x00\x18' + \ + '\xac\x02\x14SNAPPY\xfe\x06\x00\xfe\x06\x00\xfe\x06\x00\xfe\x06\x00\x96\x06\x00' + \ + '\x00\x00\x00\x18' + \ + '\xac\x02\x14XERIAL\xfe\x06\x00\xfe\x06\x00\xfe\x06\x00\xfe\x06\x00\x96\x06\x00' + + to_test = ('SNAPPY' * 50) + ('XERIAL' * 50) + + compressed = snappy_encode(to_test, xerial_compatible=True, xerial_blocksize=300) + self.assertEquals(compressed, to_ensure) class TestProtocol(unittest.TestCase): From e5fdc1c7b22c8ad2aaa66a486871d0ed65977e3d Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 25 Feb 2014 10:46:28 -0500 Subject: [PATCH 098/109] Fix version in __init__.py to match setup.py --- kafka/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/__init__.py b/kafka/__init__.py index 73aa7603c..e446f58f1 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -1,5 +1,5 @@ __title__ = 'kafka' -__version__ = '0.2-alpha' +__version__ = '0.9.0' __author__ = 'David Arthur' __license__ = 'Apache License 2.0' __copyright__ = 'Copyright 2012, David Arthur under Apache License, v2.0' From ee7e86ea712de0a0390e64752c5cf9180c1681b5 Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Tue, 25 Feb 2014 01:02:39 -0800 Subject: [PATCH 099/109] Update example.py to compile, add friendly load_example.py --- example.py | 48 ++++++++++++++++++++++++++++++----------- load_example.py | 57 +++++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 92 insertions(+), 13 deletions(-) mode change 100644 => 100755 example.py create mode 100755 load_example.py diff --git a/example.py b/example.py old mode 100644 new mode 100755 index 3a2dc928b..0eac0a506 --- a/example.py +++ b/example.py @@ -1,23 +1,45 @@ -import logging +#!/usr/bin/env python +import threading, logging, time -from kafka.client import KafkaClient, FetchRequest, ProduceRequest +from kafka.client import KafkaClient from kafka.consumer import SimpleConsumer from kafka.producer import SimpleProducer -def produce_example(client): - producer = SimpleProducer(client, "my-topic") - producer.send_messages("test") +class Producer(threading.Thread): + daemon = True -def consume_example(client): - consumer = SimpleConsumer(client, "test-group", "my-topic") - for message in consumer: - print(message) + def run(self): + client = KafkaClient("localhost", 9092) + producer = SimpleProducer(client) + + while True: + producer.send_messages('my-topic', "test") + producer.send_messages('my-topic', "\xc2Hola, mundo!") + + time.sleep(1) + + +class Consumer(threading.Thread): + daemon = True + + def run(self): + client = KafkaClient("localhost", 9092) + consumer = SimpleConsumer(client, "test-group", "my-topic") + + for message in consumer: + print(message) def main(): - client = KafkaClient("localhost", 9092) - produce_example(client) - consume_example(client) + threads = [ + Producer(), + Consumer() + ] + + for t in threads: + t.start() + + time.sleep(5) if __name__ == "__main__": - logging.basicConfig(level=logging.DEBUG) + logging.basicConfig(level=logging.WARN) main() diff --git a/load_example.py b/load_example.py new file mode 100755 index 000000000..0ef07b6be --- /dev/null +++ b/load_example.py @@ -0,0 +1,57 @@ +#!/usr/bin/env python +import threading, logging, time, collections + +from kafka.client import KafkaClient +from kafka.consumer import SimpleConsumer +from kafka.producer import SimpleProducer + +msg_size = 524288 + +class Producer(threading.Thread): + daemon = True + big_msg = "1" * msg_size + + def run(self): + client = KafkaClient("localhost", 9092) + producer = SimpleProducer(client) + self.sent = 0 + + while True: + producer.send_messages('my-topic', self.big_msg) + self.sent += 1 + + +class Consumer(threading.Thread): + daemon = True + + def run(self): + client = KafkaClient("localhost", 9092) + consumer = SimpleConsumer(client, "test-group", "my-topic", + max_buffer_size = None, + ) + self.valid = 0 + self.invalid = 0 + + for message in consumer: + if len(message.message.value) == msg_size: + self.valid += 1 + else: + self.invalid += 1 + +def main(): + threads = [ + Producer(), + Consumer() + ] + + for t in threads: + t.start() + + time.sleep(10) + print 'Messages sent: %d' % threads[0].sent + print 'Messages recvd: %d' % threads[1].valid + print 'Messages invalid: %d' % threads[1].invalid + +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + main() From 9732ed1670ef0739956900df37c0c77699628ec7 Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Tue, 25 Feb 2014 01:06:17 -0800 Subject: [PATCH 100/109] Minor refactor in conn.py, update version in __init__.py, add ErrorString --- kafka/client.py | 6 +++--- kafka/common.py | 36 +++++++++++++++++++++--------------- kafka/conn.py | 19 +++++++++++-------- load_example.py | 2 +- 4 files changed, 36 insertions(+), 27 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 155f65883..9578ee8e1 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -5,7 +5,7 @@ from functools import partial from itertools import count -from kafka.common import (ErrorMapping, TopicAndPartition, +from kafka.common import (ErrorMapping, ErrorStrings, TopicAndPartition, ConnectionError, FailedPayloadsError, BrokerResponseError, PartitionUnavailableError, KafkaUnavailableError, KafkaRequestError) @@ -174,8 +174,8 @@ def _raise_on_response_error(self, resp): self.reset_topic_metadata(resp.topic) raise BrokerResponseError( - "Request for %s failed with errorcode=%d" % - (TopicAndPartition(resp.topic, resp.partition), resp.error)) + "Request for %s failed with errorcode=%d (%s)" % + (TopicAndPartition(resp.topic, resp.partition), resp.error, ErrorStrings[resp.error])) ################# # Public API # diff --git a/kafka/common.py b/kafka/common.py index c0a1a6a1a..583bb3225 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -48,22 +48,28 @@ TopicAndPartition = namedtuple("TopicAndPartition", ["topic", "partition"]) +ErrorStrings = { + -1 : 'UNKNOWN', + 0 : 'NO_ERROR', + 1 : 'OFFSET_OUT_OF_RANGE', + 2 : 'INVALID_MESSAGE', + 3 : 'UNKNOWN_TOPIC_OR_PARTITON', + 4 : 'INVALID_FETCH_SIZE', + 5 : 'LEADER_NOT_AVAILABLE', + 6 : 'NOT_LEADER_FOR_PARTITION', + 7 : 'REQUEST_TIMED_OUT', + 8 : 'BROKER_NOT_AVAILABLE', + 9 : 'REPLICA_NOT_AVAILABLE', + 10 : 'MESSAGE_SIZE_TO_LARGE', + 11 : 'STALE_CONTROLLER_EPOCH', + 12 : 'OFFSET_METADATA_TOO_LARGE', +} + class ErrorMapping(object): - # Many of these are not actually used by the client - UNKNOWN = -1 - NO_ERROR = 0 - OFFSET_OUT_OF_RANGE = 1 - INVALID_MESSAGE = 2 - UNKNOWN_TOPIC_OR_PARTITON = 3 - INVALID_FETCH_SIZE = 4 - LEADER_NOT_AVAILABLE = 5 - NOT_LEADER_FOR_PARTITION = 6 - REQUEST_TIMED_OUT = 7 - BROKER_NOT_AVAILABLE = 8 - REPLICA_NOT_AVAILABLE = 9 - MESSAGE_SIZE_TO_LARGE = 10 - STALE_CONTROLLER_EPOCH = 11 - OFFSET_METADATA_TOO_LARGE = 12 + pass + +for k, v in ErrorStrings.items(): + setattr(ErrorMapping, v, k) ################# # Exceptions # diff --git a/kafka/conn.py b/kafka/conn.py index 2b8f1c283..cc946fcb9 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -28,11 +28,10 @@ def __init__(self, host, port, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): super(KafkaConnection, self).__init__() self.host = host self.port = port - self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self._sock.connect((host, port)) self.timeout = timeout - self._sock.settimeout(self.timeout) - self._dirty = False + self._sock = None + + self.reinit() def __repr__(self): return "" % (self.host, self.port) @@ -47,24 +46,28 @@ def _raise_connection_error(self): def _read_bytes(self, num_bytes): bytes_left = num_bytes - resp = '' + responses = [] + log.debug("About to read %d bytes from Kafka", num_bytes) if self._dirty: self.reinit() + while bytes_left: try: - data = self._sock.recv(bytes_left) + data = self._sock.recv(min(bytes_left, 4096)) except socket.error: log.exception('Unable to receive data from Kafka') self._raise_connection_error() + if data == '': log.error("Not enough data to read this response") self._raise_connection_error() + bytes_left -= len(data) log.debug("Read %d/%d bytes from Kafka", num_bytes - bytes_left, num_bytes) - resp += data + responses.append(data) - return resp + return ''.join(responses) ################## # Public API # diff --git a/load_example.py b/load_example.py index 0ef07b6be..4da584121 100755 --- a/load_example.py +++ b/load_example.py @@ -53,5 +53,5 @@ def main(): print 'Messages invalid: %d' % threads[1].invalid if __name__ == "__main__": - logging.basicConfig(level=logging.DEBUG) + logging.basicConfig(level=logging.WARN) main() From 25e63cc4223f02f9deb3d4d393db308eac7f640b Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Tue, 25 Feb 2014 02:18:46 -0800 Subject: [PATCH 101/109] Update logging config to include timestamps for better/easier debugging --- example.py | 5 ++++- load_example.py | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/example.py b/example.py index 0eac0a506..43298e8b7 100755 --- a/example.py +++ b/example.py @@ -41,5 +41,8 @@ def main(): time.sleep(5) if __name__ == "__main__": - logging.basicConfig(level=logging.WARN) + logging.basicConfig( + format='%(asctime)s.%(msecs)s:%(name)s:%(thread)d:%(levelname)s:%(process)d:%(message)s', + level=logging.DEBUG + ) main() diff --git a/load_example.py b/load_example.py index 4da584121..95f0c7f25 100755 --- a/load_example.py +++ b/load_example.py @@ -53,5 +53,8 @@ def main(): print 'Messages invalid: %d' % threads[1].invalid if __name__ == "__main__": - logging.basicConfig(level=logging.WARN) + logging.basicConfig( + format='%(asctime)s.%(msecs)s:%(name)s:%(thread)d:%(levelname)s:%(process)d:%(message)s', + level=logging.DEBUG + ) main() From 3b39d9d6589ee46ae4d31fb078f44019a2810983 Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Tue, 25 Feb 2014 16:51:36 -0800 Subject: [PATCH 102/109] Fix grammar in error string --- kafka/common.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/common.py b/kafka/common.py index 583bb3225..70d67b4f1 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -60,7 +60,7 @@ 7 : 'REQUEST_TIMED_OUT', 8 : 'BROKER_NOT_AVAILABLE', 9 : 'REPLICA_NOT_AVAILABLE', - 10 : 'MESSAGE_SIZE_TO_LARGE', + 10 : 'MESSAGE_SIZE_TOO_LARGE', 11 : 'STALE_CONTROLLER_EPOCH', 12 : 'OFFSET_METADATA_TOO_LARGE', } From 888f206d5417e95f26de407b28fe935950aea2c9 Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Wed, 26 Feb 2014 21:38:58 -0800 Subject: [PATCH 103/109] Update load_example --- load_example.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/load_example.py b/load_example.py index 95f0c7f25..1f8b41820 100755 --- a/load_example.py +++ b/load_example.py @@ -12,7 +12,7 @@ class Producer(threading.Thread): big_msg = "1" * msg_size def run(self): - client = KafkaClient("localhost", 9092) + client = KafkaClient("localhost:9092") producer = SimpleProducer(client) self.sent = 0 @@ -25,7 +25,7 @@ class Consumer(threading.Thread): daemon = True def run(self): - client = KafkaClient("localhost", 9092) + client = KafkaClient("localhost:9092") consumer = SimpleConsumer(client, "test-group", "my-topic", max_buffer_size = None, ) From 2eac578a5a75e5f8f2afcfdcd18d127c7541e5b3 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Mon, 3 Mar 2014 17:48:37 -0800 Subject: [PATCH 104/109] nit: fixed misspelling --- kafka/conn.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index 749cc02fc..7266ae840 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -15,7 +15,7 @@ def collect_hosts(hosts, randomize=True): """ - Collects a comma-separated set of hosts (host:port) and optionnaly + Collects a comma-separated set of hosts (host:port) and optionally randomize the returned list. """ From c4cbe04cf13cc94ea1c72904f2b63b91fb5c0b84 Mon Sep 17 00:00:00 2001 From: stephenarmstrong Date: Thu, 13 Mar 2014 15:14:32 -0700 Subject: [PATCH 105/109] If a broker refuses the connection, try the next --- kafka/client.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 96cc1dfbd..a683fe006 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -83,14 +83,14 @@ def _send_broker_unaware_request(self, requestId, request): brokers. Keep trying until you succeed. """ for (host, port) in self.hosts: - conn = self._get_conn(host, port) try: + conn = self._get_conn(host, port) conn.send(requestId, request) response = conn.recv(requestId) return response except Exception, e: - log.warning("Could not send request [%r] to server %s, " - "trying next server: %s" % (request, conn, e)) + log.warning("Could not send request [%r] to server %s:%i, " + "trying next server: %s" % (request, host, port, e)) continue raise KafkaUnavailableError("All servers failed to process request") From e08c718ca65c04d012f32e0053e58e851b04f40e Mon Sep 17 00:00:00 2001 From: Saulius Zemaitaitis Date: Mon, 17 Mar 2014 17:06:48 +0200 Subject: [PATCH 106/109] Check against basestring instead of str in collect.hosts. --- kafka/conn.py | 2 +- test/test_unit.py | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index 7266ae840..7538e8d3b 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -19,7 +19,7 @@ def collect_hosts(hosts, randomize=True): randomize the returned list. """ - if isinstance(hosts, str): + if isinstance(hosts, basestring): hosts = hosts.strip().split(',') result = [] diff --git a/test/test_unit.py b/test/test_unit.py index aec0a2c2f..081acc793 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -449,6 +449,16 @@ def test_init_with_csv(self): [('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)], client.hosts) + def test_init_with_unicode_csv(self): + + with patch.object(KafkaClient, 'load_metadata_for_topics'): + client = KafkaClient( + hosts=u'kafka01:9092,kafka02:9092,kafka03:9092') + + self.assertItemsEqual( + [('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)], + client.hosts) + def test_send_broker_unaware_request_fail(self): 'Tests that call fails when all hosts are unavailable' From 017f4842859d58ce32e74ad470ed619c5a8922df Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Tue, 18 Mar 2014 20:49:10 -0700 Subject: [PATCH 107/109] Fix py26 compatibility issue, add mock to tox --- test/test_unit.py | 32 +++++++++++++++----------------- tox.ini | 4 +++- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/test/test_unit.py b/test/test_unit.py index 47ec561a0..2d769fb7f 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -465,18 +465,17 @@ def mock_get_conn(host, port): return mocked_conns[(host, port)] # patch to avoid making requests before we want it - with patch.object(KafkaClient, 'load_metadata_for_topics'), \ - patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): - - client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) + with patch.object(KafkaClient, 'load_metadata_for_topics'): + with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): + client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) - self.assertRaises( - KafkaUnavailableError, - client._send_broker_unaware_request, - 1, 'fake request') + self.assertRaises( + KafkaUnavailableError, + client._send_broker_unaware_request, + 1, 'fake request') - for key, conn in mocked_conns.iteritems(): - conn.send.assert_called_with(1, 'fake request') + for key, conn in mocked_conns.iteritems(): + conn.send.assert_called_with(1, 'fake request') def test_send_broker_unaware_request(self): 'Tests that call works when at least one of the host is available' @@ -495,15 +494,14 @@ def mock_get_conn(host, port): return mocked_conns[(host, port)] # patch to avoid making requests before we want it - with patch.object(KafkaClient, 'load_metadata_for_topics'), \ - patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): - - client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + with patch.object(KafkaClient, 'load_metadata_for_topics'): + with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): + client = KafkaClient(hosts='kafka01:9092,kafka02:9092') - resp = client._send_broker_unaware_request(1, 'fake request') + resp = client._send_broker_unaware_request(1, 'fake request') - self.assertEqual('valid response', resp) - mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) + self.assertEqual('valid response', resp) + mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) @patch('kafka.client.KafkaConnection') @patch('kafka.client.KafkaProtocol') diff --git a/tox.ini b/tox.ini index f41911c62..0077c4d87 100644 --- a/tox.ini +++ b/tox.ini @@ -1,7 +1,9 @@ [tox] envlist = py26, py27 [testenv] -deps = pytest +deps = + pytest + mock commands = py.test --basetemp={envtmpdir} [] setenv = PROJECT_ROOT = {toxinidir} From e5af3170f3f7fb095526269ff20e897ab08c0015 Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Tue, 25 Mar 2014 11:07:31 -0700 Subject: [PATCH 108/109] Make seek(); commit(); work without commit discarding the seek change --- kafka/consumer.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/consumer.py b/kafka/consumer.py index 28b53ec92..140409721 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -275,6 +275,7 @@ def seek(self, offset, whence): 2 is relative to the latest known offset (tail) """ + self.count_since_commit += 1 if whence == 1: # relative to current position for partition, _offset in self.offsets.items(): self.offsets[partition] = _offset + offset From 7d425d07c26b7900aae741a7b89c7001907b73a9 Mon Sep 17 00:00:00 2001 From: Mark Roberts Date: Thu, 27 Mar 2014 10:26:52 -0700 Subject: [PATCH 109/109] Commit in seek if autocommit --- kafka/consumer.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka/consumer.py b/kafka/consumer.py index 140409721..8ac28daf4 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -275,7 +275,6 @@ def seek(self, offset, whence): 2 is relative to the latest known offset (tail) """ - self.count_since_commit += 1 if whence == 1: # relative to current position for partition, _offset in self.offsets.items(): self.offsets[partition] = _offset + offset @@ -306,6 +305,10 @@ def seek(self, offset, whence): # Reset queue and fetch offsets since they are invalid self.fetch_offsets = self.offsets.copy() + if self.auto_commit: + self.count_since_commit += 1 + self.commit() + self.queue = Queue() def get_messages(self, count=1, block=True, timeout=0.1):