Merge pull request #109 from mrtheb/develop
TopicAndPartition fix when partition has no leader = -1
This commit is contained in:
		| @@ -8,7 +8,8 @@ from itertools import count | ||||
| from kafka.common import (ErrorMapping, TopicAndPartition, | ||||
|                           ConnectionError, FailedPayloadsError, | ||||
|                           BrokerResponseError, PartitionUnavailableError, | ||||
|                           KafkaUnavailableError, KafkaRequestError) | ||||
|                           LeaderUnavailableError, | ||||
|                           KafkaUnavailableError) | ||||
|  | ||||
| from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS | ||||
| from kafka.protocol import KafkaProtocol | ||||
| @@ -62,12 +63,22 @@ class KafkaClient(object): | ||||
|         return self._get_conn(broker.host, broker.port) | ||||
|  | ||||
|     def _get_leader_for_partition(self, topic, partition): | ||||
|         """ | ||||
|         Returns the leader for a partition or None if the partition exists | ||||
|         but has no leader. | ||||
|  | ||||
|         PartitionUnavailableError will be raised if the topic or partition | ||||
|         is not part of the metadata. | ||||
|         """ | ||||
|  | ||||
|         key = TopicAndPartition(topic, partition) | ||||
|         if key not in self.topics_to_brokers: | ||||
|         # reload metadata whether the partition is not available | ||||
|         # or has no leader (broker is None) | ||||
|         if self.topics_to_brokers.get(key) is None: | ||||
|             self.load_metadata_for_topics(topic) | ||||
|  | ||||
|         if key not in self.topics_to_brokers: | ||||
|             raise KafkaRequestError("Partition does not exist: %s" % str(key)) | ||||
|             raise PartitionUnavailableError("%s not available" % str(key)) | ||||
|  | ||||
|         return self.topics_to_brokers[key] | ||||
|  | ||||
| @@ -124,8 +135,11 @@ class KafkaClient(object): | ||||
|         for payload in payloads: | ||||
|             leader = self._get_leader_for_partition(payload.topic, | ||||
|                                                     payload.partition) | ||||
|             if leader == -1: | ||||
|                 raise PartitionUnavailableError("Leader is unassigned for %s-%s" % payload.topic, payload.partition) | ||||
|             if leader is None: | ||||
|                 raise LeaderUnavailableError( | ||||
|                     "Leader not available for topic %s partition %s" % | ||||
|                     (payload.topic, payload.partition)) | ||||
|  | ||||
|             payloads_by_broker[leader].append(payload) | ||||
|             original_keys.append((payload.topic, payload.partition)) | ||||
|  | ||||
| @@ -250,13 +264,18 @@ class KafkaClient(object): | ||||
|             self.reset_topic_metadata(topic) | ||||
|  | ||||
|             if not partitions: | ||||
|                 log.warning('No partitions for %s', topic) | ||||
|                 continue | ||||
|  | ||||
|             self.topic_partitions[topic] = [] | ||||
|             for partition, meta in partitions.items(): | ||||
|                 topic_part = TopicAndPartition(topic, partition) | ||||
|                 self.topics_to_brokers[topic_part] = brokers[meta.leader] | ||||
|                 self.topic_partitions[topic].append(partition) | ||||
|                 topic_part = TopicAndPartition(topic, partition) | ||||
|                 if meta.leader == -1: | ||||
|                     log.warning('No leader for topic %s partition %s', topic, partition) | ||||
|                     self.topics_to_brokers[topic_part] = None | ||||
|                 else: | ||||
|                     self.topics_to_brokers[topic_part] = brokers[meta.leader] | ||||
|  | ||||
|     def send_produce_request(self, payloads=[], acks=1, timeout=1000, | ||||
|                              fail_on_error=True, callback=None): | ||||
|   | ||||
| @@ -74,10 +74,6 @@ class KafkaError(RuntimeError): | ||||
|     pass | ||||
|  | ||||
|  | ||||
| class KafkaRequestError(KafkaError): | ||||
|     pass | ||||
|  | ||||
|  | ||||
| class KafkaUnavailableError(KafkaError): | ||||
|     pass | ||||
|  | ||||
| @@ -86,6 +82,10 @@ class BrokerResponseError(KafkaError): | ||||
|     pass | ||||
|  | ||||
|  | ||||
| class LeaderUnavailableError(KafkaError): | ||||
|     pass | ||||
|  | ||||
|  | ||||
| class PartitionUnavailableError(KafkaError): | ||||
|     pass | ||||
|  | ||||
|   | ||||
							
								
								
									
										2
									
								
								setup.py
									
									
									
									
									
								
							
							
						
						
									
										2
									
								
								setup.py
									
									
									
									
									
								
							| @@ -23,7 +23,7 @@ setup( | ||||
|     version="0.9.0", | ||||
|  | ||||
|     install_requires=["distribute"], | ||||
|     tests_require=["tox"], | ||||
|     tests_require=["tox", "mock"], | ||||
|     cmdclass={"test": Tox}, | ||||
|  | ||||
|     packages=["kafka"], | ||||
|   | ||||
| @@ -5,14 +5,14 @@ import unittest | ||||
|  | ||||
| from mock import MagicMock, patch | ||||
|  | ||||
|  | ||||
| from kafka import KafkaClient | ||||
| from kafka.common import ( | ||||
|     ProduceRequest, FetchRequest, Message, ChecksumError, | ||||
|     ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, | ||||
|     OffsetAndMessage, BrokerMetadata, PartitionMetadata | ||||
|     OffsetAndMessage, BrokerMetadata, PartitionMetadata, | ||||
|     TopicAndPartition, KafkaUnavailableError,  | ||||
|     LeaderUnavailableError, PartitionUnavailableError | ||||
| ) | ||||
| from kafka.common import KafkaUnavailableError | ||||
| from kafka.codec import ( | ||||
|     has_gzip, has_snappy, gzip_encode, gzip_decode, | ||||
|     snappy_encode, snappy_decode | ||||
| @@ -410,6 +410,7 @@ class TestProtocol(unittest.TestCase): | ||||
|     def test_decode_offset_response(self): | ||||
|         pass | ||||
|  | ||||
|  | ||||
|     @unittest.skip("Not Implemented") | ||||
|     def test_encode_offset_commit_request(self): | ||||
|         pass | ||||
| @@ -474,18 +475,17 @@ class TestKafkaClient(unittest.TestCase): | ||||
|             return mocked_conns[(host, port)] | ||||
|  | ||||
|         # patch to avoid making requests before we want it | ||||
|         with patch.object(KafkaClient, 'load_metadata_for_topics'), \ | ||||
|                 patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): | ||||
|         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 = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) | ||||
|                 self.assertRaises( | ||||
|                     KafkaUnavailableError, | ||||
|                     client._send_broker_unaware_request, | ||||
|                     1, 'fake request') | ||||
|  | ||||
|             self.assertRaises( | ||||
|                 KafkaUnavailableError, | ||||
|                 client._send_broker_unaware_request, | ||||
|                 1, 'fake request') | ||||
|  | ||||
|             for key, conn in mocked_conns.iteritems(): | ||||
|                 conn.send.assert_called_with(1, 'fake request') | ||||
|                 for key, conn in mocked_conns.iteritems(): | ||||
|                     conn.send.assert_called_with(1, 'fake request') | ||||
|  | ||||
|     def test_send_broker_unaware_request(self): | ||||
|         'Tests that call works when at least one of the host is available' | ||||
| @@ -504,16 +504,171 @@ class TestKafkaClient(unittest.TestCase): | ||||
|             return mocked_conns[(host, port)] | ||||
|  | ||||
|         # patch to avoid making requests before we want it | ||||
|         with patch.object(KafkaClient, 'load_metadata_for_topics'), \ | ||||
|                 patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): | ||||
|         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 = KafkaClient(hosts='kafka01:9092,kafka02:9092') | ||||
|                 resp = client._send_broker_unaware_request(1, 'fake request') | ||||
|  | ||||
|             resp = client._send_broker_unaware_request(1, 'fake request') | ||||
|                 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_with(1) | ||||
|     @patch('kafka.client.KafkaConnection') | ||||
|     @patch('kafka.client.KafkaProtocol') | ||||
|     def test_load_metadata(self, protocol, conn): | ||||
|         "Load metadata for all topics" | ||||
|  | ||||
|         conn.recv.return_value = 'response'  # anything but None | ||||
|  | ||||
|         brokers = {} | ||||
|         brokers[0] = BrokerMetadata(1, 'broker_1', 4567) | ||||
|         brokers[1] = BrokerMetadata(2, 'broker_2', 5678) | ||||
|  | ||||
|         topics = {} | ||||
|         topics['topic_1'] = { | ||||
|             0: PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2]) | ||||
|         } | ||||
|         topics['topic_noleader'] = { | ||||
|             0: PartitionMetadata('topic_noleader', 0, -1, [], []), | ||||
|             1: PartitionMetadata('topic_noleader', 1, -1, [], []) | ||||
|         } | ||||
|         topics['topic_no_partitions'] = {} | ||||
|         topics['topic_3'] = { | ||||
|             0: PartitionMetadata('topic_3', 0, 0, [0, 1], [0, 1]), | ||||
|             1: PartitionMetadata('topic_3', 1, 1, [1, 0], [1, 0]), | ||||
|             2: PartitionMetadata('topic_3', 2, 0, [0, 1], [0, 1]) | ||||
|         } | ||||
|         protocol.decode_metadata_response.return_value = (brokers, topics) | ||||
|  | ||||
|         # 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]}, | ||||
|             client.topics_to_brokers) | ||||
|  | ||||
|     @patch('kafka.client.KafkaConnection') | ||||
|     @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 | ||||
|  | ||||
|         brokers = {} | ||||
|         brokers[0] = BrokerMetadata(0, 'broker_1', 4567) | ||||
|         brokers[1] = BrokerMetadata(1, 'broker_2', 5678) | ||||
|  | ||||
|         topics = {'topic_no_partitions': {}} | ||||
|         protocol.decode_metadata_response.return_value = (brokers, topics) | ||||
|  | ||||
|         client = KafkaClient(hosts=['broker_1:4567']) | ||||
|  | ||||
|         # topic metadata is loaded but empty | ||||
|         self.assertDictEqual({}, client.topics_to_brokers) | ||||
|  | ||||
|         topics['topic_no_partitions'] = { | ||||
|             0: PartitionMetadata('topic_no_partitions', 0, 0, [0, 1], [0, 1]) | ||||
|         } | ||||
|         protocol.decode_metadata_response.return_value = (brokers, topics) | ||||
|  | ||||
|         # calling _get_leader_for_partition (from any broker aware request) | ||||
|         # will try loading metadata again for the same topic | ||||
|         leader = client._get_leader_for_partition('topic_no_partitions', 0) | ||||
|  | ||||
|         self.assertEqual(brokers[0], leader) | ||||
|         self.assertDictEqual({ | ||||
|             TopicAndPartition('topic_no_partitions', 0): brokers[0]}, | ||||
|             client.topics_to_brokers) | ||||
|  | ||||
|     @patch('kafka.client.KafkaConnection') | ||||
|     @patch('kafka.client.KafkaProtocol') | ||||
|     def test_get_leader_for_unassigned_partitions(self, protocol, conn): | ||||
|         "Get leader raises if no partitions is defined for a topic" | ||||
|  | ||||
|         conn.recv.return_value = 'response'  # anything but None | ||||
|  | ||||
|         brokers = {} | ||||
|         brokers[0] = BrokerMetadata(0, 'broker_1', 4567) | ||||
|         brokers[1] = BrokerMetadata(1, 'broker_2', 5678) | ||||
|  | ||||
|         topics = {'topic_no_partitions': {}} | ||||
|         protocol.decode_metadata_response.return_value = (brokers, topics) | ||||
|  | ||||
|         client = KafkaClient(hosts=['broker_1:4567']) | ||||
|  | ||||
|         self.assertDictEqual({}, client.topics_to_brokers) | ||||
|         self.assertRaises( | ||||
|             PartitionUnavailableError, | ||||
|             client._get_leader_for_partition, | ||||
|             'topic_no_partitions', 0) | ||||
|  | ||||
|     @patch('kafka.client.KafkaConnection') | ||||
|     @patch('kafka.client.KafkaProtocol') | ||||
|     def test_get_leader_returns_none_when_noleader(self, protocol, conn): | ||||
|         "Getting leader for partitions returns None when the partiion has no leader" | ||||
|  | ||||
|         conn.recv.return_value = 'response'  # anything but None | ||||
|  | ||||
|         brokers = {} | ||||
|         brokers[0] = BrokerMetadata(0, 'broker_1', 4567) | ||||
|         brokers[1] = BrokerMetadata(1, 'broker_2', 5678) | ||||
|  | ||||
|         topics = {} | ||||
|         topics['topic_noleader'] = { | ||||
|             0: PartitionMetadata('topic_noleader', 0, -1, [], []), | ||||
|             1: PartitionMetadata('topic_noleader', 1, -1, [], []) | ||||
|         } | ||||
|         protocol.decode_metadata_response.return_value = (brokers, topics) | ||||
|  | ||||
|         client = KafkaClient(hosts=['broker_1:4567']) | ||||
|         self.assertDictEqual( | ||||
|             { | ||||
|                 TopicAndPartition('topic_noleader', 0): None, | ||||
|                 TopicAndPartition('topic_noleader', 1): None | ||||
|             }, | ||||
|             client.topics_to_brokers) | ||||
|         self.assertIsNone(client._get_leader_for_partition('topic_noleader', 0)) | ||||
|         self.assertIsNone(client._get_leader_for_partition('topic_noleader', 1)) | ||||
|  | ||||
|         topics['topic_noleader'] = { | ||||
|             0: PartitionMetadata('topic_noleader', 0, 0, [0, 1], [0, 1]), | ||||
|             1: PartitionMetadata('topic_noleader', 1, 1, [1, 0], [1, 0]) | ||||
|         } | ||||
|         protocol.decode_metadata_response.return_value = (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('kafka.client.KafkaProtocol') | ||||
|     def test_send_produce_request_raises_when_noleader(self, protocol, conn): | ||||
|         "Send producer request raises LeaderUnavailableError if leader is not available" | ||||
|  | ||||
|         conn.recv.return_value = 'response'  # anything but None | ||||
|  | ||||
|         brokers = {} | ||||
|         brokers[0] = BrokerMetadata(0, 'broker_1', 4567) | ||||
|         brokers[1] = BrokerMetadata(1, 'broker_2', 5678) | ||||
|  | ||||
|         topics = {} | ||||
|         topics['topic_noleader'] = { | ||||
|             0: PartitionMetadata('topic_noleader', 0, -1, [], []), | ||||
|             1: PartitionMetadata('topic_noleader', 1, -1, [], []) | ||||
|         } | ||||
|         protocol.decode_metadata_response.return_value = (brokers, topics) | ||||
|  | ||||
|         client = KafkaClient(hosts=['broker_1:4567']) | ||||
|  | ||||
|         requests = [ProduceRequest( | ||||
|             "topic_noleader", 0, | ||||
|             [create_message("a"), create_message("b")])] | ||||
|  | ||||
|         self.assertRaises( | ||||
|             LeaderUnavailableError, | ||||
|             client.send_produce_request, requests) | ||||
|  | ||||
| if __name__ == '__main__': | ||||
|     unittest.main() | ||||
|   | ||||
		Reference in New Issue
	
	Block a user
	 Dana Powers
					Dana Powers