Handle cases for partition with leader=-1 (not defined)
This commit is contained in:
@@ -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):
|
||||
|
||||
@@ -74,10 +74,6 @@ class KafkaError(RuntimeError):
|
||||
pass
|
||||
|
||||
|
||||
class KafkaRequestError(KafkaError):
|
||||
pass
|
||||
|
||||
|
||||
class KafkaUnavailableError(KafkaError):
|
||||
pass
|
||||
|
||||
|
||||
@@ -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()
|
||||
|
||||
Reference in New Issue
Block a user