diff options
author | Dana Powers <dana.powers@rd.io> | 2016-01-07 18:51:14 -0800 |
---|---|---|
committer | Dana Powers <dana.powers@rd.io> | 2016-01-07 18:51:14 -0800 |
commit | 828377377da43749af0d27ee256ef31bf714cf17 (patch) | |
tree | fbad4d4381fc4d1ea2be7ce2009214d18fbeb674 /test/test_client.py | |
parent | 71e7568fcb8132899f366b37c32645fd5a40dc4b (diff) | |
parent | 9a8af1499ca425366d934487469d9977fae7fe5f (diff) | |
download | kafka-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.py | 254 |
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()) |