Rename TopicAndPartition -> TopicPartition
This commit is contained in:
@@ -8,7 +8,7 @@ import time
|
||||
import six
|
||||
|
||||
import kafka.common
|
||||
from kafka.common import (TopicAndPartition, BrokerMetadata, UnknownError,
|
||||
from kafka.common import (TopicPartition, BrokerMetadata, UnknownError,
|
||||
ConnectionError, FailedPayloadsError,
|
||||
KafkaTimeoutError, KafkaUnavailableError,
|
||||
LeaderNotAvailableError, UnknownTopicOrPartitionError,
|
||||
@@ -41,7 +41,7 @@ class KafkaClient(object):
|
||||
|
||||
self._conns = {}
|
||||
self.brokers = {} # broker_id -> BrokerMetadata
|
||||
self.topics_to_brokers = {} # TopicAndPartition -> BrokerMetadata
|
||||
self.topics_to_brokers = {} # TopicPartition -> BrokerMetadata
|
||||
self.topic_partitions = {} # topic -> partition -> PartitionMetadata
|
||||
|
||||
self.load_metadata_for_topics() # bootstrap with all metadata
|
||||
@@ -77,7 +77,7 @@ class KafkaClient(object):
|
||||
no current leader
|
||||
"""
|
||||
|
||||
key = TopicAndPartition(topic, partition)
|
||||
key = TopicPartition(topic, partition)
|
||||
|
||||
# Use cached metadata if it is there
|
||||
if self.topics_to_brokers.get(key) is not None:
|
||||
@@ -511,7 +511,7 @@ class KafkaClient(object):
|
||||
self.topic_partitions[topic][partition] = leader
|
||||
|
||||
# Populate topics_to_brokers dict
|
||||
topic_part = TopicAndPartition(topic, partition)
|
||||
topic_part = TopicPartition(topic, partition)
|
||||
|
||||
# Check for partition errors
|
||||
if error:
|
||||
|
@@ -72,7 +72,7 @@ OffsetAndMessage = namedtuple("OffsetAndMessage",
|
||||
Message = namedtuple("Message",
|
||||
["magic", "attributes", "key", "value"])
|
||||
|
||||
TopicAndPartition = namedtuple("TopicAndPartition",
|
||||
TopicPartition = namedtuple("TopicPartition",
|
||||
["topic", "partition"])
|
||||
|
||||
KafkaMessage = namedtuple("KafkaMessage",
|
||||
|
@@ -15,7 +15,7 @@ from threading import Thread, Event
|
||||
import six
|
||||
|
||||
from kafka.common import (
|
||||
ProduceRequestPayload, ProduceResponsePayload, TopicAndPartition, RetryOptions,
|
||||
ProduceRequestPayload, ProduceResponsePayload, TopicPartition, RetryOptions,
|
||||
kafka_errors, UnsupportedCodecError, FailedPayloadsError,
|
||||
RequestTimedOutError, AsyncProducerQueueFull, UnknownError,
|
||||
RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES
|
||||
@@ -386,7 +386,7 @@ class Producer(object):
|
||||
if self.async:
|
||||
for idx, m in enumerate(msg):
|
||||
try:
|
||||
item = (TopicAndPartition(topic, partition), m, key)
|
||||
item = (TopicPartition(topic, partition), m, key)
|
||||
if self.async_queue_put_timeout == 0:
|
||||
self.queue.put_nowait(item)
|
||||
else:
|
||||
|
@@ -9,7 +9,7 @@ from kafka import KafkaClient
|
||||
from kafka.common import (
|
||||
ProduceRequestPayload,
|
||||
BrokerMetadata,
|
||||
TopicAndPartition, KafkaUnavailableError,
|
||||
TopicPartition, KafkaUnavailableError,
|
||||
LeaderNotAvailableError, UnknownTopicOrPartitionError,
|
||||
KafkaTimeoutError, ConnectionError
|
||||
)
|
||||
@@ -145,12 +145,12 @@ class TestKafkaClient(unittest.TestCase):
|
||||
# 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]},
|
||||
TopicPartition('topic_1', 0): brokers[1],
|
||||
TopicPartition('topic_noleader', 0): None,
|
||||
TopicPartition('topic_noleader', 1): None,
|
||||
TopicPartition('topic_3', 0): brokers[0],
|
||||
TopicPartition('topic_3', 1): brokers[1],
|
||||
TopicPartition('topic_3', 2): brokers[0]},
|
||||
client.topics_to_brokers)
|
||||
|
||||
# if we ask for metadata explicitly, it should raise errors
|
||||
@@ -260,7 +260,7 @@ class TestKafkaClient(unittest.TestCase):
|
||||
|
||||
self.assertEqual(brokers[0], leader)
|
||||
self.assertDictEqual({
|
||||
TopicAndPartition('topic_one_partition', 0): brokers[0]},
|
||||
TopicPartition('topic_one_partition', 0): brokers[0]},
|
||||
client.topics_to_brokers)
|
||||
|
||||
@patch('kafka.client.KafkaClient._get_conn')
|
||||
@@ -312,8 +312,8 @@ class TestKafkaClient(unittest.TestCase):
|
||||
client = KafkaClient(hosts=['broker_1:4567'])
|
||||
self.assertDictEqual(
|
||||
{
|
||||
TopicAndPartition('topic_noleader', 0): None,
|
||||
TopicAndPartition('topic_noleader', 1): None
|
||||
TopicPartition('topic_noleader', 0): None,
|
||||
TopicPartition('topic_noleader', 1): None
|
||||
},
|
||||
client.topics_to_brokers)
|
||||
|
||||
|
@@ -3,7 +3,7 @@ import os
|
||||
import time
|
||||
|
||||
from kafka import KafkaClient, SimpleConsumer, KeyedProducer
|
||||
from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError
|
||||
from kafka.common import TopicPartition, FailedPayloadsError, ConnectionError
|
||||
from kafka.producer.base import Producer
|
||||
|
||||
from test.fixtures import ZookeeperFixture, KafkaFixture
|
||||
@@ -202,7 +202,7 @@ class TestFailover(KafkaIntegrationTestCase):
|
||||
break
|
||||
|
||||
def _kill_leader(self, topic, partition):
|
||||
leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)]
|
||||
leader = self.client.topics_to_brokers[TopicPartition(topic, partition)]
|
||||
broker = self.brokers[leader.nodeId]
|
||||
broker.close()
|
||||
return broker
|
||||
|
@@ -10,7 +10,7 @@ from . import unittest
|
||||
from kafka import KafkaClient, SimpleProducer, KeyedProducer
|
||||
from kafka.common import (
|
||||
AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError,
|
||||
ProduceResponsePayload, RetryOptions, TopicAndPartition
|
||||
ProduceResponsePayload, RetryOptions, TopicPartition
|
||||
)
|
||||
from kafka.producer.base import Producer, _send_upstream
|
||||
from kafka.protocol import CODEC_NONE
|
||||
@@ -156,7 +156,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase):
|
||||
|
||||
# lets create a queue and add 10 messages for 1 partition
|
||||
for i in range(10):
|
||||
self.queue.put((TopicAndPartition("test", 0), "msg %i", "key %i"))
|
||||
self.queue.put((TopicPartition("test", 0), "msg %i", "key %i"))
|
||||
|
||||
self._run_process()
|
||||
|
||||
@@ -172,7 +172,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase):
|
||||
# lets create a queue and add 10 messages for 10 different partitions
|
||||
# to show how retries should work ideally
|
||||
for i in range(10):
|
||||
self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i"))
|
||||
self.queue.put((TopicPartition("test", i), "msg %i", "key %i"))
|
||||
|
||||
# Mock offsets counter for closure
|
||||
offsets = collections.defaultdict(lambda: collections.defaultdict(lambda: 0))
|
||||
@@ -206,7 +206,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase):
|
||||
# lets create a queue and add 10 messages for 10 different partitions
|
||||
# to show how retries should work ideally
|
||||
for i in range(10):
|
||||
self.queue.put((TopicAndPartition("test", i), "msg %i" % i, "key %i" % i))
|
||||
self.queue.put((TopicPartition("test", i), "msg %i" % i, "key %i" % i))
|
||||
|
||||
def send_side_effect(reqs, *args, **kwargs):
|
||||
return [FailedPayloadsError(req) for req in reqs]
|
||||
@@ -226,7 +226,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase):
|
||||
def test_async_producer_not_leader(self):
|
||||
|
||||
for i in range(10):
|
||||
self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i"))
|
||||
self.queue.put((TopicPartition("test", i), "msg %i", "key %i"))
|
||||
|
||||
# Mock offsets counter for closure
|
||||
offsets = collections.defaultdict(lambda: collections.defaultdict(lambda: 0))
|
||||
|
@@ -12,7 +12,7 @@ from kafka.common import (
|
||||
OffsetResponsePayload, OffsetCommitResponsePayload, OffsetFetchResponsePayload,
|
||||
ProduceRequestPayload, FetchRequestPayload, Message, ChecksumError,
|
||||
ProduceResponsePayload, FetchResponsePayload, OffsetAndMessage,
|
||||
BrokerMetadata, TopicMetadata, PartitionMetadata, TopicAndPartition,
|
||||
BrokerMetadata, TopicMetadata, PartitionMetadata,
|
||||
KafkaUnavailableError, UnsupportedCodecError, ConsumerFetchSizeTooSmall,
|
||||
ProtocolError, ConsumerMetadataResponse
|
||||
)
|
||||
|
@@ -104,7 +104,7 @@ class UtilTest(unittest.TestCase):
|
||||
kafka.util.relative_unpack('>hh', '\x00', 0)
|
||||
|
||||
def test_group_by_topic_and_partition(self):
|
||||
t = kafka.common.TopicAndPartition
|
||||
t = kafka.common.TopicPartition
|
||||
|
||||
l = [
|
||||
t("a", 1),
|
||||
|
Reference in New Issue
Block a user