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