Merge pull request #166 from patricklucas/teach_producers_about_compression
Add 'codec' parameter to Producer
This commit is contained in:
@@ -170,6 +170,11 @@ class ConsumerNoMoreData(KafkaError):
|
|||||||
class ProtocolError(KafkaError):
|
class ProtocolError(KafkaError):
|
||||||
pass
|
pass
|
||||||
|
|
||||||
|
|
||||||
|
class UnsupportedCodecError(KafkaError):
|
||||||
|
pass
|
||||||
|
|
||||||
|
|
||||||
kafka_errors = {
|
kafka_errors = {
|
||||||
-1 : UnknownError,
|
-1 : UnknownError,
|
||||||
1 : OffsetOutOfRangeError,
|
1 : OffsetOutOfRangeError,
|
||||||
@@ -187,6 +192,7 @@ kafka_errors = {
|
|||||||
13 : StaleLeaderEpochCodeError,
|
13 : StaleLeaderEpochCodeError,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def check_error(response):
|
def check_error(response):
|
||||||
error = kafka_errors.get(response.error)
|
error = kafka_errors.get(response.error)
|
||||||
if error:
|
if error:
|
||||||
|
@@ -9,9 +9,11 @@ from collections import defaultdict
|
|||||||
from itertools import cycle
|
from itertools import cycle
|
||||||
from multiprocessing import Queue, Process
|
from multiprocessing import Queue, Process
|
||||||
|
|
||||||
from kafka.common import ProduceRequest, TopicAndPartition
|
from kafka.common import (
|
||||||
|
ProduceRequest, TopicAndPartition, UnsupportedCodecError
|
||||||
|
)
|
||||||
from kafka.partitioner import HashedPartitioner
|
from kafka.partitioner import HashedPartitioner
|
||||||
from kafka.protocol import create_message
|
from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set
|
||||||
|
|
||||||
log = logging.getLogger("kafka")
|
log = logging.getLogger("kafka")
|
||||||
|
|
||||||
@@ -21,7 +23,7 @@ BATCH_SEND_MSG_COUNT = 20
|
|||||||
STOP_ASYNC_PRODUCER = -1
|
STOP_ASYNC_PRODUCER = -1
|
||||||
|
|
||||||
|
|
||||||
def _send_upstream(queue, client, batch_time, batch_size,
|
def _send_upstream(queue, client, codec, batch_time, batch_size,
|
||||||
req_acks, ack_timeout):
|
req_acks, ack_timeout):
|
||||||
"""
|
"""
|
||||||
Listen on the queue for a specified number of messages or till
|
Listen on the queue for a specified number of messages or till
|
||||||
@@ -62,7 +64,8 @@ def _send_upstream(queue, client, batch_time, batch_size,
|
|||||||
|
|
||||||
# Send collected requests upstream
|
# Send collected requests upstream
|
||||||
reqs = []
|
reqs = []
|
||||||
for topic_partition, messages in msgset.items():
|
for topic_partition, msg in msgset.items():
|
||||||
|
messages = create_message_set(msg, codec)
|
||||||
req = ProduceRequest(topic_partition.topic,
|
req = ProduceRequest(topic_partition.topic,
|
||||||
topic_partition.partition,
|
topic_partition.partition,
|
||||||
messages)
|
messages)
|
||||||
@@ -102,6 +105,7 @@ class Producer(object):
|
|||||||
def __init__(self, client, async=False,
|
def __init__(self, client, async=False,
|
||||||
req_acks=ACK_AFTER_LOCAL_WRITE,
|
req_acks=ACK_AFTER_LOCAL_WRITE,
|
||||||
ack_timeout=DEFAULT_ACK_TIMEOUT,
|
ack_timeout=DEFAULT_ACK_TIMEOUT,
|
||||||
|
codec=None,
|
||||||
batch_send=False,
|
batch_send=False,
|
||||||
batch_send_every_n=BATCH_SEND_MSG_COUNT,
|
batch_send_every_n=BATCH_SEND_MSG_COUNT,
|
||||||
batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL):
|
batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL):
|
||||||
@@ -119,11 +123,19 @@ class Producer(object):
|
|||||||
self.req_acks = req_acks
|
self.req_acks = req_acks
|
||||||
self.ack_timeout = ack_timeout
|
self.ack_timeout = ack_timeout
|
||||||
|
|
||||||
|
if codec is None:
|
||||||
|
codec = CODEC_NONE
|
||||||
|
elif codec not in ALL_CODECS:
|
||||||
|
raise UnsupportedCodecError("Codec 0x%02x unsupported" % codec)
|
||||||
|
|
||||||
|
self.codec = codec
|
||||||
|
|
||||||
if self.async:
|
if self.async:
|
||||||
self.queue = Queue() # Messages are sent through this queue
|
self.queue = Queue() # Messages are sent through this queue
|
||||||
self.proc = Process(target=_send_upstream,
|
self.proc = Process(target=_send_upstream,
|
||||||
args=(self.queue,
|
args=(self.queue,
|
||||||
self.client.copy(),
|
self.client.copy(),
|
||||||
|
self.codec,
|
||||||
batch_send_every_t,
|
batch_send_every_t,
|
||||||
batch_send_every_n,
|
batch_send_every_n,
|
||||||
self.req_acks,
|
self.req_acks,
|
||||||
@@ -139,11 +151,10 @@ class Producer(object):
|
|||||||
"""
|
"""
|
||||||
if self.async:
|
if self.async:
|
||||||
for m in msg:
|
for m in msg:
|
||||||
self.queue.put((TopicAndPartition(topic, partition),
|
self.queue.put((TopicAndPartition(topic, partition), m))
|
||||||
create_message(m)))
|
|
||||||
resp = []
|
resp = []
|
||||||
else:
|
else:
|
||||||
messages = [create_message(m) for m in msg]
|
messages = create_message_set(msg, self.codec)
|
||||||
req = ProduceRequest(topic, partition, messages)
|
req = ProduceRequest(topic, partition, messages)
|
||||||
try:
|
try:
|
||||||
resp = self.client.send_produce_request([req], acks=self.req_acks,
|
resp = self.client.send_produce_request([req], acks=self.req_acks,
|
||||||
@@ -168,7 +179,7 @@ class Producer(object):
|
|||||||
|
|
||||||
class SimpleProducer(Producer):
|
class SimpleProducer(Producer):
|
||||||
"""
|
"""
|
||||||
A simple, round-robbin producer. Each message goes to exactly one partition
|
A simple, round-robin producer. Each message goes to exactly one partition
|
||||||
|
|
||||||
Params:
|
Params:
|
||||||
client - The Kafka client instance to use
|
client - The Kafka client instance to use
|
||||||
@@ -189,6 +200,7 @@ class SimpleProducer(Producer):
|
|||||||
def __init__(self, client, async=False,
|
def __init__(self, client, async=False,
|
||||||
req_acks=Producer.ACK_AFTER_LOCAL_WRITE,
|
req_acks=Producer.ACK_AFTER_LOCAL_WRITE,
|
||||||
ack_timeout=Producer.DEFAULT_ACK_TIMEOUT,
|
ack_timeout=Producer.DEFAULT_ACK_TIMEOUT,
|
||||||
|
codec=None,
|
||||||
batch_send=False,
|
batch_send=False,
|
||||||
batch_send_every_n=BATCH_SEND_MSG_COUNT,
|
batch_send_every_n=BATCH_SEND_MSG_COUNT,
|
||||||
batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL,
|
batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL,
|
||||||
@@ -196,7 +208,7 @@ class SimpleProducer(Producer):
|
|||||||
self.partition_cycles = {}
|
self.partition_cycles = {}
|
||||||
self.random_start = random_start
|
self.random_start = random_start
|
||||||
super(SimpleProducer, self).__init__(client, async, req_acks,
|
super(SimpleProducer, self).__init__(client, async, req_acks,
|
||||||
ack_timeout, batch_send,
|
ack_timeout, codec, batch_send,
|
||||||
batch_send_every_n,
|
batch_send_every_n,
|
||||||
batch_send_every_t)
|
batch_send_every_t)
|
||||||
|
|
||||||
@@ -241,6 +253,7 @@ class KeyedProducer(Producer):
|
|||||||
def __init__(self, client, partitioner=None, async=False,
|
def __init__(self, client, partitioner=None, async=False,
|
||||||
req_acks=Producer.ACK_AFTER_LOCAL_WRITE,
|
req_acks=Producer.ACK_AFTER_LOCAL_WRITE,
|
||||||
ack_timeout=Producer.DEFAULT_ACK_TIMEOUT,
|
ack_timeout=Producer.DEFAULT_ACK_TIMEOUT,
|
||||||
|
codec=None,
|
||||||
batch_send=False,
|
batch_send=False,
|
||||||
batch_send_every_n=BATCH_SEND_MSG_COUNT,
|
batch_send_every_n=BATCH_SEND_MSG_COUNT,
|
||||||
batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL):
|
batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL):
|
||||||
@@ -250,7 +263,7 @@ class KeyedProducer(Producer):
|
|||||||
self.partitioners = {}
|
self.partitioners = {}
|
||||||
|
|
||||||
super(KeyedProducer, self).__init__(client, async, req_acks,
|
super(KeyedProducer, self).__init__(client, async, req_acks,
|
||||||
ack_timeout, batch_send,
|
ack_timeout, codec, batch_send,
|
||||||
batch_send_every_n,
|
batch_send_every_n,
|
||||||
batch_send_every_t)
|
batch_send_every_t)
|
||||||
|
|
||||||
|
@@ -9,7 +9,8 @@ from kafka.common import (
|
|||||||
BrokerMetadata, PartitionMetadata, Message, OffsetAndMessage,
|
BrokerMetadata, PartitionMetadata, Message, OffsetAndMessage,
|
||||||
ProduceResponse, FetchResponse, OffsetResponse,
|
ProduceResponse, FetchResponse, OffsetResponse,
|
||||||
OffsetCommitResponse, OffsetFetchResponse, ProtocolError,
|
OffsetCommitResponse, OffsetFetchResponse, ProtocolError,
|
||||||
BufferUnderflowError, ChecksumError, ConsumerFetchSizeTooSmall
|
BufferUnderflowError, ChecksumError, ConsumerFetchSizeTooSmall,
|
||||||
|
UnsupportedCodecError
|
||||||
)
|
)
|
||||||
from kafka.util import (
|
from kafka.util import (
|
||||||
read_short_string, read_int_string, relative_unpack,
|
read_short_string, read_int_string, relative_unpack,
|
||||||
@@ -18,6 +19,12 @@ from kafka.util import (
|
|||||||
|
|
||||||
log = logging.getLogger("kafka")
|
log = logging.getLogger("kafka")
|
||||||
|
|
||||||
|
ATTRIBUTE_CODEC_MASK = 0x03
|
||||||
|
CODEC_NONE = 0x00
|
||||||
|
CODEC_GZIP = 0x01
|
||||||
|
CODEC_SNAPPY = 0x02
|
||||||
|
ALL_CODECS = (CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY)
|
||||||
|
|
||||||
|
|
||||||
class KafkaProtocol(object):
|
class KafkaProtocol(object):
|
||||||
"""
|
"""
|
||||||
@@ -32,11 +39,6 @@ class KafkaProtocol(object):
|
|||||||
OFFSET_COMMIT_KEY = 8
|
OFFSET_COMMIT_KEY = 8
|
||||||
OFFSET_FETCH_KEY = 9
|
OFFSET_FETCH_KEY = 9
|
||||||
|
|
||||||
ATTRIBUTE_CODEC_MASK = 0x03
|
|
||||||
CODEC_NONE = 0x00
|
|
||||||
CODEC_GZIP = 0x01
|
|
||||||
CODEC_SNAPPY = 0x02
|
|
||||||
|
|
||||||
###################
|
###################
|
||||||
# Private API #
|
# Private API #
|
||||||
###################
|
###################
|
||||||
@@ -150,17 +152,17 @@ class KafkaProtocol(object):
|
|||||||
(key, cur) = read_int_string(data, cur)
|
(key, cur) = read_int_string(data, cur)
|
||||||
(value, cur) = read_int_string(data, cur)
|
(value, cur) = read_int_string(data, cur)
|
||||||
|
|
||||||
codec = att & KafkaProtocol.ATTRIBUTE_CODEC_MASK
|
codec = att & ATTRIBUTE_CODEC_MASK
|
||||||
|
|
||||||
if codec == KafkaProtocol.CODEC_NONE:
|
if codec == CODEC_NONE:
|
||||||
yield (offset, Message(magic, att, key, value))
|
yield (offset, Message(magic, att, key, value))
|
||||||
|
|
||||||
elif codec == KafkaProtocol.CODEC_GZIP:
|
elif codec == CODEC_GZIP:
|
||||||
gz = gzip_decode(value)
|
gz = gzip_decode(value)
|
||||||
for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz):
|
for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz):
|
||||||
yield (offset, msg)
|
yield (offset, msg)
|
||||||
|
|
||||||
elif codec == KafkaProtocol.CODEC_SNAPPY:
|
elif codec == CODEC_SNAPPY:
|
||||||
snp = snappy_decode(value)
|
snp = snappy_decode(value)
|
||||||
for (offset, msg) in KafkaProtocol._decode_message_set_iter(snp):
|
for (offset, msg) in KafkaProtocol._decode_message_set_iter(snp):
|
||||||
yield (offset, msg)
|
yield (offset, msg)
|
||||||
@@ -543,7 +545,7 @@ def create_gzip_message(payloads, key=None):
|
|||||||
[create_message(payload) for payload in payloads])
|
[create_message(payload) for payload in payloads])
|
||||||
|
|
||||||
gzipped = gzip_encode(message_set)
|
gzipped = gzip_encode(message_set)
|
||||||
codec = KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_GZIP
|
codec = ATTRIBUTE_CODEC_MASK & CODEC_GZIP
|
||||||
|
|
||||||
return Message(0, 0x00 | codec, key, gzipped)
|
return Message(0, 0x00 | codec, key, gzipped)
|
||||||
|
|
||||||
@@ -564,6 +566,22 @@ def create_snappy_message(payloads, key=None):
|
|||||||
[create_message(payload) for payload in payloads])
|
[create_message(payload) for payload in payloads])
|
||||||
|
|
||||||
snapped = snappy_encode(message_set)
|
snapped = snappy_encode(message_set)
|
||||||
codec = KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_SNAPPY
|
codec = ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY
|
||||||
|
|
||||||
return Message(0, 0x00 | codec, key, snapped)
|
return Message(0, 0x00 | codec, key, snapped)
|
||||||
|
|
||||||
|
|
||||||
|
def create_message_set(messages, codec=CODEC_NONE):
|
||||||
|
"""Create a message set using the given codec.
|
||||||
|
|
||||||
|
If codec is CODEC_NONE, return a list of raw Kafka messages. Otherwise,
|
||||||
|
return a list containing a single codec-encoded message.
|
||||||
|
"""
|
||||||
|
if codec == CODEC_NONE:
|
||||||
|
return [create_message(m) for m in messages]
|
||||||
|
elif codec == CODEC_GZIP:
|
||||||
|
return [create_gzip_message(messages)]
|
||||||
|
elif codec == CODEC_SNAPPY:
|
||||||
|
return [create_snappy_message(messages)]
|
||||||
|
else:
|
||||||
|
raise UnsupportedCodecError("Codec 0x%02x unsupported" % codec)
|
||||||
|
@@ -1,22 +1,30 @@
|
|||||||
|
import contextlib
|
||||||
|
from contextlib import contextmanager
|
||||||
import struct
|
import struct
|
||||||
import unittest2
|
import unittest2
|
||||||
|
|
||||||
|
import mock
|
||||||
|
from mock import sentinel
|
||||||
|
|
||||||
from kafka import KafkaClient
|
from kafka import KafkaClient
|
||||||
from kafka.common import (
|
from kafka.common import (
|
||||||
OffsetRequest, OffsetCommitRequest, OffsetFetchRequest,
|
OffsetRequest, OffsetCommitRequest, OffsetFetchRequest,
|
||||||
OffsetResponse, OffsetCommitResponse, OffsetFetchResponse,
|
OffsetResponse, OffsetCommitResponse, OffsetFetchResponse,
|
||||||
ProduceRequest, FetchRequest, Message, ChecksumError,
|
ProduceRequest, FetchRequest, Message, ChecksumError,
|
||||||
ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse,
|
ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, OffsetAndMessage,
|
||||||
OffsetAndMessage, BrokerMetadata, PartitionMetadata,
|
BrokerMetadata, PartitionMetadata, TopicAndPartition, KafkaUnavailableError,
|
||||||
TopicAndPartition, KafkaUnavailableError, ProtocolError,
|
ProtocolError, LeaderUnavailableError, PartitionUnavailableError,
|
||||||
LeaderUnavailableError, PartitionUnavailableError
|
UnsupportedCodecError
|
||||||
)
|
)
|
||||||
from kafka.codec import (
|
from kafka.codec import (
|
||||||
has_snappy, gzip_encode, gzip_decode,
|
has_snappy, gzip_encode, gzip_decode,
|
||||||
snappy_encode, snappy_decode
|
snappy_encode, snappy_decode
|
||||||
)
|
)
|
||||||
|
import kafka.protocol
|
||||||
from kafka.protocol import (
|
from kafka.protocol import (
|
||||||
create_gzip_message, create_message, create_snappy_message, KafkaProtocol
|
ATTRIBUTE_CODEC_MASK, CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY, KafkaProtocol,
|
||||||
|
create_message, create_gzip_message, create_snappy_message,
|
||||||
|
create_message_set
|
||||||
)
|
)
|
||||||
|
|
||||||
class TestProtocol(unittest2.TestCase):
|
class TestProtocol(unittest2.TestCase):
|
||||||
@@ -33,8 +41,7 @@ class TestProtocol(unittest2.TestCase):
|
|||||||
payloads = ["v1", "v2"]
|
payloads = ["v1", "v2"]
|
||||||
msg = create_gzip_message(payloads)
|
msg = create_gzip_message(payloads)
|
||||||
self.assertEqual(msg.magic, 0)
|
self.assertEqual(msg.magic, 0)
|
||||||
self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK &
|
self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_GZIP)
|
||||||
KafkaProtocol.CODEC_GZIP)
|
|
||||||
self.assertEqual(msg.key, None)
|
self.assertEqual(msg.key, None)
|
||||||
# Need to decode to check since gzipped payload is non-deterministic
|
# Need to decode to check since gzipped payload is non-deterministic
|
||||||
decoded = gzip_decode(msg.value)
|
decoded = gzip_decode(msg.value)
|
||||||
@@ -63,8 +70,7 @@ class TestProtocol(unittest2.TestCase):
|
|||||||
payloads = ["v1", "v2"]
|
payloads = ["v1", "v2"]
|
||||||
msg = create_snappy_message(payloads)
|
msg = create_snappy_message(payloads)
|
||||||
self.assertEqual(msg.magic, 0)
|
self.assertEqual(msg.magic, 0)
|
||||||
self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK &
|
self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY)
|
||||||
KafkaProtocol.CODEC_SNAPPY)
|
|
||||||
self.assertEqual(msg.key, None)
|
self.assertEqual(msg.key, None)
|
||||||
decoded = snappy_decode(msg.value)
|
decoded = snappy_decode(msg.value)
|
||||||
expect = "".join([
|
expect = "".join([
|
||||||
@@ -692,3 +698,48 @@ class TestProtocol(unittest2.TestCase):
|
|||||||
OffsetFetchResponse(topic = 'topic1', partition = 2, offset = 4, error = 0, metadata = "meta"),
|
OffsetFetchResponse(topic = 'topic1', partition = 2, offset = 4, error = 0, metadata = "meta"),
|
||||||
OffsetFetchResponse(topic = 'topic1', partition = 4, offset = 8, error = 0, metadata = "meta"),
|
OffsetFetchResponse(topic = 'topic1', partition = 4, offset = 8, error = 0, metadata = "meta"),
|
||||||
]))
|
]))
|
||||||
|
|
||||||
|
@contextmanager
|
||||||
|
def mock_create_message_fns(self):
|
||||||
|
patches = contextlib.nested(
|
||||||
|
mock.patch.object(kafka.protocol, "create_message",
|
||||||
|
return_value=sentinel.message),
|
||||||
|
mock.patch.object(kafka.protocol, "create_gzip_message",
|
||||||
|
return_value=sentinel.gzip_message),
|
||||||
|
mock.patch.object(kafka.protocol, "create_snappy_message",
|
||||||
|
return_value=sentinel.snappy_message),
|
||||||
|
)
|
||||||
|
|
||||||
|
with patches:
|
||||||
|
yield
|
||||||
|
|
||||||
|
def test_create_message_set(self):
|
||||||
|
messages = [1, 2, 3]
|
||||||
|
|
||||||
|
# Default codec is CODEC_NONE. Expect list of regular messages.
|
||||||
|
expect = [sentinel.message] * len(messages)
|
||||||
|
with self.mock_create_message_fns():
|
||||||
|
message_set = create_message_set(messages)
|
||||||
|
self.assertEqual(message_set, expect)
|
||||||
|
|
||||||
|
# CODEC_NONE: Expect list of regular messages.
|
||||||
|
expect = [sentinel.message] * len(messages)
|
||||||
|
with self.mock_create_message_fns():
|
||||||
|
message_set = create_message_set(messages, CODEC_NONE)
|
||||||
|
self.assertEqual(message_set, expect)
|
||||||
|
|
||||||
|
# CODEC_GZIP: Expect list of one gzip-encoded message.
|
||||||
|
expect = [sentinel.gzip_message]
|
||||||
|
with self.mock_create_message_fns():
|
||||||
|
message_set = create_message_set(messages, CODEC_GZIP)
|
||||||
|
self.assertEqual(message_set, expect)
|
||||||
|
|
||||||
|
# CODEC_SNAPPY: Expect list of one snappy-encoded message.
|
||||||
|
expect = [sentinel.snappy_message]
|
||||||
|
with self.mock_create_message_fns():
|
||||||
|
message_set = create_message_set(messages, CODEC_SNAPPY)
|
||||||
|
self.assertEqual(message_set, expect)
|
||||||
|
|
||||||
|
# Unknown codec should raise UnsupportedCodecError.
|
||||||
|
with self.assertRaises(UnsupportedCodecError):
|
||||||
|
create_message_set(messages, -1)
|
||||||
|
Reference in New Issue
Block a user