diff options
Diffstat (limited to 'test')
-rw-r--r-- | test/fixtures.py | 43 | ||||
-rw-r--r-- | test/test_integration.py | 267 | ||||
-rw-r--r-- | test/test_unit.py | 475 |
3 files changed, 505 insertions, 280 deletions
diff --git a/test/fixtures.py b/test/fixtures.py index c771a58..9e283d3 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -208,9 +208,12 @@ class ZookeeperFixture(object): self.tmp_dir = None self.child = None + def out(self, message): + print("*** Zookeeper [%s:%d]: %s" % (self.host, self.port, message)) + def open(self): self.tmp_dir = tempfile.mkdtemp() - print("*** Running local Zookeeper instance...") + self.out("Running local instance...") print(" host = %s" % self.host) print(" port = %s" % self.port) print(" tmp_dir = %s" % self.tmp_dir) @@ -229,16 +232,16 @@ class ZookeeperFixture(object): 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) @@ -272,10 +275,18 @@ class KafkaFixture(object): self.tmp_dir = None self.child = None + self.running = False + + def out(self, message): + print("*** Kafka [%s:%d]: %s" % (self.host, self.port, message)) def open(self): + if self.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) @@ -303,25 +314,31 @@ class KafkaFixture(object): 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): - print("*** Stopping Kafka...") + if not self.running: + self.out("Instance already stopped") + return + + self.out("Stopping...") self.child.stop() self.child = None - print("*** Done!") + self.out("Done!") shutil.rmtree(self.tmp_dir) + self.running = False diff --git a/test/test_integration.py b/test/test_integration.py index a10dae2..5a22630 100644 --- a/test/test_integration.py +++ b/test/test_integration.py @@ -8,10 +8,36 @@ import random from kafka import * # noqa from kafka.common import * # noqa from kafka.codec import has_gzip, has_snappy +from kafka.consumer import MAX_FETCH_BUFFER_SIZE_BYTES from .fixtures import ZookeeperFixture, KafkaFixture -class TestKafkaClient(unittest.TestCase): +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: + 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): + self.topic = "%s-%s" % (self.id()[self.id().rindex(".")+1:], random_string(10)) + ensure_topic_creation(self.client, self.topic) + + +class TestKafkaClient(KafkaTestCase): @classmethod def setUpClass(cls): # noqa cls.zk = ZookeeperFixture.instance() @@ -29,7 +55,8 @@ class TestKafkaClient(unittest.TestCase): ##################### 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) ]) @@ -37,25 +64,25 @@ class TestKafkaClient(unittest.TestCase): 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) ]) @@ -63,7 +90,7 @@ class TestKafkaClient(unittest.TestCase): 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): @@ -72,13 +99,13 @@ class TestKafkaClient(unittest.TestCase): 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): @@ -87,13 +114,13 @@ class TestKafkaClient(unittest.TestCase): 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): @@ -103,17 +130,17 @@ class TestKafkaClient(unittest.TestCase): 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)]) ]) @@ -121,10 +148,10 @@ class TestKafkaClient(unittest.TestCase): 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)]) ]) @@ -132,7 +159,7 @@ class TestKafkaClient(unittest.TestCase): 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) ##################### @@ -140,18 +167,18 @@ class TestKafkaClient(unittest.TestCase): ##################### 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"), ]) @@ -160,7 +187,7 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -175,7 +202,7 @@ class TestKafkaClient(unittest.TestCase): 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) ]) @@ -184,7 +211,7 @@ class TestKafkaClient(unittest.TestCase): 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]) @@ -194,7 +221,7 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -207,10 +234,10 @@ class TestKafkaClient(unittest.TestCase): 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) ]) @@ -218,8 +245,8 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -244,11 +271,11 @@ class TestKafkaClient(unittest.TestCase): @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) @@ -257,7 +284,7 @@ class TestKafkaClient(unittest.TestCase): # 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 @@ -271,8 +298,8 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -296,15 +323,15 @@ class TestKafkaClient(unittest.TestCase): 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]) @@ -330,15 +357,15 @@ class TestKafkaClient(unittest.TestCase): producer.stop() def test_hashed_partitioner(self): - producer = KeyedProducer(self.client, "test_hash_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_hash_partitioner", 0, 0, 1024) - fetch2 = FetchRequest("test_hash_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]) @@ -364,12 +391,12 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -383,12 +410,12 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -403,12 +430,12 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -422,16 +449,14 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -445,8 +470,7 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -454,7 +478,7 @@ class TestKafkaClient(unittest.TestCase): # 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) @@ -468,7 +492,7 @@ class TestKafkaClient(unittest.TestCase): 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) @@ -483,8 +507,8 @@ class TestKafkaClient(unittest.TestCase): # 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]) @@ -503,8 +527,8 @@ class TestKafkaClient(unittest.TestCase): # 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]) @@ -522,8 +546,8 @@ class TestKafkaClient(unittest.TestCase): 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]) @@ -535,8 +559,8 @@ class TestKafkaClient(unittest.TestCase): # 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]) @@ -548,13 +572,13 @@ class TestKafkaClient(unittest.TestCase): producer.stop() -class TestConsumer(unittest.TestCase): +class TestConsumer(KafkaTestCase): @classmethod 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 @@ -565,7 +589,7 @@ class TestConsumer(unittest.TestCase): 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) ]) @@ -574,7 +598,7 @@ class TestConsumer(unittest.TestCase): 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) ]) @@ -583,7 +607,9 @@ class TestConsumer(unittest.TestCase): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = SimpleConsumer(self.client, "group1", "test_simple_consumer", auto_commit=False) + consumer = SimpleConsumer(self.client, "group1", + self.topic, auto_commit=False, + iter_timeout=0) all_messages = [] for message in consumer: all_messages.append(message) @@ -609,7 +635,9 @@ class TestConsumer(unittest.TestCase): 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", + self.topic, + auto_commit=False, iter_timeout=0) # Blocking API start = datetime.now() @@ -619,7 +647,7 @@ class TestConsumer(unittest.TestCase): 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) ]) @@ -643,21 +671,22 @@ class TestConsumer(unittest.TestCase): def test_simple_consumer_pending(self): # Produce 10 messages to partition 0 and 1 - produce1 = ProduceRequest("test_simple_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_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_pending", auto_commit=False) + 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) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -665,7 +694,7 @@ class TestConsumer(unittest.TestCase): def test_multi_process_consumer(self): # Produce 100 messages to partition 0 - produce1 = ProduceRequest("test_mpconsumer", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(100) ]) @@ -674,7 +703,7 @@ class TestConsumer(unittest.TestCase): self.assertEquals(resp.offset, 0) # Produce 100 messages to partition 1 - produce2 = ProduceRequest("test_mpconsumer", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Test message 1 %d" % i) for i in range(100) ]) @@ -683,7 +712,7 @@ class TestConsumer(unittest.TestCase): self.assertEquals(resp.offset, 0) # Start a consumer - consumer = MultiProcessConsumer(self.client, "grp1", "test_mpconsumer", auto_commit=False) + consumer = MultiProcessConsumer(self.client, "grp1", self.topic, auto_commit=False) all_messages = [] for message in consumer: all_messages.append(message) @@ -696,11 +725,11 @@ class TestConsumer(unittest.TestCase): 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.999) self.assertEqual(len(messages), 0) # Send 10 messages - produce = ProduceRequest("test_mpconsumer", 0, messages=[ + produce = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -723,7 +752,7 @@ class TestConsumer(unittest.TestCase): def test_multi_proc_pending(self): # Produce 10 messages to partition 0 and 1 - produce1 = ProduceRequest("test_mppending", 0, messages=[ + produce1 = ProduceRequest(self.topic, 0, messages=[ create_message("Test message 0 %d" % i) for i in range(10) ]) @@ -731,7 +760,7 @@ class TestConsumer(unittest.TestCase): self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 0) - produce2 = ProduceRequest("test_mppending", 1, messages=[ + produce2 = ProduceRequest(self.topic, 1, messages=[ create_message("Test message 1 %d" % i) for i in range(10) ]) @@ -739,7 +768,7 @@ class TestConsumer(unittest.TestCase): 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", self.topic, auto_commit=False) self.assertEquals(consumer.pending(), 20) self.assertEquals(consumer.pending(partitions=[0]), 10) self.assertEquals(consumer.pending(partitions=[1]), 10) @@ -749,52 +778,74 @@ class TestConsumer(unittest.TestCase): 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) 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) + 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", auto_commit=False) + consumer = SimpleConsumer(self.client, "group1", self.topic, + auto_commit=False, iter_timeout=0) all_messages = messages1 + messages2 for i, message in enumerate(consumer): self.assertEquals(all_messages[i], message.message) self.assertEquals(i, 19) -class TestFailover(unittest.TestCase): + # 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(self.topic, 0, [big_message]) + for resp in self.client.send_produce_request([produce3]): + self.assertEquals(resp.error, 0) + self.assertEquals(resp.offset, 20) - @classmethod - def setUpClass(cls): + self.assertRaises(ConsumerFetchSizeTooSmall, consumer.get_message, False, 0.1) + # Create a consumer with no fetch size limit + big_consumer = SimpleConsumer(self.client, "group1", self.topic, + 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(KafkaTestCase): + + def setUp(self): 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.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) + super(TestFailover, self).setUp() + + def tearDown(self): + self.client.close() + for broker in self.brokers: broker.close() - cls.zk.close() + 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): @@ -807,7 +858,7 @@ class TestFailover(unittest.TestCase): 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) @@ -825,8 +876,7 @@ class TestFailover(unittest.TestCase): 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): @@ -858,18 +908,18 @@ class TestFailover(unittest.TestCase): 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) - consumer = SimpleConsumer(client, group, topic, auto_commit=False) + 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: all_messages.append(message) @@ -877,11 +927,6 @@ class TestFailover(unittest.TestCase): 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() diff --git a/test/test_unit.py b/test/test_unit.py index 3f3af66..e3fd4bb 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -3,13 +3,18 @@ import random 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, + 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 +25,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,173 +50,334 @@ class TestPackage(unittest.TestCase): 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): + + @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)) 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 + @unittest.skipUnless(has_gzip(), "Snappy not available") 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) + + @unittest.skipUnless(has_snappy(), "Snappy not available") 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")) + + @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' + '\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")) + + @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' + '\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__': |