Handle cases for partition with leader=-1 (not defined)

This commit is contained in:
Marc Labbe
2014-01-31 20:06:30 -05:00
parent 9e2778efbb
commit 8bcf0f0940
3 changed files with 82 additions and 63 deletions

View File

@@ -8,7 +8,7 @@ from itertools import count
from kafka.common import (ErrorMapping, TopicAndPartition,
ConnectionError, FailedPayloadsError,
BrokerResponseError, PartitionUnavailableError,
KafkaUnavailableError, KafkaRequestError)
KafkaUnavailableError)
from kafka.conn import KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS
from kafka.protocol import KafkaProtocol
@@ -53,11 +53,13 @@ class KafkaClient(object):
def _get_leader_for_partition(self, topic, partition):
key = TopicAndPartition(topic, partition)
if key not in self.topics_to_brokers:
# reload metadata whether the partition is not available
# or has not 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("No leader for %s" % str(key))
return self.topics_to_brokers[key]
@@ -239,14 +241,18 @@ class KafkaClient(object):
self.reset_topic_metadata(topic)
if not partitions:
log.info('No partitions for %s', topic)
continue
self.topic_partitions[topic] = []
for partition, meta in partitions.items():
if meta.leader != -1:
topic_part = TopicAndPartition(topic, partition)
self.topic_partitions[topic].append(partition)
topic_part = TopicAndPartition(topic, partition)
if meta.leader == -1:
log.info('No leader for topic %s partition %d', topic, partition)
self.topics_to_brokers[topic_part] = None
else:
self.topics_to_brokers[topic_part] = brokers[meta.leader]
self.topic_partitions[topic].append(partition)
def send_produce_request(self, payloads=[], acks=1, timeout=1000,
fail_on_error=True, callback=None):

View File

@@ -74,10 +74,6 @@ class KafkaError(RuntimeError):
pass
class KafkaRequestError(KafkaError):
pass
class KafkaUnavailableError(KafkaError):
pass

View File

@@ -7,7 +7,7 @@ from kafka.common import (
ProduceRequest, FetchRequest, Message, ChecksumError,
ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse,
OffsetAndMessage, BrokerMetadata, PartitionMetadata,
TopicAndPartition
TopicAndPartition, PartitionUnavailableError
)
from kafka.codec import (
has_gzip, has_snappy, gzip_encode, gzip_decode,
@@ -55,7 +55,6 @@ class TestPackage(unittest.TestCase):
from kafka import KafkaClient as KafkaClient2
self.assertEquals(KafkaClient2.__name__, "KafkaClient")
from kafka.codec import snappy_encode
self.assertEquals(snappy_encode.__name__, "snappy_encode")
@@ -391,7 +390,8 @@ class TestClient(unittest.TestCase):
@patch('kafka.client.KafkaConnection')
@patch('kafka.client.KafkaProtocol')
def test_client_load_metadata(self, protocol, conn):
def test_load_metadata(self, protocol, conn):
"Load metadata for all topics"
conn.recv.return_value = 'response' # anything but None
@@ -403,22 +403,33 @@ class TestClient(unittest.TestCase):
topics['topic_1'] = {
0: PartitionMetadata('topic_1', 0, 1, [1, 2], [1, 2])
}
topics['topic_2'] = {
0: PartitionMetadata('topic_2', 0, 0, [0, 1], [0, 1]),
1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0])
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(host='broker_1', port=4567)
self.assertItemsEqual({
TopicAndPartition('topic_1', 0): brokers[0],
TopicAndPartition('topic_2', 0): brokers[0],
TopicAndPartition('topic_2', 1): 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_client_load_metadata_unassigned_partitions(self, protocol, conn):
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
@@ -426,35 +437,54 @@ class TestClient(unittest.TestCase):
brokers[0] = BrokerMetadata(0, 'broker_1', 4567)
brokers[1] = BrokerMetadata(1, 'broker_2', 5678)
topics = {}
topics['topic_1'] = {
0: PartitionMetadata('topic_1', 0, -1, [], [])
topics = {'topic_no_partitions': {}}
protocol.decode_metadata_response.return_value = (brokers, topics)
client = KafkaClient(host='broker_1', port=4567)
# topic metadata is loaded but empty
self.assertItemsEqual({}, 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.assertItemsEqual({
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(host='broker_1', port=4567)
self.assertItemsEqual({}, client.topics_to_brokers)
self.assertRaises(
Exception,
PartitionUnavailableError,
client._get_leader_for_partition,
'topic_1', 0)
# calling _get_leader_for_partition (from any broker aware request)
# will try loading metadata again for the same topic
topics['topic_1'] = {
0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1])
}
leader = client._get_leader_for_partition('topic_1', 0)
self.assertEqual(brokers[0], leader)
self.assertItemsEqual({
TopicAndPartition('topic_1', 0): brokers[0]},
client.topics_to_brokers)
'topic_no_partitions', 0)
@patch('kafka.client.KafkaConnection')
@patch('kafka.client.KafkaProtocol')
def test_client_load_metadata_noleader_partitions(self, protocol, conn):
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
@@ -463,42 +493,29 @@ class TestClient(unittest.TestCase):
brokers[1] = BrokerMetadata(1, 'broker_2', 5678)
topics = {}
topics['topic_1'] = {
0: PartitionMetadata('topic_1', 0, -1, [], [])
}
topics['topic_2'] = {
0: PartitionMetadata('topic_2', 0, 0, [0, 1], []),
1: PartitionMetadata('topic_2', 1, 1, [1, 0], [1, 0])
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(host='broker_1', port=4567)
self.assertItemsEqual(
{
TopicAndPartition('topic_2', 0): brokers[0],
TopicAndPartition('topic_2', 1): brokers[1]
TopicAndPartition('topic_noleader', 0): None,
TopicAndPartition('topic_noleader', 1): None
},
client.topics_to_brokers)
self.assertRaises(
Exception,
client._get_leader_for_partition,
'topic_1', 0)
self.assertIsNone(client._get_leader_for_partition('topic_noleader', 0))
self.assertIsNone(client._get_leader_for_partition('topic_noleader', 1))
# calling _get_leader_for_partition (from any broker aware request)
# will try loading metadata again for the same topic
topics['topic_1'] = {
0: PartitionMetadata('topic_1', 0, 0, [0, 1], [0, 1])
topics['topic_noleader'] = {
0: PartitionMetadata('topic_noleader', 0, 0, [0, 1], [0, 1]),
1: PartitionMetadata('topic_noleader', 1, 1, [1, 0], [1, 0])
}
leader = client._get_leader_for_partition('topic_1', 0)
self.assertEqual(brokers[0], leader)
self.assertItemsEqual(
{
TopicAndPartition('topic_1', 0): brokers[0],
TopicAndPartition('topic_2', 0): brokers[0],
TopicAndPartition('topic_2', 1): brokers[1]
},
client.topics_to_brokers)
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))
if __name__ == '__main__':
unittest.main()