summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--kafka/client.py76
-rw-r--r--kafka/consumer.py4
-rw-r--r--kafka/producer.py19
-rw-r--r--test/test_producer_integration.py3
4 files changed, 69 insertions, 33 deletions
diff --git a/kafka/client.py b/kafka/client.py
index 2eab1e3..46cd7ce 100644
--- a/kafka/client.py
+++ b/kafka/client.py
@@ -36,8 +36,9 @@ class KafkaClient(object):
# create connections only when we need them
self.conns = {}
self.brokers = {} # broker_id -> BrokerMetadata
- self.topics_to_brokers = {} # topic_id -> broker_id
- self.topic_partitions = {} # topic_id -> [0, 1, 2, ...]
+ self.topics_to_brokers = {} # TopicAndPartition -> BrokerMetadata
+ self.topic_partitions = {} # topic -> partition -> PartitionMetadata
+
self.load_metadata_for_topics() # bootstrap with all metadata
@@ -235,50 +236,85 @@ class KafkaClient(object):
self.topic_partitions.clear()
def has_metadata_for_topic(self, topic):
- return topic in self.topic_partitions
+ return (
+ topic in self.topic_partitions
+ and len(self.topic_partitions[topic]) > 0
+ )
+
+ def get_partition_ids_for_topic(self, topic):
+ if topic not in self.topic_partitions:
+ return None
+
+ return self.topic_partitions[topic].keys()
def ensure_topic_exists(self, topic, timeout = 30):
start_time = time.time()
- self.load_metadata_for_topics(topic)
while not self.has_metadata_for_topic(topic):
if time.time() > start_time + timeout:
raise KafkaTimeoutError("Unable to create topic {0}".format(topic))
- self.load_metadata_for_topics(topic)
+ try:
+ self.load_metadata_for_topics(topic)
+ except LeaderNotAvailableError:
+ pass
time.sleep(.5)
def load_metadata_for_topics(self, *topics):
"""
Discover brokers and metadata for a set of topics. This function is called
lazily whenever metadata is unavailable.
- """
+ If broker does not auto-create topics, expect
+ UnknownTopicOrPartitionError for new topics
+
+ If broker auto-creates topics, expect
+ LeaderNotAvailableError for new topics
+ until partitions have been initialized.
+ Retry.
+ """
resp = self.send_metadata_request(topics)
- brokers = dict([(broker.nodeId, broker) for broker in resp.brokers])
- topics = dict([(t.topic, dict([(p.partition, p) for p in t.partitions]) ) for t in resp.topics])
+ log.debug("Broker metadata: %s", resp.brokers)
+ log.debug("Topic metadata: %s", resp.topics)
- log.debug("Broker metadata: %s", brokers)
- log.debug("Topic metadata: %s", topics)
+ self.brokers = dict([(broker.nodeId, broker)
+ for broker in resp.brokers])
- self.brokers = brokers
+ for topic_metadata in resp.topics:
+ topic = topic_metadata.topic
+ partitions = topic_metadata.partitions
- for topic, partitions in topics.items():
self.reset_topic_metadata(topic)
- if not partitions:
- log.warning('No partitions for %s', topic)
- continue
+ # Errors expected for new topics
+ # 3 if topic doesn't exist, or 5 if server is auto-creating
+ kafka.common.check_error(topic_metadata)
+
+ self.topic_partitions[topic] = {}
+ for partition_metadata in partitions:
+ partition = partition_metadata.partition
+ leader = partition_metadata.leader
+
+ self.topic_partitions[topic][partition] = partition_metadata
- self.topic_partitions[topic] = []
- for partition, meta in partitions.items():
- self.topic_partitions[topic].append(partition)
+ # Populate topics_to_brokers dict
topic_part = TopicAndPartition(topic, partition)
- if meta.leader == -1:
+
+ # If No Leader, topics_to_brokers topic_partition -> None
+ if leader == -1:
log.warning('No leader for topic %s partition %s', topic, partition)
self.topics_to_brokers[topic_part] = None
+
+ # If Known Broker, topic_partition -> BrokerMetadata
+ elif leader in self.brokers:
+ self.topics_to_brokers[topic_part] = self.brokers[leader]
+
+ # If Unknown Broker, fake BrokerMetadata so we dont lose the id
+ # (not sure how this could happen. server could be in bad state)
else:
- self.topics_to_brokers[topic_part] = brokers[meta.leader]
+ self.topics_to_brokers[topic_part] = BrokerMetadata(
+ leader, None, None
+ )
def send_metadata_request(self, payloads=[], fail_on_error=True,
callback=None):
diff --git a/kafka/consumer.py b/kafka/consumer.py
index 0935dd2..51f446c 100644
--- a/kafka/consumer.py
+++ b/kafka/consumer.py
@@ -82,7 +82,7 @@ class Consumer(object):
self.offsets = {}
if not partitions:
- partitions = self.client.topic_partitions[topic]
+ partitions = self.client.get_partition_ids_for_topic(topic)
else:
assert all(isinstance(x, numbers.Integral) for x in partitions)
@@ -108,7 +108,7 @@ class Consumer(object):
def fetch_last_known_offsets(self, partitions=None):
if not partitions:
- partitions = self.client.topic_partitions[self.topic]
+ partitions = self.client.get_partition_ids_for_topic(self.topic)
def get_or_init_offset_callback(resp):
try:
diff --git a/kafka/producer.py b/kafka/producer.py
index 8a6bff0..b60f13d 100644
--- a/kafka/producer.py
+++ b/kafka/producer.py
@@ -241,16 +241,14 @@ class SimpleProducer(Producer):
def _next_partition(self, topic):
if topic not in self.partition_cycles:
- if topic not in self.client.topic_partitions:
+ if not self.client.has_metadata_for_topic(topic):
self.client.load_metadata_for_topics(topic)
- try:
- self.partition_cycles[topic] = cycle(self.client.topic_partitions[topic])
- except KeyError:
- raise UnknownTopicOrPartitionError(topic)
+
+ self.partition_cycles[topic] = cycle(self.client.get_partition_ids_for_topic(topic))
# Randomize the initial partition that is returned
if self.random_start:
- num_partitions = len(self.client.topic_partitions[topic])
+ num_partitions = len(self.client.get_partition_ids_for_topic(topic))
for _ in xrange(random.randint(0, num_partitions-1)):
self.partition_cycles[topic].next()
@@ -299,12 +297,13 @@ class KeyedProducer(Producer):
def _next_partition(self, topic, key):
if topic not in self.partitioners:
- if topic not in self.client.topic_partitions:
+ if not self.client.has_metadata_for_topic(topic):
self.client.load_metadata_for_topics(topic)
- self.partitioners[topic] = \
- self.partitioner_class(self.client.topic_partitions[topic])
+
+ self.partitioners[topic] = self.partitioner_class(self.client.get_partition_ids_for_topic(topic))
+
partitioner = self.partitioners[topic]
- return partitioner.partition(key, self.client.topic_partitions[topic])
+ return partitioner.partition(key, self.client.get_partition_ids_for_topic(topic))
def send(self, topic, key, msg):
partition = self._next_partition(topic, key)
diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py
index 7d3a180..b4e076a 100644
--- a/test/test_producer_integration.py
+++ b/test/test_producer_integration.py
@@ -148,7 +148,8 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase):
producer = SimpleProducer(self.client)
# At first it doesn't exist
- with self.assertRaises(UnknownTopicOrPartitionError):
+ with self.assertRaises((UnknownTopicOrPartitionError,
+ LeaderNotAvailableError)):
resp = producer.send_messages(new_topic, self.msg("one"))
@kafka_versions("all")