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,
 | 
					from kafka.common import (ErrorMapping, TopicAndPartition,
 | 
				
			||||||
                          ConnectionError, FailedPayloadsError,
 | 
					                          ConnectionError, FailedPayloadsError,
 | 
				
			||||||
                          BrokerResponseError, PartitionUnavailableError,
 | 
					                          BrokerResponseError, PartitionUnavailableError,
 | 
				
			||||||
                          KafkaUnavailableError, KafkaRequestError)
 | 
					                          LeaderUnavailableError,
 | 
				
			||||||
 | 
					                          KafkaUnavailableError)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS
 | 
					from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS
 | 
				
			||||||
from kafka.protocol import KafkaProtocol
 | 
					from kafka.protocol import KafkaProtocol
 | 
				
			||||||
@@ -62,12 +63,22 @@ class KafkaClient(object):
 | 
				
			|||||||
        return self._get_conn(broker.host, broker.port)
 | 
					        return self._get_conn(broker.host, broker.port)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    def _get_leader_for_partition(self, topic, partition):
 | 
					    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)
 | 
					        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)
 | 
					            self.load_metadata_for_topics(topic)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        if key not in self.topics_to_brokers:
 | 
					        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]
 | 
					        return self.topics_to_brokers[key]
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -124,8 +135,11 @@ class KafkaClient(object):
 | 
				
			|||||||
        for payload in payloads:
 | 
					        for payload in payloads:
 | 
				
			||||||
            leader = self._get_leader_for_partition(payload.topic,
 | 
					            leader = self._get_leader_for_partition(payload.topic,
 | 
				
			||||||
                                                    payload.partition)
 | 
					                                                    payload.partition)
 | 
				
			||||||
            if leader == -1:
 | 
					            if leader is None:
 | 
				
			||||||
                raise PartitionUnavailableError("Leader is unassigned for %s-%s" % payload.topic, payload.partition)
 | 
					                raise LeaderUnavailableError(
 | 
				
			||||||
 | 
					                    "Leader not available for topic %s partition %s" %
 | 
				
			||||||
 | 
					                    (payload.topic, payload.partition))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            payloads_by_broker[leader].append(payload)
 | 
					            payloads_by_broker[leader].append(payload)
 | 
				
			||||||
            original_keys.append((payload.topic, payload.partition))
 | 
					            original_keys.append((payload.topic, payload.partition))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -250,13 +264,18 @@ class KafkaClient(object):
 | 
				
			|||||||
            self.reset_topic_metadata(topic)
 | 
					            self.reset_topic_metadata(topic)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            if not partitions:
 | 
					            if not partitions:
 | 
				
			||||||
 | 
					                log.warning('No partitions for %s', topic)
 | 
				
			||||||
                continue
 | 
					                continue
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            self.topic_partitions[topic] = []
 | 
					            self.topic_partitions[topic] = []
 | 
				
			||||||
            for partition, meta in partitions.items():
 | 
					            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)
 | 
					                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,
 | 
					    def send_produce_request(self, payloads=[], acks=1, timeout=1000,
 | 
				
			||||||
                             fail_on_error=True, callback=None):
 | 
					                             fail_on_error=True, callback=None):
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -74,10 +74,6 @@ class KafkaError(RuntimeError):
 | 
				
			|||||||
    pass
 | 
					    pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
class KafkaRequestError(KafkaError):
 | 
					 | 
				
			||||||
    pass
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
class KafkaUnavailableError(KafkaError):
 | 
					class KafkaUnavailableError(KafkaError):
 | 
				
			||||||
    pass
 | 
					    pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -86,6 +82,10 @@ class BrokerResponseError(KafkaError):
 | 
				
			|||||||
    pass
 | 
					    pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					class LeaderUnavailableError(KafkaError):
 | 
				
			||||||
 | 
					    pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
class PartitionUnavailableError(KafkaError):
 | 
					class PartitionUnavailableError(KafkaError):
 | 
				
			||||||
    pass
 | 
					    pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 
 | 
				
			|||||||
							
								
								
									
										2
									
								
								setup.py
									
									
									
									
									
								
							
							
						
						
									
										2
									
								
								setup.py
									
									
									
									
									
								
							@@ -23,7 +23,7 @@ setup(
 | 
				
			|||||||
    version="0.9.0",
 | 
					    version="0.9.0",
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    install_requires=["distribute"],
 | 
					    install_requires=["distribute"],
 | 
				
			||||||
    tests_require=["tox"],
 | 
					    tests_require=["tox", "mock"],
 | 
				
			||||||
    cmdclass={"test": Tox},
 | 
					    cmdclass={"test": Tox},
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    packages=["kafka"],
 | 
					    packages=["kafka"],
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -5,14 +5,14 @@ import unittest
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
from mock import MagicMock, patch
 | 
					from mock import MagicMock, patch
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					 | 
				
			||||||
from kafka import KafkaClient
 | 
					from kafka import KafkaClient
 | 
				
			||||||
from kafka.common import (
 | 
					from kafka.common import (
 | 
				
			||||||
    ProduceRequest, FetchRequest, Message, ChecksumError,
 | 
					    ProduceRequest, FetchRequest, Message, ChecksumError,
 | 
				
			||||||
    ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse,
 | 
					    ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse,
 | 
				
			||||||
    OffsetAndMessage, BrokerMetadata, PartitionMetadata
 | 
					    OffsetAndMessage, BrokerMetadata, PartitionMetadata,
 | 
				
			||||||
 | 
					    TopicAndPartition, KafkaUnavailableError, 
 | 
				
			||||||
 | 
					    LeaderUnavailableError, PartitionUnavailableError
 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
from kafka.common import KafkaUnavailableError
 | 
					 | 
				
			||||||
from kafka.codec import (
 | 
					from kafka.codec import (
 | 
				
			||||||
    has_gzip, has_snappy, gzip_encode, gzip_decode,
 | 
					    has_gzip, has_snappy, gzip_encode, gzip_decode,
 | 
				
			||||||
    snappy_encode, snappy_decode
 | 
					    snappy_encode, snappy_decode
 | 
				
			||||||
@@ -410,6 +410,7 @@ class TestProtocol(unittest.TestCase):
 | 
				
			|||||||
    def test_decode_offset_response(self):
 | 
					    def test_decode_offset_response(self):
 | 
				
			||||||
        pass
 | 
					        pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    @unittest.skip("Not Implemented")
 | 
					    @unittest.skip("Not Implemented")
 | 
				
			||||||
    def test_encode_offset_commit_request(self):
 | 
					    def test_encode_offset_commit_request(self):
 | 
				
			||||||
        pass
 | 
					        pass
 | 
				
			||||||
@@ -474,18 +475,17 @@ class TestKafkaClient(unittest.TestCase):
 | 
				
			|||||||
            return mocked_conns[(host, port)]
 | 
					            return mocked_conns[(host, port)]
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        # patch to avoid making requests before we want it
 | 
					        # patch to avoid making requests before we want it
 | 
				
			||||||
        with patch.object(KafkaClient, 'load_metadata_for_topics'), \
 | 
					        with patch.object(KafkaClient, 'load_metadata_for_topics'):
 | 
				
			||||||
                patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn):
 | 
					            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(
 | 
					                for key, conn in mocked_conns.iteritems():
 | 
				
			||||||
                KafkaUnavailableError,
 | 
					                    conn.send.assert_called_with(1, 'fake request')
 | 
				
			||||||
                client._send_broker_unaware_request,
 | 
					 | 
				
			||||||
                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):
 | 
					    def test_send_broker_unaware_request(self):
 | 
				
			||||||
        'Tests that call works when at least one of the host is available'
 | 
					        '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)]
 | 
					            return mocked_conns[(host, port)]
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        # patch to avoid making requests before we want it
 | 
					        # patch to avoid making requests before we want it
 | 
				
			||||||
        with patch.object(KafkaClient, 'load_metadata_for_topics'), \
 | 
					        with patch.object(KafkaClient, 'load_metadata_for_topics'):
 | 
				
			||||||
                patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn):
 | 
					            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)
 | 
					    @patch('kafka.client.KafkaConnection')
 | 
				
			||||||
            mocked_conns[('kafka02', 9092)].recv.assert_called_with(1)
 | 
					    @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__':
 | 
					if __name__ == '__main__':
 | 
				
			||||||
    unittest.main()
 | 
					    unittest.main()
 | 
				
			||||||
 
 | 
				
			|||||||
							
								
								
									
										4
									
								
								tox.ini
									
									
									
									
									
								
							
							
						
						
									
										4
									
								
								tox.ini
									
									
									
									
									
								
							@@ -1,7 +1,9 @@
 | 
				
			|||||||
[tox]
 | 
					[tox]
 | 
				
			||||||
envlist = py26, py27
 | 
					envlist = py26, py27
 | 
				
			||||||
[testenv]
 | 
					[testenv]
 | 
				
			||||||
deps = pytest
 | 
					deps =
 | 
				
			||||||
 | 
					    pytest
 | 
				
			||||||
 | 
					    mock
 | 
				
			||||||
commands = py.test --basetemp={envtmpdir} []
 | 
					commands = py.test --basetemp={envtmpdir} []
 | 
				
			||||||
setenv =
 | 
					setenv =
 | 
				
			||||||
    PROJECT_ROOT = {toxinidir}
 | 
					    PROJECT_ROOT = {toxinidir}
 | 
				
			||||||
 
 | 
				
			|||||||
		Reference in New Issue
	
	Block a user