diff options
author | Dana Powers <dana.powers@gmail.com> | 2014-03-21 22:50:53 -0700 |
---|---|---|
committer | Dana Powers <dana.powers@gmail.com> | 2014-03-21 22:50:53 -0700 |
commit | e937e3f971f5958c8da6249b08288aafd5ed5bcd (patch) | |
tree | 0d495dfaee84d551ec97ca6ddac13d635fca75d1 | |
parent | 9599215bf28b65a29908b8644dcaa6f3614a425d (diff) | |
parent | 51246fb31859e9cee03b1e1359ad871274dca87e (diff) | |
download | kafka-python-e937e3f971f5958c8da6249b08288aafd5ed5bcd.tar.gz |
Merge pull request #109 from mrtheb/develop
TopicAndPartition fix when partition has no leader = -1
-rw-r--r-- | kafka/client.py | 33 | ||||
-rw-r--r-- | kafka/common.py | 6 | ||||
-rw-r--r-- | setup.py | 2 | ||||
-rw-r--r-- | test/test_unit.py | 193 | ||||
-rw-r--r-- | tox.ini | 4 |
5 files changed, 207 insertions, 31 deletions
diff --git a/kafka/client.py b/kafka/client.py index a683fe0..ab0eb8d 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -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): diff --git a/kafka/common.py b/kafka/common.py index c0a1a6a..b4fe5c7 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -74,15 +74,15 @@ class KafkaError(RuntimeError): pass -class KafkaRequestError(KafkaError): +class KafkaUnavailableError(KafkaError): pass -class KafkaUnavailableError(KafkaError): +class BrokerResponseError(KafkaError): pass -class BrokerResponseError(KafkaError): +class LeaderUnavailableError(KafkaError): pass @@ -23,7 +23,7 @@ setup( version="0.9.0", install_requires=["distribute"], - tests_require=["tox"], + tests_require=["tox", "mock"], cmdclass={"test": Tox}, packages=["kafka"], diff --git a/test/test_unit.py b/test/test_unit.py index 081acc7..8c0dd00 100644 --- a/test/test_unit.py +++ b/test/test_unit.py @@ -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): - - client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) + 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']) - 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') + + resp = client._send_broker_unaware_request(1, 'fake request') + + 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 - client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + 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) - resp = client._send_broker_unaware_request(1, 'fake request') + client = KafkaClient(hosts=['broker_1:4567']) - self.assertEqual('valid response', resp) - mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) + requests = [ProduceRequest( + "topic_noleader", 0, + [create_message("a"), create_message("b")])] + self.assertRaises( + LeaderUnavailableError, + client.send_produce_request, requests) if __name__ == '__main__': unittest.main() @@ -1,7 +1,9 @@ [tox] envlist = py26, py27 [testenv] -deps = pytest +deps = + pytest + mock commands = py.test --basetemp={envtmpdir} [] setenv = PROJECT_ROOT = {toxinidir} |