summaryrefslogtreecommitdiff
path: root/kafka
diff options
context:
space:
mode:
Diffstat (limited to 'kafka')
-rw-r--r--kafka/client.py7
-rw-r--r--kafka/consumer/base.py6
-rw-r--r--kafka/consumer/group.py15
-rw-r--r--kafka/consumer/kafka.py16
-rw-r--r--kafka/producer/base.py6
-rw-r--r--kafka/producer/keyed.py2
-rw-r--r--kafka/producer/simple.py3
-rw-r--r--kafka/util.py12
8 files changed, 19 insertions, 48 deletions
diff --git a/kafka/client.py b/kafka/client.py
index cb60d98..ca737c4 100644
--- a/kafka/client.py
+++ b/kafka/client.py
@@ -17,7 +17,6 @@ from kafka.common import (TopicAndPartition, BrokerMetadata, UnknownError,
from kafka.conn import collect_hosts, BrokerConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS
from kafka.protocol import KafkaProtocol
-from kafka.util import kafka_bytestring
log = logging.getLogger(__name__)
@@ -212,7 +211,7 @@ class KafkaClient(object):
failed_payloads(broker_payloads)
continue
- conn = self._get_conn(broker.host.decode('utf-8'), broker.port)
+ conn = self._get_conn(broker.host, broker.port)
request = encoder_fn(payloads=broker_payloads)
# decoder_fn=None signal that the server is expected to not
# send a response. This probably only applies to
@@ -305,7 +304,7 @@ class KafkaClient(object):
# Send the request, recv the response
try:
- conn = self._get_conn(broker.host.decode('utf-8'), broker.port)
+ conn = self._get_conn(broker.host, broker.port)
conn.send(requestId, request)
except ConnectionError as e:
@@ -410,14 +409,12 @@ class KafkaClient(object):
self.topic_partitions.clear()
def has_metadata_for_topic(self, topic):
- topic = kafka_bytestring(topic)
return (
topic in self.topic_partitions
and len(self.topic_partitions[topic]) > 0
)
def get_partition_ids_for_topic(self, topic):
- topic = kafka_bytestring(topic)
if topic not in self.topic_partitions:
return []
diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py
index 5859d36..4ac8c66 100644
--- a/kafka/consumer/base.py
+++ b/kafka/consumer/base.py
@@ -11,7 +11,7 @@ from kafka.common import (
UnknownTopicOrPartitionError, check_error, KafkaError
)
-from kafka.util import kafka_bytestring, ReentrantTimer
+from kafka.util import ReentrantTimer
log = logging.getLogger('kafka.consumer')
@@ -47,8 +47,8 @@ class Consumer(object):
auto_commit_every_t=AUTO_COMMIT_INTERVAL):
self.client = client
- self.topic = kafka_bytestring(topic)
- self.group = None if group is None else kafka_bytestring(group)
+ self.topic = topic
+ self.group = group
self.client.load_metadata_for_topics(topic)
self.offsets = {}
diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py
index 4a630ed..5d91469 100644
--- a/kafka/consumer/group.py
+++ b/kafka/consumer/group.py
@@ -16,7 +16,6 @@ from kafka.common import (
OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout,
FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError
)
-from kafka.util import kafka_bytestring
logger = logging.getLogger(__name__)
@@ -307,13 +306,13 @@ class KafkaConsumer(object):
# Topic name str -- all partitions
if isinstance(arg, (six.string_types, six.binary_type)):
- topic = kafka_bytestring(arg)
+ topic = arg
for partition in self._cluster.partitions_for_topic(topic):
self._consume_topic_partition(topic, partition)
# (topic, partition [, offset]) tuple
elif isinstance(arg, tuple):
- topic = kafka_bytestring(arg[0])
+ topic = arg[0]
partition = arg[1]
self._consume_topic_partition(topic, partition)
if len(arg) == 3:
@@ -326,7 +325,7 @@ class KafkaConsumer(object):
# key can be string (a topic)
if isinstance(key, (six.string_types, six.binary_type)):
- topic = kafka_bytestring(key)
+ topic = key
# topic: partition
if isinstance(value, int):
@@ -344,7 +343,7 @@ class KafkaConsumer(object):
# (topic, partition): offset
elif isinstance(key, tuple):
- topic = kafka_bytestring(key[0])
+ topic = key[0]
partition = key[1]
self._consume_topic_partition(topic, partition)
self._offsets.fetch[(topic, partition)] = value
@@ -463,7 +462,7 @@ class KafkaConsumer(object):
self._refresh_metadata_on_error()
continue
- topic = kafka_bytestring(resp.topic)
+ topic = resp.topic
partition = resp.partition
try:
check_error(resp)
@@ -662,7 +661,7 @@ class KafkaConsumer(object):
if commits:
logger.info('committing consumer offsets to group %s', self._config['group_id'])
resps = self._client.send_offset_commit_request(
- kafka_bytestring(self._config['group_id']), commits,
+ self._config['group_id'], commits,
fail_on_error=False
)
@@ -725,7 +724,7 @@ class KafkaConsumer(object):
logger.info("Consumer fetching stored offsets")
for topic_partition in self._topics:
(resp,) = self._client.send_offset_fetch_request(
- kafka_bytestring(self._config['group_id']),
+ self._config['group_id'],
[OffsetFetchRequest(topic_partition[0], topic_partition[1])],
fail_on_error=False)
try:
diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py
index fa70124..cdf8760 100644
--- a/kafka/consumer/kafka.py
+++ b/kafka/consumer/kafka.py
@@ -17,7 +17,6 @@ from kafka.common import (
OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout,
FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError
)
-from kafka.util import kafka_bytestring
logger = logging.getLogger(__name__)
@@ -193,14 +192,14 @@ class KafkaConsumer(object):
# Topic name str -- all partitions
if isinstance(arg, (six.string_types, six.binary_type)):
- topic = kafka_bytestring(arg)
+ topic = arg
for partition in self._client.get_partition_ids_for_topic(topic):
self._consume_topic_partition(topic, partition)
# (topic, partition [, offset]) tuple
elif isinstance(arg, tuple):
- topic = kafka_bytestring(arg[0])
+ topic = arg[0]
partition = arg[1]
self._consume_topic_partition(topic, partition)
if len(arg) == 3:
@@ -213,7 +212,7 @@ class KafkaConsumer(object):
# key can be string (a topic)
if isinstance(key, (six.string_types, six.binary_type)):
- topic = kafka_bytestring(key)
+ topic = key
# topic: partition
if isinstance(value, int):
@@ -231,7 +230,7 @@ class KafkaConsumer(object):
# (topic, partition): offset
elif isinstance(key, tuple):
- topic = kafka_bytestring(key[0])
+ topic = key[0]
partition = key[1]
self._consume_topic_partition(topic, partition)
self._offsets.fetch[(topic, partition)] = value
@@ -354,7 +353,7 @@ class KafkaConsumer(object):
self._refresh_metadata_on_error()
continue
- topic = kafka_bytestring(resp.topic)
+ topic = resp.topic
partition = resp.partition
try:
check_error(resp)
@@ -553,7 +552,7 @@ class KafkaConsumer(object):
if commits:
logger.info('committing consumer offsets to group %s', self._config['group_id'])
resps = self._client.send_offset_commit_request(
- kafka_bytestring(self._config['group_id']), commits,
+ self._config['group_id'], commits,
fail_on_error=False
)
@@ -577,7 +576,6 @@ class KafkaConsumer(object):
#
def _consume_topic_partition(self, topic, partition):
- topic = kafka_bytestring(topic)
if not isinstance(partition, int):
raise KafkaConfigurationError('Unknown partition type (%s) '
'-- expected int' % type(partition))
@@ -617,7 +615,7 @@ class KafkaConsumer(object):
logger.info("Consumer fetching stored offsets")
for topic_partition in self._topics:
(resp,) = self._client.send_offset_fetch_request(
- kafka_bytestring(self._config['group_id']),
+ self._config['group_id'],
[OffsetFetchRequestPayload(topic_partition[0], topic_partition[1])],
fail_on_error=False)
try:
diff --git a/kafka/producer/base.py b/kafka/producer/base.py
index 3f2bba6..595ac37 100644
--- a/kafka/producer/base.py
+++ b/kafka/producer/base.py
@@ -22,7 +22,6 @@ from kafka.common import (
)
from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set
-from kafka.util import kafka_bytestring
log = logging.getLogger('kafka.producer')
@@ -361,7 +360,6 @@ class Producer(object):
All messages produced via this method will set the message 'key' to Null
"""
- topic = kafka_bytestring(topic)
return self._send_messages(topic, partition, *msg)
def _send_messages(self, topic, partition, *msg, **kwargs):
@@ -381,10 +379,6 @@ class Producer(object):
elif not isinstance(m, six.binary_type):
raise TypeError("all produce message payloads must be null or type bytes")
- # Raise TypeError if topic is not encoded as bytes
- if not isinstance(topic, six.binary_type):
- raise TypeError("the topic must be type bytes")
-
# Raise TypeError if the key is not encoded as bytes
if key is not None and not isinstance(key, six.binary_type):
raise TypeError("the key must be type bytes")
diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py
index a5a26c9..f35aef0 100644
--- a/kafka/producer/keyed.py
+++ b/kafka/producer/keyed.py
@@ -5,7 +5,6 @@ import warnings
from .base import Producer
from ..partitioner import HashedPartitioner
-from ..util import kafka_bytestring
log = logging.getLogger(__name__)
@@ -38,7 +37,6 @@ class KeyedProducer(Producer):
return partitioner.partition(key)
def send_messages(self, topic, key, *msg):
- topic = kafka_bytestring(topic)
partition = self._next_partition(topic, key)
return self._send_messages(topic, partition, *msg, key=key)
diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py
index 13e60d9..d8647b6 100644
--- a/kafka/producer/simple.py
+++ b/kafka/producer/simple.py
@@ -46,9 +46,6 @@ class SimpleProducer(Producer):
return next(self.partition_cycles[topic])
def send_messages(self, topic, *msg):
- if not isinstance(topic, six.binary_type):
- topic = topic.encode('utf-8')
-
partition = self._next_partition(topic)
return super(SimpleProducer, self).send_messages(
topic, partition, *msg
diff --git a/kafka/util.py b/kafka/util.py
index e95d51d..c6e77fa 100644
--- a/kafka/util.py
+++ b/kafka/util.py
@@ -95,18 +95,6 @@ def group_by_topic_and_partition(tuples):
return out
-def kafka_bytestring(s):
- """
- Takes a string or bytes instance
- Returns bytes, encoding strings in utf-8 as necessary
- """
- if isinstance(s, six.binary_type):
- return s
- if isinstance(s, six.string_types):
- return s.encode('utf-8')
- raise TypeError(s)
-
-
class ReentrantTimer(object):
"""
A timer that can be restarted, unlike threading.Timer