summaryrefslogtreecommitdiff
path: root/test
diff options
context:
space:
mode:
Diffstat (limited to 'test')
-rw-r--r--test/test_client.py70
-rw-r--r--test/test_client_integration.py8
-rw-r--r--test/test_consumer_integration.py2
-rw-r--r--test/test_failover_integration.py5
-rw-r--r--test/test_producer_integration.py4
-rw-r--r--test/testutil.py8
6 files changed, 52 insertions, 45 deletions
diff --git a/test/test_client.py b/test/test_client.py
index abda421..bab7916 100644
--- a/test/test_client.py
+++ b/test/test_client.py
@@ -117,21 +117,21 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_1', NO_ERROR, [
- PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2], NO_ERROR)
+ TopicMetadata(b'topic_1', NO_ERROR, [
+ PartitionMetadata(b'topic_1', 0, 1, [1, 2], [1, 2], NO_ERROR)
]),
- TopicMetadata('topic_noleader', NO_ERROR, [
- PartitionMetadata('topic_noleader', 0, -1, [], [],
+ TopicMetadata(b'topic_noleader', NO_ERROR, [
+ PartitionMetadata(b'topic_noleader', 0, -1, [], [],
NO_LEADER),
- PartitionMetadata('topic_noleader', 1, -1, [], [],
+ PartitionMetadata(b'topic_noleader', 1, -1, [], [],
NO_LEADER),
]),
- TopicMetadata('topic_no_partitions', NO_LEADER, []),
- TopicMetadata('topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []),
- TopicMetadata('topic_3', NO_ERROR, [
- PartitionMetadata('topic_3', 0, 0, [0, 1], [0, 1], NO_ERROR),
- PartitionMetadata('topic_3', 1, 1, [1, 0], [1, 0], NO_ERROR),
- PartitionMetadata('topic_3', 2, 0, [0, 1], [0, 1], NO_ERROR)
+ TopicMetadata(b'topic_no_partitions', NO_LEADER, []),
+ TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []),
+ TopicMetadata(b'topic_3', NO_ERROR, [
+ PartitionMetadata(b'topic_3', 0, 0, [0, 1], [0, 1], NO_ERROR),
+ PartitionMetadata(b'topic_3', 1, 1, [1, 0], [1, 0], NO_ERROR),
+ PartitionMetadata(b'topic_3', 2, 0, [0, 1], [0, 1], NO_ERROR)
])
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
@@ -139,12 +139,12 @@ class TestKafkaClient(unittest.TestCase):
# client loads metadata at init
client = KafkaClient(hosts=['broker_1:4567'])
self.assertDictEqual({
- TopicAndPartition('topic_1', 0): brokers[1],
- TopicAndPartition('topic_noleader', 0): None,
- TopicAndPartition('topic_noleader', 1): None,
- TopicAndPartition('topic_3', 0): brokers[0],
- TopicAndPartition('topic_3', 1): brokers[1],
- TopicAndPartition('topic_3', 2): brokers[0]},
+ TopicAndPartition(b'topic_1', 0): brokers[1],
+ TopicAndPartition(b'topic_noleader', 0): None,
+ TopicAndPartition(b'topic_noleader', 1): None,
+ TopicAndPartition(b'topic_3', 0): brokers[0],
+ TopicAndPartition(b'topic_3', 1): brokers[1],
+ TopicAndPartition(b'topic_3', 2): brokers[0]},
client.topics_to_brokers)
# if we ask for metadata explicitly, it should raise errors
@@ -156,6 +156,7 @@ class TestKafkaClient(unittest.TestCase):
# This should not raise
client.load_metadata_for_topics('topic_no_leader')
+ client.load_metadata_for_topics(b'topic_no_leader')
@patch('kafka.client.KafkaConnection')
@patch('kafka.client.KafkaProtocol')
@@ -169,11 +170,11 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_still_creating', NO_LEADER, []),
- TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []),
- TopicMetadata('topic_noleaders', NO_ERROR, [
- PartitionMetadata('topic_noleaders', 0, -1, [], [], NO_LEADER),
- PartitionMetadata('topic_noleaders', 1, -1, [], [], NO_LEADER),
+ TopicMetadata(b'topic_still_creating', NO_LEADER, []),
+ TopicMetadata(b'topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []),
+ TopicMetadata(b'topic_noleaders', NO_ERROR, [
+ PartitionMetadata(b'topic_noleaders', 0, -1, [], [], NO_LEADER),
+ PartitionMetadata(b'topic_noleaders', 1, -1, [], [], NO_LEADER),
]),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
@@ -188,8 +189,8 @@ class TestKafkaClient(unittest.TestCase):
self.assertTrue(client.has_metadata_for_topic('topic_noleaders'))
@patch('kafka.client.KafkaConnection')
- @patch('kafka.client.KafkaProtocol')
- def test_ensure_topic_exists(self, protocol, conn):
+ @patch('kafka.client.KafkaProtocol.decode_metadata_response')
+ def test_ensure_topic_exists(self, decode_metadata_response, conn):
conn.recv.return_value = 'response' # anything but None
@@ -199,14 +200,14 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_still_creating', NO_LEADER, []),
- TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []),
- TopicMetadata('topic_noleaders', NO_ERROR, [
- PartitionMetadata('topic_noleaders', 0, -1, [], [], NO_LEADER),
- PartitionMetadata('topic_noleaders', 1, -1, [], [], NO_LEADER),
+ TopicMetadata(b'topic_still_creating', NO_LEADER, []),
+ TopicMetadata(b'topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []),
+ TopicMetadata(b'topic_noleaders', NO_ERROR, [
+ PartitionMetadata(b'topic_noleaders', 0, -1, [], [], NO_LEADER),
+ PartitionMetadata(b'topic_noleaders', 1, -1, [], [], NO_LEADER),
]),
]
- protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
+ decode_metadata_response.return_value = MetadataResponse(brokers, topics)
client = KafkaClient(hosts=['broker_1:4567'])
@@ -218,6 +219,7 @@ class TestKafkaClient(unittest.TestCase):
# This should not raise
client.ensure_topic_exists('topic_noleaders', timeout=1)
+ client.ensure_topic_exists(b'topic_noleaders', timeout=1)
@patch('kafka.client.KafkaConnection')
@patch('kafka.client.KafkaProtocol')
@@ -269,8 +271,8 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_no_partitions', NO_LEADER, []),
- TopicMetadata('topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []),
+ TopicMetadata(b'topic_no_partitions', NO_LEADER, []),
+ TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
@@ -279,10 +281,10 @@ class TestKafkaClient(unittest.TestCase):
self.assertDictEqual({}, client.topics_to_brokers)
with self.assertRaises(LeaderNotAvailableError):
- client._get_leader_for_partition('topic_no_partitions', 0)
+ client._get_leader_for_partition(b'topic_no_partitions', 0)
with self.assertRaises(UnknownTopicOrPartitionError):
- client._get_leader_for_partition('topic_unknown', 0)
+ client._get_leader_for_partition(b'topic_unknown', 0)
@patch('kafka.client.KafkaConnection')
@patch('kafka.client.KafkaProtocol')
diff --git a/test/test_client_integration.py b/test/test_client_integration.py
index c0331ea..d963d85 100644
--- a/test/test_client_integration.py
+++ b/test/test_client_integration.py
@@ -29,11 +29,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase):
@kafka_versions("all")
def test_consume_none(self):
- fetch = FetchRequest(self.topic, 0, 0, 1024)
+ fetch = FetchRequest(self.bytes_topic, 0, 0, 1024)
fetch_resp, = self.client.send_fetch_request([fetch])
self.assertEqual(fetch_resp.error, 0)
- self.assertEqual(fetch_resp.topic, self.topic)
+ self.assertEqual(fetch_resp.topic, self.bytes_topic)
self.assertEqual(fetch_resp.partition, 0)
messages = list(fetch_resp.messages)
@@ -56,11 +56,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase):
@kafka_versions("0.8.1", "0.8.1.1", "0.8.2.0")
def test_commit_fetch_offsets(self):
- req = OffsetCommitRequest(self.topic, 0, 42, b"metadata")
+ req = OffsetCommitRequest(self.bytes_topic, 0, 42, b"metadata")
(resp,) = self.client.send_offset_commit_request(b"group", [req])
self.assertEqual(resp.error, 0)
- req = OffsetFetchRequest(self.topic, 0)
+ req = OffsetFetchRequest(self.bytes_topic, 0)
(resp,) = self.client.send_offset_fetch_request(b"group", [req])
self.assertEqual(resp.error, 0)
self.assertEqual(resp.offset, 42)
diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py
index 403ce0f..ddb54a7 100644
--- a/test/test_consumer_integration.py
+++ b/test/test_consumer_integration.py
@@ -37,7 +37,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase):
def send_messages(self, partition, messages):
messages = [ create_message(self.msg(str(msg))) for msg in messages ]
- produce = ProduceRequest(self.topic, partition, messages = messages)
+ produce = ProduceRequest(self.bytes_topic, partition, messages = messages)
resp, = self.client.send_produce_request([produce])
self.assertEqual(resp.error, 0)
diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py
index 15f0338..3be0189 100644
--- a/test/test_failover_integration.py
+++ b/test/test_failover_integration.py
@@ -8,6 +8,7 @@ from kafka import KafkaClient, SimpleConsumer
from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError
from kafka.producer.base import Producer
from kafka.producer import KeyedProducer
+from kafka.util import kafka_bytestring
from test.fixtures import ZookeeperFixture, KafkaFixture
from test.testutil import (
@@ -147,7 +148,7 @@ class TestFailover(KafkaIntegrationTestCase):
key = random_string(3)
msg = random_string(10)
producer.send_messages(topic, key, msg)
- if producer.partitioners[topic].partition(key) == 0:
+ if producer.partitioners[kafka_bytestring(topic)].partition(key) == 0:
recovered = True
except (FailedPayloadsError, ConnectionError):
logging.debug("caught exception sending message -- will retry")
@@ -172,7 +173,7 @@ class TestFailover(KafkaIntegrationTestCase):
logging.debug('_send_random_message to %s:%d -- try %d success', topic, partition, j)
def _kill_leader(self, topic, partition):
- leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)]
+ leader = self.client.topics_to_brokers[TopicAndPartition(kafka_bytestring(topic), partition)]
broker = self.brokers[leader.nodeId]
broker.close()
return broker
diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py
index 1804af0..e3f7767 100644
--- a/test/test_producer_integration.py
+++ b/test/test_producer_integration.py
@@ -453,7 +453,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase):
def assert_produce_request(self, messages, initial_offset, message_ct,
partition=0):
- produce = ProduceRequest(self.topic, partition, messages=messages)
+ produce = ProduceRequest(self.bytes_topic, partition, messages=messages)
# There should only be one response message from the server.
# This will throw an exception if there's more than one.
@@ -471,7 +471,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase):
# There should only be one response message from the server.
# This will throw an exception if there's more than one.
- resp, = self.client.send_fetch_request([ FetchRequest(self.topic, partition, start_offset, 1024) ])
+ resp, = self.client.send_fetch_request([ FetchRequest(self.bytes_topic, partition, start_offset, 1024) ])
self.assertEqual(resp.error, 0)
self.assertEqual(resp.partition, partition)
diff --git a/test/testutil.py b/test/testutil.py
index 7661cbc..e6947b4 100644
--- a/test/testutil.py
+++ b/test/testutil.py
@@ -12,6 +12,7 @@ from . import unittest
from kafka import KafkaClient
from kafka.common import OffsetRequest
+from kafka.util import kafka_bytestring
__all__ = [
'random_string',
@@ -50,6 +51,7 @@ def get_open_port():
class KafkaIntegrationTestCase(unittest.TestCase):
create_client = True
topic = None
+ bytes_topic = None
server = None
def setUp(self):
@@ -59,7 +61,8 @@ class KafkaIntegrationTestCase(unittest.TestCase):
if not self.topic:
topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10).decode('utf-8'))
- self.topic = topic.encode('utf-8')
+ self.topic = topic
+ self.bytes_topic = topic.encode('utf-8')
if self.create_client:
self.client = KafkaClient('%s:%d' % (self.server.host, self.server.port))
@@ -77,7 +80,8 @@ class KafkaIntegrationTestCase(unittest.TestCase):
self.client.close()
def current_offset(self, topic, partition):
- offsets, = self.client.send_offset_request([ OffsetRequest(topic, partition, -1, 1) ])
+ offsets, = self.client.send_offset_request([ OffsetRequest(kafka_bytestring(topic),
+ partition, -1, 1) ])
return offsets.offsets[0]
def msgs(self, iterable):