summaryrefslogtreecommitdiff
path: root/test/test_client.py
diff options
context:
space:
mode:
authorDana Powers <dana.powers@rd.io>2016-01-07 18:51:14 -0800
committerDana Powers <dana.powers@rd.io>2016-01-07 18:51:14 -0800
commit828377377da43749af0d27ee256ef31bf714cf17 (patch)
treefbad4d4381fc4d1ea2be7ce2009214d18fbeb674 /test/test_client.py
parent71e7568fcb8132899f366b37c32645fd5a40dc4b (diff)
parent9a8af1499ca425366d934487469d9977fae7fe5f (diff)
downloadkafka-python-828377377da43749af0d27ee256ef31bf714cf17.tar.gz
Merge branch '0.9'
Conflicts: kafka/codec.py kafka/version.py test/test_producer.py test/test_producer_integration.py
Diffstat (limited to 'test/test_client.py')
-rw-r--r--test/test_client.py254
1 files changed, 126 insertions, 128 deletions
diff --git a/test/test_client.py b/test/test_client.py
index bab7916..5a35c83 100644
--- a/test/test_client.py
+++ b/test/test_client.py
@@ -5,16 +5,18 @@ from mock import ANY, MagicMock, patch
import six
from . import unittest
-from kafka import KafkaClient
+from kafka import SimpleClient
from kafka.common import (
- ProduceRequest, MetadataResponse,
- BrokerMetadata, TopicMetadata, PartitionMetadata,
- TopicAndPartition, KafkaUnavailableError,
+ ProduceRequestPayload,
+ BrokerMetadata,
+ TopicPartition, KafkaUnavailableError,
LeaderNotAvailableError, UnknownTopicOrPartitionError,
KafkaTimeoutError, ConnectionError
)
from kafka.conn import KafkaConnection
+from kafka.future import Future
from kafka.protocol import KafkaProtocol, create_message
+from kafka.protocol.metadata import MetadataResponse
from test.testutil import Timer
@@ -22,94 +24,100 @@ NO_ERROR = 0
UNKNOWN_TOPIC_OR_PARTITION = 3
NO_LEADER = 5
-class TestKafkaClient(unittest.TestCase):
+
+def mock_conn(conn, success=True):
+ mocked = MagicMock()
+ mocked.connected.return_value = True
+ if success:
+ mocked.send.return_value = Future().success(True)
+ else:
+ mocked.send.return_value = Future().failure(Exception())
+ conn.return_value = mocked
+
+
+class TestSimpleClient(unittest.TestCase):
def test_init_with_list(self):
- with patch.object(KafkaClient, 'load_metadata_for_topics'):
- client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092', 'kafka03:9092'])
+ with patch.object(SimpleClient, 'load_metadata_for_topics'):
+ client = SimpleClient(hosts=['kafka01:9092', 'kafka02:9092', 'kafka03:9092'])
self.assertEqual(
sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]),
sorted(client.hosts))
def test_init_with_csv(self):
- with patch.object(KafkaClient, 'load_metadata_for_topics'):
- client = KafkaClient(hosts='kafka01:9092,kafka02:9092,kafka03:9092')
+ with patch.object(SimpleClient, 'load_metadata_for_topics'):
+ client = SimpleClient(hosts='kafka01:9092,kafka02:9092,kafka03:9092')
self.assertEqual(
sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]),
sorted(client.hosts))
def test_init_with_unicode_csv(self):
- with patch.object(KafkaClient, 'load_metadata_for_topics'):
- client = KafkaClient(hosts=u'kafka01:9092,kafka02:9092,kafka03:9092')
+ with patch.object(SimpleClient, 'load_metadata_for_topics'):
+ client = SimpleClient(hosts=u'kafka01:9092,kafka02:9092,kafka03:9092')
self.assertEqual(
sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]),
sorted(client.hosts))
- def test_send_broker_unaware_request_fail(self):
- 'Tests that call fails when all hosts are unavailable'
-
+ @patch.object(SimpleClient, '_get_conn')
+ @patch.object(SimpleClient, 'load_metadata_for_topics')
+ def test_send_broker_unaware_request_fail(self, load_metadata, conn):
mocked_conns = {
('kafka01', 9092): MagicMock(),
('kafka02', 9092): MagicMock()
}
-
- # inject KafkaConnection side effects
- mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)")
- mocked_conns[('kafka02', 9092)].send.side_effect = RuntimeError("Kafka02 went away (unittest)")
+ for val in mocked_conns.values():
+ mock_conn(val, success=False)
def mock_get_conn(host, port):
return mocked_conns[(host, port)]
+ conn.side_effect = mock_get_conn
- # patch to avoid making requests before we want it
- with patch.object(KafkaClient, 'load_metadata_for_topics'):
- with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn):
- client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092'])
+ client = SimpleClient(hosts=['kafka01:9092', 'kafka02:9092'])
- req = KafkaProtocol.encode_metadata_request(b'client', 0)
- with self.assertRaises(KafkaUnavailableError):
- client._send_broker_unaware_request(payloads=['fake request'],
- encoder_fn=MagicMock(return_value='fake encoded message'),
- decoder_fn=lambda x: x)
+ req = KafkaProtocol.encode_metadata_request()
+ with self.assertRaises(KafkaUnavailableError):
+ client._send_broker_unaware_request(payloads=['fake request'],
+ encoder_fn=MagicMock(return_value='fake encoded message'),
+ decoder_fn=lambda x: x)
- for key, conn in six.iteritems(mocked_conns):
- conn.send.assert_called_with(ANY, 'fake encoded message')
+ for key, conn in six.iteritems(mocked_conns):
+ conn.send.assert_called_with('fake encoded message')
def test_send_broker_unaware_request(self):
- 'Tests that call works when at least one of the host is available'
-
mocked_conns = {
('kafka01', 9092): MagicMock(),
('kafka02', 9092): MagicMock(),
('kafka03', 9092): MagicMock()
}
# inject KafkaConnection side effects
- mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)")
- mocked_conns[('kafka02', 9092)].recv.return_value = 'valid response'
- mocked_conns[('kafka03', 9092)].send.side_effect = RuntimeError("kafka03 went away (unittest)")
+ mock_conn(mocked_conns[('kafka01', 9092)], success=False)
+ mock_conn(mocked_conns[('kafka03', 9092)], success=False)
+ future = Future()
+ mocked_conns[('kafka02', 9092)].send.return_value = future
+ mocked_conns[('kafka02', 9092)].recv.side_effect = lambda: future.success('valid response')
def mock_get_conn(host, port):
return mocked_conns[(host, port)]
# patch to avoid making requests before we want it
- with patch.object(KafkaClient, 'load_metadata_for_topics'):
- with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn):
- with patch.object(KafkaClient, '_next_id', return_value=1):
- client = KafkaClient(hosts='kafka01:9092,kafka02:9092')
+ with patch.object(SimpleClient, 'load_metadata_for_topics'):
+ with patch.object(SimpleClient, '_get_conn', side_effect=mock_get_conn):
- resp = client._send_broker_unaware_request(payloads=['fake request'],
- encoder_fn=MagicMock(),
- decoder_fn=lambda x: x)
+ client = SimpleClient(hosts='kafka01:9092,kafka02:9092')
+ resp = client._send_broker_unaware_request(payloads=['fake request'],
+ encoder_fn=MagicMock(),
+ decoder_fn=lambda x: x)
- self.assertEqual('valid response', resp)
- mocked_conns[('kafka02', 9092)].recv.assert_called_with(1)
+ self.assertEqual('valid response', resp)
+ mocked_conns[('kafka02', 9092)].recv.assert_called_once_with()
- @patch('kafka.client.KafkaConnection')
+ @patch('kafka.SimpleClient._get_conn')
@patch('kafka.client.KafkaProtocol')
def test_load_metadata(self, protocol, conn):
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -117,34 +125,32 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata(b'topic_1', NO_ERROR, [
- PartitionMetadata(b'topic_1', 0, 1, [1, 2], [1, 2], NO_ERROR)
+ (NO_ERROR, 'topic_1', [
+ (NO_ERROR, 0, 1, [1, 2], [1, 2])
]),
- TopicMetadata(b'topic_noleader', NO_ERROR, [
- PartitionMetadata(b'topic_noleader', 0, -1, [], [],
- NO_LEADER),
- PartitionMetadata(b'topic_noleader', 1, -1, [], [],
- NO_LEADER),
+ (NO_ERROR, 'topic_noleader', [
+ (NO_LEADER, 0, -1, [], []),
+ (NO_LEADER, 1, -1, [], []),
]),
- 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)
+ (NO_LEADER, 'topic_no_partitions', []),
+ (UNKNOWN_TOPIC_OR_PARTITION, 'topic_unknown', []),
+ (NO_ERROR, 'topic_3', [
+ (NO_ERROR, 0, 0, [0, 1], [0, 1]),
+ (NO_ERROR, 1, 1, [1, 0], [1, 0]),
+ (NO_ERROR, 2, 0, [0, 1], [0, 1])
])
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
# client loads metadata at init
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(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]},
+ TopicPartition('topic_1', 0): brokers[1],
+ TopicPartition('topic_noleader', 0): None,
+ TopicPartition('topic_noleader', 1): None,
+ TopicPartition('topic_3', 0): brokers[0],
+ TopicPartition('topic_3', 1): brokers[1],
+ TopicPartition('topic_3', 2): brokers[0]},
client.topics_to_brokers)
# if we ask for metadata explicitly, it should raise errors
@@ -156,13 +162,12 @@ 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.SimpleClient._get_conn')
@patch('kafka.client.KafkaProtocol')
def test_has_metadata_for_topic(self, protocol, conn):
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -170,16 +175,16 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- 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),
+ (NO_LEADER, 'topic_still_creating', []),
+ (UNKNOWN_TOPIC_OR_PARTITION, 'topic_doesnt_exist', []),
+ (NO_ERROR, 'topic_noleaders', [
+ (NO_LEADER, 0, -1, [], []),
+ (NO_LEADER, 1, -1, [], []),
]),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
# Topics with no partitions return False
self.assertFalse(client.has_metadata_for_topic('topic_still_creating'))
@@ -188,11 +193,11 @@ class TestKafkaClient(unittest.TestCase):
# Topic with partition metadata, but no leaders return True
self.assertTrue(client.has_metadata_for_topic('topic_noleaders'))
- @patch('kafka.client.KafkaConnection')
+ @patch('kafka.SimpleClient._get_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
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -200,16 +205,16 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- 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),
+ (NO_LEADER, 'topic_still_creating', []),
+ (UNKNOWN_TOPIC_OR_PARTITION, 'topic_doesnt_exist', []),
+ (NO_ERROR, 'topic_noleaders', [
+ (NO_LEADER, 0, -1, [], []),
+ (NO_LEADER, 1, -1, [], []),
]),
]
decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
with self.assertRaises(UnknownTopicOrPartitionError):
client.ensure_topic_exists('topic_doesnt_exist', timeout=1)
@@ -219,14 +224,13 @@ 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.SimpleClient._get_conn')
@patch('kafka.client.KafkaProtocol')
def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn):
"Get leader for partitions reload metadata if it is not available"
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -234,18 +238,18 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_no_partitions', NO_LEADER, [])
+ (NO_LEADER, 'topic_no_partitions', [])
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
# topic metadata is loaded but empty
self.assertDictEqual({}, client.topics_to_brokers)
topics = [
- TopicMetadata('topic_one_partition', NO_ERROR, [
- PartitionMetadata('topic_no_partition', 0, 0, [0, 1], [0, 1], NO_ERROR)
+ (NO_ERROR, 'topic_one_partition', [
+ (NO_ERROR, 0, 0, [0, 1], [0, 1])
])
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
@@ -256,14 +260,14 @@ class TestKafkaClient(unittest.TestCase):
self.assertEqual(brokers[0], leader)
self.assertDictEqual({
- TopicAndPartition('topic_one_partition', 0): brokers[0]},
+ TopicPartition('topic_one_partition', 0): brokers[0]},
client.topics_to_brokers)
- @patch('kafka.client.KafkaConnection')
+ @patch('kafka.SimpleClient._get_conn')
@patch('kafka.client.KafkaProtocol')
def test_get_leader_for_unassigned_partitions(self, protocol, conn):
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -271,26 +275,26 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata(b'topic_no_partitions', NO_LEADER, []),
- TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []),
+ (NO_LEADER, 'topic_no_partitions', []),
+ (UNKNOWN_TOPIC_OR_PARTITION, 'topic_unknown', []),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
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.KafkaConnection')
+ @patch('kafka.SimpleClient._get_conn')
@patch('kafka.client.KafkaProtocol')
def test_get_leader_exceptions_when_noleader(self, protocol, conn):
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -298,20 +302,18 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_noleader', NO_ERROR, [
- PartitionMetadata('topic_noleader', 0, -1, [], [],
- NO_LEADER),
- PartitionMetadata('topic_noleader', 1, -1, [], [],
- NO_LEADER),
+ (NO_ERROR, 'topic_noleader', [
+ (NO_LEADER, 0, -1, [], []),
+ (NO_LEADER, 1, -1, [], []),
]),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
self.assertDictEqual(
{
- TopicAndPartition('topic_noleader', 0): None,
- TopicAndPartition('topic_noleader', 1): None
+ TopicPartition('topic_noleader', 0): None,
+ TopicPartition('topic_noleader', 1): None
},
client.topics_to_brokers)
@@ -326,21 +328,19 @@ class TestKafkaClient(unittest.TestCase):
self.assertIsNone(client._get_leader_for_partition('topic_noleader', 2))
topics = [
- TopicMetadata('topic_noleader', NO_ERROR, [
- PartitionMetadata('topic_noleader', 0, 0, [0, 1], [0, 1], NO_ERROR),
- PartitionMetadata('topic_noleader', 1, 1, [1, 0], [1, 0], NO_ERROR)
+ (NO_ERROR, 'topic_noleader', [
+ (NO_ERROR, 0, 0, [0, 1], [0, 1]),
+ (NO_ERROR, 1, 1, [1, 0], [1, 0])
]),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
self.assertEqual(brokers[0], client._get_leader_for_partition('topic_noleader', 0))
self.assertEqual(brokers[1], client._get_leader_for_partition('topic_noleader', 1))
- @patch('kafka.client.KafkaConnection')
+ @patch.object(SimpleClient, '_get_conn')
@patch('kafka.client.KafkaProtocol')
def test_send_produce_request_raises_when_noleader(self, protocol, conn):
- "Send producer request raises LeaderNotAvailableError if leader is not available"
-
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -348,29 +348,27 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_noleader', NO_ERROR, [
- PartitionMetadata('topic_noleader', 0, -1, [], [],
- NO_LEADER),
- PartitionMetadata('topic_noleader', 1, -1, [], [],
- NO_LEADER),
+ (NO_ERROR, 'topic_noleader', [
+ (NO_LEADER, 0, -1, [], []),
+ (NO_LEADER, 1, -1, [], []),
]),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
- requests = [ProduceRequest(
+ requests = [ProduceRequestPayload(
"topic_noleader", 0,
[create_message("a"), create_message("b")])]
with self.assertRaises(LeaderNotAvailableError):
client.send_produce_request(requests)
- @patch('kafka.client.KafkaConnection')
+ @patch('kafka.SimpleClient._get_conn')
@patch('kafka.client.KafkaProtocol')
def test_send_produce_request_raises_when_topic_unknown(self, protocol, conn):
- conn.recv.return_value = 'response' # anything but None
+ mock_conn(conn)
brokers = [
BrokerMetadata(0, 'broker_1', 4567),
@@ -378,13 +376,13 @@ class TestKafkaClient(unittest.TestCase):
]
topics = [
- TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []),
+ (UNKNOWN_TOPIC_OR_PARTITION, 'topic_doesnt_exist', []),
]
protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics)
- client = KafkaClient(hosts=['broker_1:4567'])
+ client = SimpleClient(hosts=['broker_1:4567'])
- requests = [ProduceRequest(
+ requests = [ProduceRequestPayload(
"topic_doesnt_exist", 0,
[create_message("a"), create_message("b")])]
@@ -405,9 +403,9 @@ class TestKafkaClient(unittest.TestCase):
self.assertGreaterEqual(t.interval, 1.0)
def test_correlation_rollover(self):
- with patch.object(KafkaClient, 'load_metadata_for_topics'):
+ with patch.object(SimpleClient, 'load_metadata_for_topics'):
big_num = 2**31 - 3
- client = KafkaClient(hosts=[], correlation_id=big_num)
+ client = SimpleClient(hosts=[], correlation_id=big_num)
self.assertEqual(big_num + 1, client._next_id())
self.assertEqual(big_num + 2, client._next_id())
self.assertEqual(0, client._next_id())