summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--kafka/client.py33
-rw-r--r--kafka/common.py6
-rw-r--r--setup.py2
-rw-r--r--test/test_unit.py193
-rw-r--r--tox.ini4
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
diff --git a/setup.py b/setup.py
index 4176135..009e14f 100644
--- a/setup.py
+++ b/setup.py
@@ -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()
diff --git a/tox.ini b/tox.ini
index f41911c..0077c4d 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,7 +1,9 @@
[tox]
envlist = py26, py27
[testenv]
-deps = pytest
+deps =
+ pytest
+ mock
commands = py.test --basetemp={envtmpdir} []
setenv =
PROJECT_ROOT = {toxinidir}