Split out kafka version environments, default tox no longer runs any integration tests, make skipped integration also skip setupClass, implement rudimentary offset support in consumer.py
This commit is contained in:
		| @@ -10,6 +10,7 @@ from Queue import Empty, Queue | |||||||
| from kafka.common import ( | from kafka.common import ( | ||||||
|     ErrorMapping, FetchRequest, |     ErrorMapping, FetchRequest, | ||||||
|     OffsetRequest, OffsetCommitRequest, |     OffsetRequest, OffsetCommitRequest, | ||||||
|  |     OffsetFetchRequest, | ||||||
|     ConsumerFetchSizeTooSmall, ConsumerNoMoreData |     ConsumerFetchSizeTooSmall, ConsumerNoMoreData | ||||||
| ) | ) | ||||||
|  |  | ||||||
| @@ -105,15 +106,14 @@ class Consumer(object): | |||||||
|                                 "partition=%d failed with errorcode=%s" % ( |                                 "partition=%d failed with errorcode=%s" % ( | ||||||
|                                     resp.topic, resp.partition, resp.error)) |                                     resp.topic, resp.partition, resp.error)) | ||||||
|  |  | ||||||
|         # Uncomment for 0.8.1 |         if auto_commit: | ||||||
|         # |             for partition in partitions: | ||||||
|         #for partition in partitions: |                 req = OffsetFetchRequest(topic, partition) | ||||||
|         #    req = OffsetFetchRequest(topic, partition) |                 (offset,) = self.client.send_offset_fetch_request(group, [req], | ||||||
|         #    (offset,) = self.client.send_offset_fetch_request(group, [req], |                               callback=get_or_init_offset_callback, | ||||||
|         #                  callback=get_or_init_offset_callback, |                               fail_on_error=False) | ||||||
|         #                  fail_on_error=False) |                 self.offsets[partition] = offset | ||||||
|         #    self.offsets[partition] = offset |         else: | ||||||
|  |  | ||||||
|             for partition in partitions: |             for partition in partitions: | ||||||
|                 self.offsets[partition] = 0 |                 self.offsets[partition] = 0 | ||||||
|  |  | ||||||
|   | |||||||
| @@ -1,25 +1,32 @@ | |||||||
| import unittest | import os | ||||||
| import time |  | ||||||
| import socket |  | ||||||
| import random | import random | ||||||
|  | import socket | ||||||
|  | import time | ||||||
|  | import unittest | ||||||
|  |  | ||||||
| import kafka | import kafka | ||||||
| from kafka.common import * | from kafka.common import * | ||||||
| from fixtures import ZookeeperFixture, KafkaFixture | from fixtures import ZookeeperFixture, KafkaFixture | ||||||
| from testutil import * | from testutil import * | ||||||
|  |  | ||||||
| @unittest.skipIf(skip_integration(), 'Skipping Integration') |  | ||||||
| class TestKafkaClientIntegration(KafkaIntegrationTestCase): | class TestKafkaClientIntegration(KafkaIntegrationTestCase): | ||||||
|     @classmethod |     @classmethod | ||||||
|     def setUpClass(cls):  # noqa |     def setUpClass(cls):  # noqa | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.zk = ZookeeperFixture.instance() |         cls.zk = ZookeeperFixture.instance() | ||||||
|         cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) |         cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) | ||||||
|  |  | ||||||
|     @classmethod |     @classmethod | ||||||
|     def tearDownClass(cls):  # noqa |     def tearDownClass(cls):  # noqa | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.server.close() |         cls.server.close() | ||||||
|         cls.zk.close() |         cls.zk.close() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_timeout(self): |     def test_timeout(self): | ||||||
|         server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) |         server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) | ||||||
|         server_port = get_open_port() |         server_port = get_open_port() | ||||||
| @@ -30,6 +37,7 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): | |||||||
|                 conn = kafka.conn.KafkaConnection("localhost", server_port, 1.0) |                 conn = kafka.conn.KafkaConnection("localhost", server_port, 1.0) | ||||||
|         self.assertGreaterEqual(t.interval, 1.0) |         self.assertGreaterEqual(t.interval, 1.0) | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_consume_none(self): |     def test_consume_none(self): | ||||||
|         fetch = FetchRequest(self.topic, 0, 0, 1024) |         fetch = FetchRequest(self.topic, 0, 0, 1024) | ||||||
|  |  | ||||||
|   | |||||||
| @@ -14,7 +14,7 @@ from kafka.common import ( | |||||||
|     LeaderUnavailableError, PartitionUnavailableError |     LeaderUnavailableError, PartitionUnavailableError | ||||||
| ) | ) | ||||||
| from kafka.codec import ( | from kafka.codec import ( | ||||||
|     has_gzip, has_snappy, gzip_encode, gzip_decode, |     has_snappy, gzip_encode, gzip_decode, | ||||||
|     snappy_encode, snappy_decode |     snappy_encode, snappy_decode | ||||||
| ) | ) | ||||||
| from kafka.protocol import ( | from kafka.protocol import ( | ||||||
| @@ -23,7 +23,6 @@ from kafka.protocol import ( | |||||||
| from testutil import * | from testutil import * | ||||||
|  |  | ||||||
| class TestCodec(unittest.TestCase): | class TestCodec(unittest.TestCase): | ||||||
|     @unittest.skipUnless(has_gzip(), "Gzip not available") |  | ||||||
|     def test_gzip(self): |     def test_gzip(self): | ||||||
|         for i in xrange(1000): |         for i in xrange(1000): | ||||||
|             s1 = random_string(100) |             s1 = random_string(100) | ||||||
|   | |||||||
| @@ -1,3 +1,4 @@ | |||||||
|  | import os | ||||||
| import unittest | import unittest | ||||||
| from datetime import datetime | from datetime import datetime | ||||||
|  |  | ||||||
| @@ -7,10 +8,12 @@ from kafka.consumer import MAX_FETCH_BUFFER_SIZE_BYTES | |||||||
| from fixtures import ZookeeperFixture, KafkaFixture | from fixtures import ZookeeperFixture, KafkaFixture | ||||||
| from testutil import * | from testutil import * | ||||||
|  |  | ||||||
| @unittest.skipIf(skip_integration(), 'Skipping Integration') |  | ||||||
| class TestConsumerIntegration(KafkaIntegrationTestCase): | class TestConsumerIntegration(KafkaIntegrationTestCase): | ||||||
|     @classmethod |     @classmethod | ||||||
|     def setUpClass(cls): |     def setUpClass(cls): | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.zk = ZookeeperFixture.instance() |         cls.zk = ZookeeperFixture.instance() | ||||||
|         cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) |         cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) | ||||||
|         cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) |         cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) | ||||||
| @@ -19,6 +22,9 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|     @classmethod |     @classmethod | ||||||
|     def tearDownClass(cls):  # noqa |     def tearDownClass(cls):  # noqa | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.server1.close() |         cls.server1.close() | ||||||
|         cls.server2.close() |         cls.server2.close() | ||||||
|         cls.zk.close() |         cls.zk.close() | ||||||
| @@ -38,6 +44,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|         # Make sure there are no duplicates |         # Make sure there are no duplicates | ||||||
|         self.assertEquals(len(set(messages)), num_messages) |         self.assertEquals(len(set(messages)), num_messages) | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_simple_consumer(self): |     def test_simple_consumer(self): | ||||||
|         self.send_messages(0, range(0, 100)) |         self.send_messages(0, range(0, 100)) | ||||||
|         self.send_messages(1, range(100, 200)) |         self.send_messages(1, range(100, 200)) | ||||||
| @@ -51,6 +58,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_simple_consumer__seek(self): |     def test_simple_consumer__seek(self): | ||||||
|         self.send_messages(0, range(0, 100)) |         self.send_messages(0, range(0, 100)) | ||||||
|         self.send_messages(1, range(100, 200)) |         self.send_messages(1, range(100, 200)) | ||||||
| @@ -69,6 +77,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_simple_consumer_blocking(self): |     def test_simple_consumer_blocking(self): | ||||||
|         consumer = SimpleConsumer(self.client, "group1", |         consumer = SimpleConsumer(self.client, "group1", | ||||||
|                                   self.topic, |                                   self.topic, | ||||||
| @@ -96,6 +105,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_simple_consumer_pending(self): |     def test_simple_consumer_pending(self): | ||||||
|         # Produce 10 messages to partitions 0 and 1 |         # Produce 10 messages to partitions 0 and 1 | ||||||
|         self.send_messages(0, range(0, 10)) |         self.send_messages(0, range(0, 10)) | ||||||
| @@ -110,6 +120,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_multi_process_consumer(self): |     def test_multi_process_consumer(self): | ||||||
|         # Produce 100 messages to partitions 0 and 1 |         # Produce 100 messages to partitions 0 and 1 | ||||||
|         self.send_messages(0, range(0, 100)) |         self.send_messages(0, range(0, 100)) | ||||||
| @@ -121,6 +132,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_multi_process_consumer_blocking(self): |     def test_multi_process_consumer_blocking(self): | ||||||
|         consumer = MultiProcessConsumer(self.client, "grp1", self.topic, auto_commit=False) |         consumer = MultiProcessConsumer(self.client, "grp1", self.topic, auto_commit=False) | ||||||
|  |  | ||||||
| @@ -148,6 +160,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_multi_proc_pending(self): |     def test_multi_proc_pending(self): | ||||||
|         self.send_messages(0, range(0, 10)) |         self.send_messages(0, range(0, 10)) | ||||||
|         self.send_messages(1, range(10, 20)) |         self.send_messages(1, range(10, 20)) | ||||||
| @@ -160,6 +173,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_large_messages(self): |     def test_large_messages(self): | ||||||
|         # Produce 10 "normal" size messages |         # Produce 10 "normal" size messages | ||||||
|         small_messages = self.send_messages(0, [ str(x) for x in range(10) ]) |         small_messages = self.send_messages(0, [ str(x) for x in range(10) ]) | ||||||
| @@ -177,6 +191,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         consumer.stop() |         consumer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_huge_messages(self): |     def test_huge_messages(self): | ||||||
|         huge_message, = self.send_messages(0, [ |         huge_message, = self.send_messages(0, [ | ||||||
|             create_message(random_string(MAX_FETCH_BUFFER_SIZE_BYTES + 10)), |             create_message(random_string(MAX_FETCH_BUFFER_SIZE_BYTES + 10)), | ||||||
| @@ -213,23 +228,25 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): | |||||||
|         msgs2 = self.send_messages(1, range(100, 200)) |         msgs2 = self.send_messages(1, range(100, 200)) | ||||||
|  |  | ||||||
|         # Start a consumer |         # Start a consumer | ||||||
|         consumer = SimpleConsumer(self.client, "group1", |         consumer1 = SimpleConsumer(self.client, "group1", | ||||||
|                                   self.topic, auto_commit=True, |                                   self.topic, auto_commit=True, | ||||||
|  |                                   auto_commit_every_t=600, | ||||||
|                                   auto_commit_every_n=20, |                                   auto_commit_every_n=20, | ||||||
|                                   iter_timeout=0) |                                   iter_timeout=0) | ||||||
|  |  | ||||||
|         # Grab the first 195 messages |         # Grab the first 195 messages | ||||||
|         output_msgs1 = [ consumer.get_message().message.value for _ in xrange(195) ] |         output_msgs1 = [ consumer1.get_message().message.value for _ in xrange(195) ] | ||||||
|         self.assert_message_count(output_msgs1, 195) |         self.assert_message_count(output_msgs1, 195) | ||||||
|         consumer.stop() |  | ||||||
|  |  | ||||||
|         # The offset should be at 180 |         # The offset should be at 180 | ||||||
|         consumer = SimpleConsumer(self.client, "group1", |         consumer2 = SimpleConsumer(self.client, "group1", | ||||||
|                                   self.topic, auto_commit=True, |                                   self.topic, auto_commit=True, | ||||||
|  |                                   auto_commit_every_t=600, | ||||||
|                                   auto_commit_every_n=20, |                                   auto_commit_every_n=20, | ||||||
|                                   iter_timeout=0) |                                   iter_timeout=0) | ||||||
|  |  | ||||||
|         # 180-200 |         # 180-200 | ||||||
|         self.assert_message_count([ message for message in consumer ], 20) |         self.assert_message_count([ message for message in consumer2 ], 20) | ||||||
|  |  | ||||||
|         consumer.stop() |         consumer1.stop() | ||||||
|  |         consumer2.stop() | ||||||
|   | |||||||
| @@ -1,17 +1,20 @@ | |||||||
| import unittest | import os | ||||||
| import time | import time | ||||||
|  | import unittest | ||||||
|  |  | ||||||
| from kafka import *  # noqa | from kafka import *  # noqa | ||||||
| from kafka.common import *  # noqa | from kafka.common import *  # noqa | ||||||
| from fixtures import ZookeeperFixture, KafkaFixture | from fixtures import ZookeeperFixture, KafkaFixture | ||||||
| from testutil import * | from testutil import * | ||||||
|  |  | ||||||
| @unittest.skipIf(skip_integration(), 'Skipping Integration') |  | ||||||
| class TestFailover(KafkaIntegrationTestCase): | class TestFailover(KafkaIntegrationTestCase): | ||||||
|     create_client = False |     create_client = False | ||||||
|  |  | ||||||
|     @classmethod |     @classmethod | ||||||
|     def setUpClass(cls):  # noqa |     def setUpClass(cls):  # noqa | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         zk_chroot = random_string(10) |         zk_chroot = random_string(10) | ||||||
|         replicas = 2 |         replicas = 2 | ||||||
|         partitions = 2 |         partitions = 2 | ||||||
| @@ -26,11 +29,15 @@ class TestFailover(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|     @classmethod |     @classmethod | ||||||
|     def tearDownClass(cls): |     def tearDownClass(cls): | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.client.close() |         cls.client.close() | ||||||
|         for broker in cls.brokers: |         for broker in cls.brokers: | ||||||
|             broker.close() |             broker.close() | ||||||
|         cls.zk.close() |         cls.zk.close() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_switch_leader(self): |     def test_switch_leader(self): | ||||||
|         key, topic, partition = random_string(5), self.topic, 0 |         key, topic, partition = random_string(5), self.topic, 0 | ||||||
|         producer = SimpleProducer(self.client) |         producer = SimpleProducer(self.client) | ||||||
| @@ -62,6 +69,7 @@ class TestFailover(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_switch_leader_async(self): |     def test_switch_leader_async(self): | ||||||
|         key, topic, partition = random_string(5), self.topic, 0 |         key, topic, partition = random_string(5), self.topic, 0 | ||||||
|         producer = SimpleProducer(self.client, async=True) |         producer = SimpleProducer(self.client, async=True) | ||||||
|   | |||||||
| @@ -1,6 +1,7 @@ | |||||||
| import uuid | import os | ||||||
| import time | import time | ||||||
| import unittest | import unittest | ||||||
|  | import uuid | ||||||
|  |  | ||||||
| from kafka import *  # noqa | from kafka import *  # noqa | ||||||
| from kafka.common import *  # noqa | from kafka.common import *  # noqa | ||||||
| @@ -13,14 +14,21 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|     @classmethod |     @classmethod | ||||||
|     def setUpClass(cls):  # noqa |     def setUpClass(cls):  # noqa | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.zk = ZookeeperFixture.instance() |         cls.zk = ZookeeperFixture.instance() | ||||||
|         cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) |         cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) | ||||||
|  |  | ||||||
|     @classmethod |     @classmethod | ||||||
|     def tearDownClass(cls):  # noqa |     def tearDownClass(cls):  # noqa | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         cls.server.close() |         cls.server.close() | ||||||
|         cls.zk.close() |         cls.zk.close() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_produce_many_simple(self): |     def test_produce_many_simple(self): | ||||||
|         start_offset = self.current_offset(self.topic, 0) |         start_offset = self.current_offset(self.topic, 0) | ||||||
|  |  | ||||||
| @@ -36,6 +44,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|             100, |             100, | ||||||
|         ) |         ) | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_produce_10k_simple(self): |     def test_produce_10k_simple(self): | ||||||
|         start_offset = self.current_offset(self.topic, 0) |         start_offset = self.current_offset(self.topic, 0) | ||||||
|  |  | ||||||
| @@ -45,6 +54,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|             10000, |             10000, | ||||||
|         ) |         ) | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_produce_many_gzip(self): |     def test_produce_many_gzip(self): | ||||||
|         start_offset = self.current_offset(self.topic, 0) |         start_offset = self.current_offset(self.topic, 0) | ||||||
|  |  | ||||||
| @@ -57,8 +67,9 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|             200, |             200, | ||||||
|         ) |         ) | ||||||
|  |  | ||||||
|     @unittest.skip("All snappy integration tests fail with nosnappyjava") |     @kafka_versions("all") | ||||||
|     def test_produce_many_snappy(self): |     def test_produce_many_snappy(self): | ||||||
|  |         self.skipTest("All snappy integration tests fail with nosnappyjava") | ||||||
|         start_offset = self.current_offset(self.topic, 0) |         start_offset = self.current_offset(self.topic, 0) | ||||||
|  |  | ||||||
|         self.assert_produce_request([ |         self.assert_produce_request([ | ||||||
| @@ -69,6 +80,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|             200, |             200, | ||||||
|         ) |         ) | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_produce_mixed(self): |     def test_produce_mixed(self): | ||||||
|         start_offset = self.current_offset(self.topic, 0) |         start_offset = self.current_offset(self.topic, 0) | ||||||
|  |  | ||||||
| @@ -85,6 +97,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         self.assert_produce_request(messages, start_offset, msg_count) |         self.assert_produce_request(messages, start_offset, msg_count) | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_produce_100k_gzipped(self): |     def test_produce_100k_gzipped(self): | ||||||
|         start_offset = self.current_offset(self.topic, 0) |         start_offset = self.current_offset(self.topic, 0) | ||||||
|  |  | ||||||
| @@ -106,6 +119,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|     #   SimpleProducer Tests   # |     #   SimpleProducer Tests   # | ||||||
|     ############################ |     ############################ | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_simple_producer(self): |     def test_simple_producer(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -130,6 +144,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_round_robin_partitioner(self): |     def test_round_robin_partitioner(self): | ||||||
|         msg1, msg2, msg3, msg4 = [ str(uuid.uuid4()) for _ in range(4) ] |         msg1, msg2, msg3, msg4 = [ str(uuid.uuid4()) for _ in range(4) ] | ||||||
|  |  | ||||||
| @@ -152,6 +167,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_hashed_partitioner(self): |     def test_hashed_partitioner(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -174,6 +190,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_acks_none(self): |     def test_acks_none(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -185,6 +202,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|         self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) |         self.assert_fetch_offset(0, start_offset0, [ self.msg("one") ]) | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_acks_local_write(self): |     def test_acks_local_write(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -197,6 +215,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_acks_cluster_commit(self): |     def test_acks_cluster_commit(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -211,6 +230,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_batched_simple_producer__triggers_by_message(self): |     def test_batched_simple_producer__triggers_by_message(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -259,6 +279,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_batched_simple_producer__triggers_by_time(self): |     def test_batched_simple_producer__triggers_by_time(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -310,6 +331,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_async_simple_producer(self): |     def test_async_simple_producer(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
| @@ -322,6 +344,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): | |||||||
|  |  | ||||||
|         producer.stop() |         producer.stop() | ||||||
|  |  | ||||||
|  |     @kafka_versions("all") | ||||||
|     def test_async_keyed_producer(self): |     def test_async_keyed_producer(self): | ||||||
|         start_offset0 = self.current_offset(self.topic, 0) |         start_offset0 = self.current_offset(self.topic, 0) | ||||||
|         start_offset1 = self.current_offset(self.topic, 1) |         start_offset1 = self.current_offset(self.topic, 1) | ||||||
|   | |||||||
| @@ -12,7 +12,7 @@ from kafka.common import ( | |||||||
|     LeaderUnavailableError, PartitionUnavailableError |     LeaderUnavailableError, PartitionUnavailableError | ||||||
| ) | ) | ||||||
| from kafka.codec import ( | from kafka.codec import ( | ||||||
|     has_gzip, has_snappy, gzip_encode, gzip_decode, |     has_snappy, gzip_encode, gzip_decode, | ||||||
|     snappy_encode, snappy_decode |     snappy_encode, snappy_decode | ||||||
| ) | ) | ||||||
| from kafka.protocol import ( | from kafka.protocol import ( | ||||||
| @@ -29,7 +29,6 @@ class TestProtocol(unittest.TestCase): | |||||||
|         self.assertEqual(msg.key, key) |         self.assertEqual(msg.key, key) | ||||||
|         self.assertEqual(msg.value, payload) |         self.assertEqual(msg.value, payload) | ||||||
|  |  | ||||||
|     @unittest.skipUnless(has_gzip(), "gzip not available") |  | ||||||
|     def test_create_gzip(self): |     def test_create_gzip(self): | ||||||
|         payloads = ["v1", "v2"] |         payloads = ["v1", "v2"] | ||||||
|         msg = create_gzip_message(payloads) |         msg = create_gzip_message(payloads) | ||||||
| @@ -197,7 +196,6 @@ class TestProtocol(unittest.TestCase): | |||||||
|         self.assertEqual(returned_offset2, 1) |         self.assertEqual(returned_offset2, 1) | ||||||
|         self.assertEqual(decoded_message2, create_message("v2", "k2")) |         self.assertEqual(decoded_message2, create_message("v2", "k2")) | ||||||
|  |  | ||||||
|     @unittest.skipUnless(has_gzip(), "Gzip not available") |  | ||||||
|     def test_decode_message_gzip(self): |     def test_decode_message_gzip(self): | ||||||
|         gzip_encoded = ('\xc0\x11\xb2\xf0\x00\x01\xff\xff\xff\xff\x00\x00\x000' |         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' |                         '\x1f\x8b\x08\x00\xa1\xc1\xc5R\x02\xffc`\x80\x03\x01' | ||||||
|   | |||||||
| @@ -13,7 +13,6 @@ from kafka import KafkaClient | |||||||
|  |  | ||||||
| __all__ = [ | __all__ = [ | ||||||
|     'random_string', |     'random_string', | ||||||
|     'skip_integration', |  | ||||||
|     'ensure_topic_creation', |     'ensure_topic_creation', | ||||||
|     'get_open_port', |     'get_open_port', | ||||||
|     'kafka_versions', |     'kafka_versions', | ||||||
| @@ -25,15 +24,17 @@ def random_string(l): | |||||||
|     s = "".join(random.choice(string.letters) for i in xrange(l)) |     s = "".join(random.choice(string.letters) for i in xrange(l)) | ||||||
|     return s |     return s | ||||||
|  |  | ||||||
| def skip_integration(): |  | ||||||
|     return os.environ.get('SKIP_INTEGRATION') |  | ||||||
|  |  | ||||||
| def kafka_versions(*versions): | def kafka_versions(*versions): | ||||||
|     def kafka_versions(func): |     def kafka_versions(func): | ||||||
|         @functools.wraps(func) |         @functools.wraps(func) | ||||||
|         def wrapper(self): |         def wrapper(self): | ||||||
|             if os.environ.get('KAFKA_VERSION', None) not in versions: |             kafka_version = os.environ.get('KAFKA_VERSION') | ||||||
|  |  | ||||||
|  |             if not kafka_version: | ||||||
|  |                 self.skipTest("no kafka version specified") | ||||||
|  |             elif 'all' not in versions and kafka_version not in versions: | ||||||
|                 self.skipTest("unsupported kafka version") |                 self.skipTest("unsupported kafka version") | ||||||
|  |  | ||||||
|             return func(self) |             return func(self) | ||||||
|         return wrapper |         return wrapper | ||||||
|     return kafka_versions |     return kafka_versions | ||||||
| @@ -61,6 +62,9 @@ class KafkaIntegrationTestCase(unittest.TestCase): | |||||||
|  |  | ||||||
|     def setUp(self): |     def setUp(self): | ||||||
|         super(KafkaIntegrationTestCase, self).setUp() |         super(KafkaIntegrationTestCase, self).setUp() | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         if not self.topic: |         if not self.topic: | ||||||
|             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)) | ||||||
|  |  | ||||||
| @@ -73,6 +77,9 @@ class KafkaIntegrationTestCase(unittest.TestCase): | |||||||
|  |  | ||||||
|     def tearDown(self): |     def tearDown(self): | ||||||
|         super(KafkaIntegrationTestCase, self).tearDown() |         super(KafkaIntegrationTestCase, self).tearDown() | ||||||
|  |         if not os.environ.get('KAFKA_VERSION'): | ||||||
|  |             return | ||||||
|  |  | ||||||
|         if self.create_client: |         if self.create_client: | ||||||
|             self.client.close() |             self.client.close() | ||||||
|  |  | ||||||
|   | |||||||
		Reference in New Issue
	
	Block a user
	 Mark Roberts
					Mark Roberts