diff options
Diffstat (limited to 'test')
-rw-r--r-- | test/test_client.py | 17 | ||||
-rw-r--r-- | test/test_client_integration.py | 28 | ||||
-rw-r--r-- | test/test_consumer_integration.py | 2 | ||||
-rw-r--r-- | test/test_failover_integration.py | 5 | ||||
-rw-r--r-- | test/test_producer_integration.py | 4 | ||||
-rw-r--r-- | test/testutil.py | 5 |
6 files changed, 28 insertions, 33 deletions
diff --git a/test/test_client.py b/test/test_client.py index a3e04f4..dd8948f 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -133,12 +133,12 @@ class TestKafkaClient(unittest.TestCase): # client loads metadata at init client = KafkaClient(hosts=['broker_1:4567']) self.assertDictEqual({ - TopicAndPartition(b'topic_1', 0): brokers[1], - TopicAndPartition(b'topic_noleader', 0): None, - TopicAndPartition(b'topic_noleader', 1): None, - TopicAndPartition(b'topic_3', 0): brokers[0], - TopicAndPartition(b'topic_3', 1): brokers[1], - TopicAndPartition(b'topic_3', 2): brokers[0]}, + TopicAndPartition('topic_1', 0): brokers[1], + TopicAndPartition('topic_noleader', 0): None, + TopicAndPartition('topic_noleader', 1): None, + TopicAndPartition('topic_3', 0): brokers[0], + TopicAndPartition('topic_3', 1): brokers[1], + TopicAndPartition('topic_3', 2): brokers[0]}, client.topics_to_brokers) # if we ask for metadata explicitly, it should raise errors @@ -150,7 +150,6 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.load_metadata_for_topics('topic_no_leader') - client.load_metadata_for_topics(b'topic_no_leader') @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') @@ -274,10 +273,10 @@ class TestKafkaClient(unittest.TestCase): self.assertDictEqual({}, client.topics_to_brokers) with self.assertRaises(LeaderNotAvailableError): - client._get_leader_for_partition(b'topic_no_partitions', 0) + client._get_leader_for_partition('topic_no_partitions', 0) with self.assertRaises(UnknownTopicOrPartitionError): - client._get_leader_for_partition(b'topic_unknown', 0) + client._get_leader_for_partition('topic_unknown', 0) @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') diff --git a/test/test_client_integration.py b/test/test_client_integration.py index edd62da..1599006 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -29,11 +29,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_consume_none(self): - fetch = FetchRequestPayload(self.bytes_topic, 0, 0, 1024) + fetch = FetchRequestPayload(self.topic, 0, 0, 1024) fetch_resp, = self.client.send_fetch_request([fetch]) self.assertEqual(fetch_resp.error, 0) - self.assertEqual(fetch_resp.topic, self.bytes_topic) + self.assertEqual(fetch_resp.topic, self.topic) self.assertEqual(fetch_resp.partition, 0) messages = list(fetch_resp.messages) @@ -48,26 +48,26 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): # ensure_topic_exists should fail with KafkaTimeoutError with self.assertRaises(KafkaTimeoutError): - self.client.ensure_topic_exists(b"this_topic_doesnt_exist", timeout=0) + self.client.ensure_topic_exists('this_topic_doesnt_exist', timeout=0) @kafka_versions('all') def test_send_produce_request_maintains_request_response_order(self): - self.client.ensure_topic_exists(b'foo') - self.client.ensure_topic_exists(b'bar') + self.client.ensure_topic_exists('foo') + self.client.ensure_topic_exists('bar') requests = [ ProduceRequestPayload( - b'foo', 0, + 'foo', 0, [create_message(b'a'), create_message(b'b')]), ProduceRequestPayload( - b'bar', 1, + 'bar', 1, [create_message(b'a'), create_message(b'b')]), ProduceRequestPayload( - b'foo', 1, + 'foo', 1, [create_message(b'a'), create_message(b'b')]), ProduceRequestPayload( - b'bar', 0, + 'bar', 0, [create_message(b'a'), create_message(b'b')]), ] @@ -85,12 +85,12 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_commit_fetch_offsets(self): - req = OffsetCommitRequestPayload(self.bytes_topic, 0, 42, b"metadata") - (resp,) = self.client.send_offset_commit_request(b"group", [req]) + req = OffsetCommitRequestPayload(self.topic, 0, 42, 'metadata') + (resp,) = self.client.send_offset_commit_request('group', [req]) self.assertEqual(resp.error, 0) - req = OffsetFetchRequestPayload(self.bytes_topic, 0) - (resp,) = self.client.send_offset_fetch_request(b"group", [req]) + req = OffsetFetchRequestPayload(self.topic, 0) + (resp,) = self.client.send_offset_fetch_request('group', [req]) self.assertEqual(resp.error, 0) self.assertEqual(resp.offset, 42) - self.assertEqual(resp.metadata, b"") # Metadata isn't stored for now + self.assertEqual(resp.metadata, '') # Metadata isn't stored for now diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 4cebed8..43370e2 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -41,7 +41,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def send_messages(self, partition, messages): messages = [ create_message(self.msg(str(msg))) for msg in messages ] - produce = ProduceRequestPayload(self.bytes_topic, partition, messages = messages) + produce = ProduceRequestPayload(self.topic, partition, messages = messages) resp, = self.client.send_produce_request([produce]) self.assertEqual(resp.error, 0) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 91779d7..8c5efe2 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -5,7 +5,6 @@ import time from kafka import KafkaClient, SimpleConsumer, KeyedProducer from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer -from kafka.util import kafka_bytestring from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( @@ -165,7 +164,7 @@ class TestFailover(KafkaIntegrationTestCase): key = random_string(3).encode('utf-8') msg = random_string(10).encode('utf-8') producer.send_messages(topic, key, msg) - if producer.partitioners[kafka_bytestring(topic)].partition(key) == 0: + if producer.partitioners[topic].partition(key) == 0: recovered = True except (FailedPayloadsError, ConnectionError): log.debug("caught exception sending message -- will retry") @@ -203,7 +202,7 @@ class TestFailover(KafkaIntegrationTestCase): break def _kill_leader(self, topic, partition): - leader = self.client.topics_to_brokers[TopicAndPartition(kafka_bytestring(topic), partition)] + leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] broker = self.brokers[leader.nodeId] broker.close() return broker diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index ee0b2fd..d09c1af 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -488,7 +488,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): def assert_produce_request(self, messages, initial_offset, message_ct, partition=0): - produce = ProduceRequestPayload(self.bytes_topic, partition, messages=messages) + produce = ProduceRequestPayload(self.topic, partition, messages=messages) # There should only be one response message from the server. # This will throw an exception if there's more than one. @@ -506,7 +506,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # There should only be one response message from the server. # This will throw an exception if there's more than one. - resp, = self.client.send_fetch_request([FetchRequestPayload(self.bytes_topic, partition, start_offset, 1024)]) + resp, = self.client.send_fetch_request([FetchRequestPayload(self.topic, partition, start_offset, 1024)]) self.assertEqual(resp.error, 0) self.assertEqual(resp.partition, partition) diff --git a/test/testutil.py b/test/testutil.py index b5b2529..5c6ea1b 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -12,7 +12,6 @@ from . import unittest from kafka import KafkaClient from kafka.common import OffsetRequestPayload -from kafka.util import kafka_bytestring __all__ = [ 'random_string', @@ -50,7 +49,6 @@ def get_open_port(): class KafkaIntegrationTestCase(unittest.TestCase): create_client = True topic = None - bytes_topic = None zk = None server = None @@ -62,7 +60,6 @@ class KafkaIntegrationTestCase(unittest.TestCase): if not self.topic: topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10)) self.topic = topic - self.bytes_topic = topic.encode('utf-8') if self.create_client: self.client = KafkaClient('%s:%d' % (self.server.host, self.server.port)) @@ -81,7 +78,7 @@ class KafkaIntegrationTestCase(unittest.TestCase): def current_offset(self, topic, partition): try: - offsets, = self.client.send_offset_request([OffsetRequestPayload(kafka_bytestring(topic), partition, -1, 1)]) + offsets, = self.client.send_offset_request([OffsetRequestPayload(topic, partition, -1, 1)]) except: # XXX: We've seen some UnknownErrors here and cant debug w/o server logs self.zk.child.dump_logs() |