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):
 | 
			
		||||
    pass
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
class UnsupportedCodecError(KafkaError):
 | 
			
		||||
    pass
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
kafka_errors = {
 | 
			
		||||
    -1 : UnknownError,
 | 
			
		||||
    1  : OffsetOutOfRangeError,
 | 
			
		||||
@@ -187,6 +192,7 @@ kafka_errors = {
 | 
			
		||||
    13 : StaleLeaderEpochCodeError,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
def check_error(response):
 | 
			
		||||
    error = kafka_errors.get(response.error)
 | 
			
		||||
    if error:
 | 
			
		||||
 
 | 
			
		||||
@@ -9,9 +9,11 @@ from collections import defaultdict
 | 
			
		||||
from itertools import cycle
 | 
			
		||||
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.protocol import create_message
 | 
			
		||||
from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set
 | 
			
		||||
 | 
			
		||||
log = logging.getLogger("kafka")
 | 
			
		||||
 | 
			
		||||
@@ -21,7 +23,7 @@ BATCH_SEND_MSG_COUNT = 20
 | 
			
		||||
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):
 | 
			
		||||
    """
 | 
			
		||||
    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
 | 
			
		||||
        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,
 | 
			
		||||
                                 topic_partition.partition,
 | 
			
		||||
                                 messages)
 | 
			
		||||
@@ -102,6 +105,7 @@ class Producer(object):
 | 
			
		||||
    def __init__(self, client, async=False,
 | 
			
		||||
                 req_acks=ACK_AFTER_LOCAL_WRITE,
 | 
			
		||||
                 ack_timeout=DEFAULT_ACK_TIMEOUT,
 | 
			
		||||
                 codec=None,
 | 
			
		||||
                 batch_send=False,
 | 
			
		||||
                 batch_send_every_n=BATCH_SEND_MSG_COUNT,
 | 
			
		||||
                 batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL):
 | 
			
		||||
@@ -119,11 +123,19 @@ class Producer(object):
 | 
			
		||||
        self.req_acks = req_acks
 | 
			
		||||
        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:
 | 
			
		||||
            self.queue = Queue()  # Messages are sent through this queue
 | 
			
		||||
            self.proc = Process(target=_send_upstream,
 | 
			
		||||
                                args=(self.queue,
 | 
			
		||||
                                      self.client.copy(),
 | 
			
		||||
                                      self.codec,
 | 
			
		||||
                                      batch_send_every_t,
 | 
			
		||||
                                      batch_send_every_n,
 | 
			
		||||
                                      self.req_acks,
 | 
			
		||||
@@ -139,11 +151,10 @@ class Producer(object):
 | 
			
		||||
        """
 | 
			
		||||
        if self.async:
 | 
			
		||||
            for m in msg:
 | 
			
		||||
                self.queue.put((TopicAndPartition(topic, partition),
 | 
			
		||||
                                create_message(m)))
 | 
			
		||||
                self.queue.put((TopicAndPartition(topic, partition), m))
 | 
			
		||||
            resp = []
 | 
			
		||||
        else:
 | 
			
		||||
            messages = [create_message(m) for m in msg]
 | 
			
		||||
            messages = create_message_set(msg, self.codec)
 | 
			
		||||
            req = ProduceRequest(topic, partition, messages)
 | 
			
		||||
            try:
 | 
			
		||||
                resp = self.client.send_produce_request([req], acks=self.req_acks,
 | 
			
		||||
@@ -168,7 +179,7 @@ class Producer(object):
 | 
			
		||||
 | 
			
		||||
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:
 | 
			
		||||
    client - The Kafka client instance to use
 | 
			
		||||
@@ -189,6 +200,7 @@ class SimpleProducer(Producer):
 | 
			
		||||
    def __init__(self, client, async=False,
 | 
			
		||||
                 req_acks=Producer.ACK_AFTER_LOCAL_WRITE,
 | 
			
		||||
                 ack_timeout=Producer.DEFAULT_ACK_TIMEOUT,
 | 
			
		||||
                 codec=None,
 | 
			
		||||
                 batch_send=False,
 | 
			
		||||
                 batch_send_every_n=BATCH_SEND_MSG_COUNT,
 | 
			
		||||
                 batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL,
 | 
			
		||||
@@ -196,7 +208,7 @@ class SimpleProducer(Producer):
 | 
			
		||||
        self.partition_cycles = {}
 | 
			
		||||
        self.random_start = random_start
 | 
			
		||||
        super(SimpleProducer, self).__init__(client, async, req_acks,
 | 
			
		||||
                                             ack_timeout, batch_send,
 | 
			
		||||
                                             ack_timeout, codec, batch_send,
 | 
			
		||||
                                             batch_send_every_n,
 | 
			
		||||
                                             batch_send_every_t)
 | 
			
		||||
 | 
			
		||||
@@ -241,6 +253,7 @@ class KeyedProducer(Producer):
 | 
			
		||||
    def __init__(self, client, partitioner=None, async=False,
 | 
			
		||||
                 req_acks=Producer.ACK_AFTER_LOCAL_WRITE,
 | 
			
		||||
                 ack_timeout=Producer.DEFAULT_ACK_TIMEOUT,
 | 
			
		||||
                 codec=None,
 | 
			
		||||
                 batch_send=False,
 | 
			
		||||
                 batch_send_every_n=BATCH_SEND_MSG_COUNT,
 | 
			
		||||
                 batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL):
 | 
			
		||||
@@ -250,7 +263,7 @@ class KeyedProducer(Producer):
 | 
			
		||||
        self.partitioners = {}
 | 
			
		||||
 | 
			
		||||
        super(KeyedProducer, self).__init__(client, async, req_acks,
 | 
			
		||||
                                            ack_timeout, batch_send,
 | 
			
		||||
                                            ack_timeout, codec, batch_send,
 | 
			
		||||
                                            batch_send_every_n,
 | 
			
		||||
                                            batch_send_every_t)
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -9,7 +9,8 @@ from kafka.common import (
 | 
			
		||||
    BrokerMetadata, PartitionMetadata, Message, OffsetAndMessage,
 | 
			
		||||
    ProduceResponse, FetchResponse, OffsetResponse,
 | 
			
		||||
    OffsetCommitResponse, OffsetFetchResponse, ProtocolError,
 | 
			
		||||
    BufferUnderflowError, ChecksumError, ConsumerFetchSizeTooSmall
 | 
			
		||||
    BufferUnderflowError, ChecksumError, ConsumerFetchSizeTooSmall,
 | 
			
		||||
    UnsupportedCodecError
 | 
			
		||||
)
 | 
			
		||||
from kafka.util import (
 | 
			
		||||
    read_short_string, read_int_string, relative_unpack,
 | 
			
		||||
@@ -18,6 +19,12 @@ from kafka.util import (
 | 
			
		||||
 | 
			
		||||
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):
 | 
			
		||||
    """
 | 
			
		||||
@@ -32,11 +39,6 @@ class KafkaProtocol(object):
 | 
			
		||||
    OFFSET_COMMIT_KEY = 8
 | 
			
		||||
    OFFSET_FETCH_KEY = 9
 | 
			
		||||
 | 
			
		||||
    ATTRIBUTE_CODEC_MASK = 0x03
 | 
			
		||||
    CODEC_NONE = 0x00
 | 
			
		||||
    CODEC_GZIP = 0x01
 | 
			
		||||
    CODEC_SNAPPY = 0x02
 | 
			
		||||
 | 
			
		||||
    ###################
 | 
			
		||||
    #   Private API   #
 | 
			
		||||
    ###################
 | 
			
		||||
@@ -150,17 +152,17 @@ class KafkaProtocol(object):
 | 
			
		||||
        (key, 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))
 | 
			
		||||
 | 
			
		||||
        elif codec == KafkaProtocol.CODEC_GZIP:
 | 
			
		||||
        elif codec == CODEC_GZIP:
 | 
			
		||||
            gz = gzip_decode(value)
 | 
			
		||||
            for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz):
 | 
			
		||||
                yield (offset, msg)
 | 
			
		||||
 | 
			
		||||
        elif codec == KafkaProtocol.CODEC_SNAPPY:
 | 
			
		||||
        elif codec == CODEC_SNAPPY:
 | 
			
		||||
            snp = snappy_decode(value)
 | 
			
		||||
            for (offset, msg) in KafkaProtocol._decode_message_set_iter(snp):
 | 
			
		||||
                yield (offset, msg)
 | 
			
		||||
@@ -543,7 +545,7 @@ def create_gzip_message(payloads, key=None):
 | 
			
		||||
        [create_message(payload) for payload in payloads])
 | 
			
		||||
 | 
			
		||||
    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)
 | 
			
		||||
 | 
			
		||||
@@ -564,6 +566,22 @@ def create_snappy_message(payloads, key=None):
 | 
			
		||||
        [create_message(payload) for payload in payloads])
 | 
			
		||||
 | 
			
		||||
    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)
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
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 unittest2
 | 
			
		||||
 | 
			
		||||
import mock
 | 
			
		||||
from mock import sentinel
 | 
			
		||||
 | 
			
		||||
from kafka import KafkaClient
 | 
			
		||||
from kafka.common import (
 | 
			
		||||
    OffsetRequest, OffsetCommitRequest, OffsetFetchRequest,
 | 
			
		||||
    OffsetResponse, OffsetCommitResponse, OffsetFetchResponse,
 | 
			
		||||
    ProduceRequest, FetchRequest, Message, ChecksumError,
 | 
			
		||||
    ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse,
 | 
			
		||||
    OffsetAndMessage, BrokerMetadata, PartitionMetadata,
 | 
			
		||||
    TopicAndPartition, KafkaUnavailableError, ProtocolError,
 | 
			
		||||
    LeaderUnavailableError, PartitionUnavailableError
 | 
			
		||||
    ConsumerFetchSizeTooSmall, ProduceResponse, FetchResponse, OffsetAndMessage,
 | 
			
		||||
    BrokerMetadata, PartitionMetadata, TopicAndPartition, KafkaUnavailableError,
 | 
			
		||||
    ProtocolError, LeaderUnavailableError, PartitionUnavailableError,
 | 
			
		||||
    UnsupportedCodecError
 | 
			
		||||
)
 | 
			
		||||
from kafka.codec import (
 | 
			
		||||
    has_snappy, gzip_encode, gzip_decode,
 | 
			
		||||
    snappy_encode, snappy_decode
 | 
			
		||||
)
 | 
			
		||||
import kafka.protocol
 | 
			
		||||
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):
 | 
			
		||||
@@ -33,8 +41,7 @@ class TestProtocol(unittest2.TestCase):
 | 
			
		||||
        payloads = ["v1", "v2"]
 | 
			
		||||
        msg = create_gzip_message(payloads)
 | 
			
		||||
        self.assertEqual(msg.magic, 0)
 | 
			
		||||
        self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK &
 | 
			
		||||
                                         KafkaProtocol.CODEC_GZIP)
 | 
			
		||||
        self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_GZIP)
 | 
			
		||||
        self.assertEqual(msg.key, None)
 | 
			
		||||
        # Need to decode to check since gzipped payload is non-deterministic
 | 
			
		||||
        decoded = gzip_decode(msg.value)
 | 
			
		||||
@@ -63,8 +70,7 @@ class TestProtocol(unittest2.TestCase):
 | 
			
		||||
        payloads = ["v1", "v2"]
 | 
			
		||||
        msg = create_snappy_message(payloads)
 | 
			
		||||
        self.assertEqual(msg.magic, 0)
 | 
			
		||||
        self.assertEqual(msg.attributes, KafkaProtocol.ATTRIBUTE_CODEC_MASK &
 | 
			
		||||
                                         KafkaProtocol.CODEC_SNAPPY)
 | 
			
		||||
        self.assertEqual(msg.attributes, ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY)
 | 
			
		||||
        self.assertEqual(msg.key, None)
 | 
			
		||||
        decoded = snappy_decode(msg.value)
 | 
			
		||||
        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 = 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