Refactor internal metadata dicts in KafkaClient
- use helper methods not direct access - add get_partition_ids_for_topic - check for topic and partition errors during load_metadata_for_topics - raise LeaderNotAvailableError when topic is being auto-created or UnknownTopicOrPartitionError if auto-creation off
This commit is contained in:
@@ -36,8 +36,9 @@ class KafkaClient(object):
|
|||||||
# create connections only when we need them
|
# create connections only when we need them
|
||||||
self.conns = {}
|
self.conns = {}
|
||||||
self.brokers = {} # broker_id -> BrokerMetadata
|
self.brokers = {} # broker_id -> BrokerMetadata
|
||||||
self.topics_to_brokers = {} # topic_id -> broker_id
|
self.topics_to_brokers = {} # TopicAndPartition -> BrokerMetadata
|
||||||
self.topic_partitions = {} # topic_id -> [0, 1, 2, ...]
|
self.topic_partitions = {} # topic -> partition -> PartitionMetadata
|
||||||
|
|
||||||
self.load_metadata_for_topics() # bootstrap with all metadata
|
self.load_metadata_for_topics() # bootstrap with all metadata
|
||||||
|
|
||||||
|
|
||||||
@@ -235,50 +236,85 @@ class KafkaClient(object):
|
|||||||
self.topic_partitions.clear()
|
self.topic_partitions.clear()
|
||||||
|
|
||||||
def has_metadata_for_topic(self, topic):
|
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):
|
def ensure_topic_exists(self, topic, timeout = 30):
|
||||||
start_time = time.time()
|
start_time = time.time()
|
||||||
|
|
||||||
self.load_metadata_for_topics(topic)
|
|
||||||
while not self.has_metadata_for_topic(topic):
|
while not self.has_metadata_for_topic(topic):
|
||||||
if time.time() > start_time + timeout:
|
if time.time() > start_time + timeout:
|
||||||
raise KafkaTimeoutError("Unable to create topic {0}".format(topic))
|
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)
|
time.sleep(.5)
|
||||||
|
|
||||||
def load_metadata_for_topics(self, *topics):
|
def load_metadata_for_topics(self, *topics):
|
||||||
"""
|
"""
|
||||||
Discover brokers and metadata for a set of topics. This function is called
|
Discover brokers and metadata for a set of topics. This function is called
|
||||||
lazily whenever metadata is unavailable.
|
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)
|
resp = self.send_metadata_request(topics)
|
||||||
|
|
||||||
brokers = dict([(broker.nodeId, broker) for broker in resp.brokers])
|
log.debug("Broker metadata: %s", resp.brokers)
|
||||||
topics = dict([(t.topic, dict([(p.partition, p) for p in t.partitions]) ) for t in resp.topics])
|
log.debug("Topic metadata: %s", resp.topics)
|
||||||
|
|
||||||
log.debug("Broker metadata: %s", brokers)
|
self.brokers = dict([(broker.nodeId, broker)
|
||||||
log.debug("Topic metadata: %s", topics)
|
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)
|
self.reset_topic_metadata(topic)
|
||||||
|
|
||||||
if not partitions:
|
# Errors expected for new topics
|
||||||
log.warning('No partitions for %s', topic)
|
# 3 if topic doesn't exist, or 5 if server is auto-creating
|
||||||
continue
|
kafka.common.check_error(topic_metadata)
|
||||||
|
|
||||||
self.topic_partitions[topic] = []
|
self.topic_partitions[topic] = {}
|
||||||
for partition, meta in partitions.items():
|
for partition_metadata in partitions:
|
||||||
self.topic_partitions[topic].append(partition)
|
partition = partition_metadata.partition
|
||||||
|
leader = partition_metadata.leader
|
||||||
|
|
||||||
|
self.topic_partitions[topic][partition] = partition_metadata
|
||||||
|
|
||||||
|
# Populate topics_to_brokers dict
|
||||||
topic_part = TopicAndPartition(topic, partition)
|
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)
|
log.warning('No leader for topic %s partition %s', topic, partition)
|
||||||
self.topics_to_brokers[topic_part] = None
|
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:
|
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,
|
def send_metadata_request(self, payloads=[], fail_on_error=True,
|
||||||
callback=None):
|
callback=None):
|
||||||
|
|||||||
@@ -82,7 +82,7 @@ class Consumer(object):
|
|||||||
self.offsets = {}
|
self.offsets = {}
|
||||||
|
|
||||||
if not partitions:
|
if not partitions:
|
||||||
partitions = self.client.topic_partitions[topic]
|
partitions = self.client.get_partition_ids_for_topic(topic)
|
||||||
else:
|
else:
|
||||||
assert all(isinstance(x, numbers.Integral) for x in partitions)
|
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):
|
def fetch_last_known_offsets(self, partitions=None):
|
||||||
if not partitions:
|
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):
|
def get_or_init_offset_callback(resp):
|
||||||
try:
|
try:
|
||||||
|
|||||||
@@ -241,16 +241,14 @@ class SimpleProducer(Producer):
|
|||||||
|
|
||||||
def _next_partition(self, topic):
|
def _next_partition(self, topic):
|
||||||
if topic not in self.partition_cycles:
|
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)
|
self.client.load_metadata_for_topics(topic)
|
||||||
try:
|
|
||||||
self.partition_cycles[topic] = cycle(self.client.topic_partitions[topic])
|
self.partition_cycles[topic] = cycle(self.client.get_partition_ids_for_topic(topic))
|
||||||
except KeyError:
|
|
||||||
raise UnknownTopicOrPartitionError(topic)
|
|
||||||
|
|
||||||
# Randomize the initial partition that is returned
|
# Randomize the initial partition that is returned
|
||||||
if self.random_start:
|
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)):
|
for _ in xrange(random.randint(0, num_partitions-1)):
|
||||||
self.partition_cycles[topic].next()
|
self.partition_cycles[topic].next()
|
||||||
|
|
||||||
@@ -299,12 +297,13 @@ class KeyedProducer(Producer):
|
|||||||
|
|
||||||
def _next_partition(self, topic, key):
|
def _next_partition(self, topic, key):
|
||||||
if topic not in self.partitioners:
|
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.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]
|
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):
|
def send(self, topic, key, msg):
|
||||||
partition = self._next_partition(topic, key)
|
partition = self._next_partition(topic, key)
|
||||||
|
|||||||
@@ -148,7 +148,8 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase):
|
|||||||
producer = SimpleProducer(self.client)
|
producer = SimpleProducer(self.client)
|
||||||
|
|
||||||
# At first it doesn't exist
|
# At first it doesn't exist
|
||||||
with self.assertRaises(UnknownTopicOrPartitionError):
|
with self.assertRaises((UnknownTopicOrPartitionError,
|
||||||
|
LeaderNotAvailableError)):
|
||||||
resp = producer.send_messages(new_topic, self.msg("one"))
|
resp = producer.send_messages(new_topic, self.msg("one"))
|
||||||
|
|
||||||
@kafka_versions("all")
|
@kafka_versions("all")
|
||||||
|
|||||||
Reference in New Issue
Block a user