diff options
author | David Arthur <mumrah@gmail.com> | 2013-03-01 15:12:21 -0500 |
---|---|---|
committer | David Arthur <mumrah@gmail.com> | 2013-04-02 20:19:30 -0400 |
commit | eac51e9c68c50f15962b6c785ede92cb3d512a17 (patch) | |
tree | ec814fa65e96a8a4db1d2e6aceb319ebb23a781f | |
parent | 2a3d231aa61642c57537bc2128dd4f2bd30f35dd (diff) | |
download | kafka-python-eac51e9c68c50f15962b6c785ede92cb3d512a17.tar.gz |
Integration tests passing
m--------- | kafka-src | 0 | ||||
-rw-r--r-- | kafka/__init__.py | 3 | ||||
-rw-r--r-- | kafka/client.py | 1239 | ||||
-rw-r--r-- | kafka/client08.py | 872 | ||||
-rw-r--r-- | kafka/util.py | 4 | ||||
-rw-r--r-- | setup.py | 2 | ||||
-rw-r--r-- | test/integration.py | 23 | ||||
-rw-r--r-- | test/resources/server.properties | 2 |
8 files changed, 761 insertions, 1384 deletions
diff --git a/kafka-src b/kafka-src -Subproject e7edb5e1e933f5535378d546bcf4d8b178d2e69 +Subproject 218e6a53c1385be897d9f8a3a39baa38b68d799 diff --git a/kafka/__init__.py b/kafka/__init__.py index 1dcae86..166eeb9 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -5,8 +5,7 @@ __license__ = 'Apache License 2.0' __copyright__ = 'Copyright 2012, David Arthur under Apache License, v2.0' from .client import ( - KafkaClient, KafkaException, - Message, ProduceRequest, FetchRequest, OffsetRequest + KafkaClient ) from .codec import gzip_encode, gzip_decode from .codec import snappy_encode, snappy_decode diff --git a/kafka/client.py b/kafka/client.py index c25e2d2..5da3919 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -1,631 +1,874 @@ -from collections import namedtuple +import base64 +from collections import namedtuple, defaultdict +from functools import partial +from itertools import groupby, count import logging +from operator import attrgetter import socket import struct +import time import zlib from .codec import gzip_encode, gzip_decode from .codec import snappy_encode, snappy_decode +from .util import read_short_string, read_int_string +from .util import relative_unpack +from .util import write_short_string, write_int_string +from .util import group_by_topic_and_partition +from .util import BufferUnderflowError, ChecksumError log = logging.getLogger("kafka") -error_codes = { - -1: "UnknownError", - 0: None, - 1: "OffsetOutOfRange", - 2: "InvalidMessage", - 3: "WrongPartition", - 4: "InvalidFetchSize" -} - -class KafkaException(Exception): - def __init__(self, errorType): - self.errorType = errorType - def __str__(self): - return str(self.errorType) +############### +# Structs # +############### -Message = namedtuple("Message", ["magic", "attributes", "crc", "payload"]) -FetchRequest = namedtuple("FetchRequest", ["topic", "partition", "offset", "size"]) +# Request payloads ProduceRequest = namedtuple("ProduceRequest", ["topic", "partition", "messages"]) -OffsetRequest = namedtuple("OffsetRequest", ["topic", "partition", "time", "maxOffsets"]) - -def length_prefix_message(msg): - """ - Prefix a message with it's length as an int - """ - return struct.pack('>i', len(msg)) + msg - -class KafkaClient(object): +FetchRequest = namedtuple("FetchRequest", ["topic", "partition", "offset", "max_bytes"]) +OffsetRequest = namedtuple("OffsetRequest", ["topic", "partition", "time", "max_offsets"]) +OffsetCommitRequest = namedtuple("OffsetCommitRequest", ["topic", "partition", "offset", "metadata"]) +OffsetFetchRequest = namedtuple("OffsetFetchRequest", ["topic", "partition"]) + +# Response payloads +ProduceResponse = namedtuple("ProduceResponse", ["topic", "partition", "error", "offset"]) +FetchResponse = namedtuple("FetchResponse", ["topic", "partition", "error", "highwaterMark", "messages"]) +OffsetResponse = namedtuple("OffsetResponse", ["topic", "partition", "error", "offsets"]) +OffsetCommitResponse = namedtuple("OffsetCommitResponse", ["topic", "partition", "error"]) +OffsetFetchResponse = namedtuple("OffsetFetchResponse", ["topic", "partition", "offset", "metadata", "error"]) +BrokerMetadata = namedtuple("BrokerMetadata", ["nodeId", "host", "port"]) +PartitionMetadata = namedtuple("PartitionMetadata", ["topic", "partition", "leader", "replicas", "isr"]) + +# Other useful structs +OffsetAndMessage = namedtuple("OffsetAndMessage", ["offset", "message"]) +Message = namedtuple("Message", ["magic", "attributes", "key", "value"]) +TopicAndPartition = namedtuple("TopicAndPartition", ["topic", "partition"]) + +class ErrorMapping(object): + # Many of these are not actually used by the client + UNKNOWN = -1 + NO_ERROR = 0 + OFFSET_OUT_OF_RANGE = 1 + INVALID_MESSAGE = 2 + UNKNOWN_TOPIC_OR_PARTITON = 3 + INVALID_FETCH_SIZE = 4 + LEADER_NOT_AVAILABLE = 5 + NOT_LEADER_FOR_PARTITION = 6 + REQUEST_TIMED_OUT = 7 + BROKER_NOT_AVAILABLE = 8 + REPLICA_NOT_AVAILABLE = 9 + MESSAGE_SIZE_TO_LARGE = 10 + STALE_CONTROLLER_EPOCH = 11 + OFFSET_METADATA_TOO_LARGE = 12 + +class KafkaProtocol(object): """ - Request Structure - ================= - - <Request> ::= <len> <request-key> <payload> - <len> ::= <int32> - <request-key> ::= 0 | 1 | 2 | 3 | 4 - <payload> ::= <ProduceRequest> | <FetchRequest> | <MultiFetchRequest> | <MultiProduceRequest> | <OffsetRequest> - - Response Structure - ================== - - <Response> ::= <len> <err> <payload> - <len> ::= <int32> - <err> ::= -1 | 0 | 1 | 2 | 3 | 4 - <payload> ::= <ProduceResponse> | <FetchResponse> | <MultiFetchResponse> | <MultiProduceResponse> | <OffsetResponse> - - Messages are big-endian byte order + Class to encapsulate all of the protocol encoding/decoding. This class does not + have any state associated with it, it is purely for organization. """ - - PRODUCE_KEY = 0 - FETCH_KEY = 1 - MULTIFETCH_KEY = 2 - MULTIPRODUCE_KEY = 3 - OFFSET_KEY = 4 + PRODUCE_KEY = 0 + FETCH_KEY = 1 + OFFSET_KEY = 2 + METADATA_KEY = 3 + OFFSET_COMMIT_KEY = 6 + OFFSET_FETCH_KEY = 7 ATTRIBUTE_CODEC_MASK = 0x03 + CODEC_NONE = 0x00 + CODEC_GZIP = 0x01 + CODEC_SNAPPY = 0x02 - def __init__(self, host, port, bufsize=1024): - self.host = host - self.port = port - self.bufsize = bufsize - self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self._sock.connect((host, port)) - self._sock.settimeout(10) - log.debug("Connected to %s on %d", host, port) - - def __copy__(self): - return KafkaClient(self.host, self.port, self.bufsize) + ################### + # Private API # + ################### - ###################### - # Protocol Stuff # - ###################### - - def _consume_response_iter(self): + @classmethod + def _encode_message_header(cls, client_id, correlation_id, request_key): """ - This method handles the response header and error messages. It - then returns an iterator for the chunks of the response + Encode the common request envelope """ - log.debug("Handling response from Kafka") - # Header - resp = self._sock.recv(6) - if resp == "": - raise Exception("Got no response from Kafka") - (size, err) = struct.unpack('>iH', resp) - - log.debug("About to read %d bytes from Kafka", size-2) - # Handle error - error = error_codes.get(err) - if error is not None: - raise KafkaException(error) - - # Response iterator - total = 0 - while total < (size-2): - resp = self._sock.recv(self.bufsize) - log.debug("Read %d bytes from Kafka", len(resp)) - if resp == "": - raise Exception("Underflow") - total += len(resp) - yield resp + return struct.pack('>hhih%ds' % len(client_id), + request_key, # ApiKey + 0, # ApiVersion + correlation_id, # CorrelationId + len(client_id), # + client_id) # ClientId - def _consume_response(self): + @classmethod + def _encode_message_set(cls, messages): """ - Fully consumer the response iterator + Encode a MessageSet. Unlike other arrays in the protocol, MessageSets are + not length-prefixed + + Format + ====== + MessageSet => [Offset MessageSize Message] + Offset => int64 + MessageSize => int32 """ - data = "" - for chunk in self._consume_response_iter(): - data += chunk - return data + message_set = "" + for message in messages: + encoded_message = KafkaProtocol._encode_message(message) + message_set += struct.pack('>qi%ds' % len(encoded_message), 0, len(encoded_message), encoded_message) + return message_set @classmethod - def encode_message(cls, message): + def _encode_message(cls, message): """ - Encode a Message from a Message tuple + Encode a single message. - Params + The magic number of a message is a format version number. The only supported + magic number right now is zero + + Format ====== - message: Message - - Wire Format - =========== - <Message> ::= <Message-0> | <Message-1> - <Message-0> ::= <N> 0 <header-0> <payload> - <Message-1> ::= <N> 1 <header-1> <payload> - <N> ::= <int32> - <header-0> ::= <crc> - <header-1> ::= <attributes><crc> - <crc> ::= <int32> - <payload> ::= <bytes> - <attributes> ::= <int8> - - The crc is a crc32 checksum of the message payload. The attributes are bitmask - used for indicating the compression algorithm. + Message => Crc MagicByte Attributes Key Value + Crc => int32 + MagicByte => int8 + Attributes => int8 + Key => bytes + Value => bytes """ if message.magic == 0: - msg = struct.pack('>Bi%ds' % len(message.payload), - message.magic, message.crc, message.payload) - elif message.magic == 1: - msg = struct.pack('>BBi%ds' % len(message.payload), - message.magic, message.attributes, message.crc, message.payload) + msg = struct.pack('>BB', message.magic, message.attributes) + msg += write_int_string(message.key) + msg += write_int_string(message.value) + crc = zlib.crc32(msg) + msg = struct.pack('>i%ds' % len(msg), crc, msg) else: raise Exception("Unexpected magic number: %d" % message.magic) - msg = length_prefix_message(msg) - log.debug("Encoded %s as %r" % (message, msg)) return msg + @classmethod - def encode_message_set(cls, messages): + def _decode_message_set_iter(cls, data): """ - Encode a MessageSet + Iteratively decode a MessageSet - One or more concatenated Messages + Reads repeated elements of (offset, message), calling decode_message to decode a + single message. Since compressed messages contain futher MessageSets, these two methods + have been decoupled so that they may recurse easily. """ - message_set = "" - for message in messages: - encoded_message = cls.encode_message(message) - message_set += encoded_message - return message_set + cur = 0 + while cur < len(data): + try: + ((offset, ), cur) = relative_unpack('>q', data, cur) + (msg, cur) = read_int_string(data, cur) + for (offset, message) in KafkaProtocol._decode_message(msg, offset): + yield OffsetAndMessage(offset, message) + except BufferUnderflowError: # If we get a partial read of a message, stop + raise StopIteration() @classmethod - def encode_produce_request(cls, produceRequest): - """ - Encode a ProduceRequest - - Wire Format - =========== - <ProduceRequest> ::= <request-key> <topic> <partition> <len> <MessageSet> - <request-key> ::= 0 - <topic> ::= <topic-length><string> - <topic-length> ::= <int16> - <partition> ::= <int32> - <len> ::= <int32> - - The request-key (0) is encoded as a short (int16). len is the length of the proceeding MessageSet - """ - (topic, partition, messages) = produceRequest - message_set = cls.encode_message_set(messages) - log.debug("Sending MessageSet: %r" % message_set) - req = struct.pack('>HH%dsii%ds' % (len(topic), len(message_set)), - KafkaClient.PRODUCE_KEY, len(topic), topic, partition, len(message_set), message_set) - return req + def _decode_message(cls, data, offset): + """ + Decode a single Message + + The only caller of this method is decode_message_set_iter. They are decoupled to + support nested messages (compressed MessageSets). The offset is actually read from + decode_message_set_iter (it is part of the MessageSet payload). + """ + ((crc, magic, att), cur) = relative_unpack('>iBB', data, 0) + if crc != zlib.crc32(data[4:]): + raise ChecksumError("Message checksum failed") + + (key, cur) = read_int_string(data, cur) + (value, cur) = read_int_string(data, cur) + if att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_NONE: + yield (offset, Message(magic, att, key, value)) + elif att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_GZIP: + gz = gzip_decode(value) + for (offset, message) in KafkaProtocol._decode_message_set_iter(gz): + yield (offset, message) + elif att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_SNAPPY: + snp = snappy_decode(value) + for (offset, message) in KafkaProtocol._decode_message_set_iter(snp): + yield (offset, message) + + ################## + # Public API # + ################## @classmethod - def encode_multi_produce_request(cls, produceRequests): + def create_message(cls, payload, key=None): """ - Encode a MultiProducerRequest + Construct a Message Params ====== - produceRequest: list of ProduceRequest objects - - Returns - ======= - Encoded request - - Wire Format - =========== - <MultiProducerReqeust> ::= <request-key> <num> <ProduceRequests> - <num> ::= <int16> - <ProduceRequests> ::= <ProduceRequest> [ <ProduceRequests> ] - <ProduceRequest> ::= <topic> <partition> <len> <MessageSet> - <topic> ::= <topic-length><string> - <topic-length> ::= <int16> - <partition> ::= <int32> - <len> ::= <int32> - - num is the number of ProduceRequests being encoded - """ - req = struct.pack('>HH', KafkaClient.MULTIPRODUCE_KEY, len(produceRequests)) - for (topic, partition, messages) in produceRequests: - message_set = cls.encode_message_set(messages) - req += struct.pack('>H%dsii%ds' % (len(topic), len(message_set)), - len(topic), topic, partition, len(message_set), message_set) - return req - - @classmethod - def encode_fetch_request(cls, fetchRequest): - """ - Encode a FetchRequest message - - Wire Format - =========== - <FetchRequest> ::= <request-key> <topic> <partition> <offset> <size> - <request-key> ::= 1 - <topic> ::= <topic-length><string> - <topic-length> ::= <int16> - <partition> ::= <int32> - <offset> ::= <int64> - <size> ::= <int32> - - The request-key (1) is encoded as a short (int16). - """ - (topic, partition, offset, size) = fetchRequest - req = struct.pack('>HH%dsiqi' % len(topic), - KafkaClient.FETCH_KEY, len(topic), topic, partition, offset, size) - return req + payload: bytes, the payload to send to Kafka + key: bytes, a key used for partition routing (optional) + """ + return Message(0, 0, key, payload) @classmethod - def encode_multi_fetch_request(cls, fetchRequests): + def create_gzip_message(cls, payloads, key=None): """ - Encode the MultiFetchRequest message from a list of FetchRequest objects + Construct a Gzipped Message containing multiple Messages + + The given payloads will be encoded, compressed, and sent as a single atomic + message to Kafka. Params ====== - fetchRequests: list of FetchRequest - - Returns - ======= - req: bytes, The message to send to Kafka - - Wire Format - =========== - <MultiFetchRequest> ::= <request-key> <num> [ <FetchRequests> ] - <request-key> ::= 2 - <num> ::= <int16> - <FetchRequests> ::= <FetchRequest> [ <FetchRequests> ] - <FetchRequest> ::= <topic> <partition> <offset> <size> - <topic> ::= <topic-length><string> - <topic-length> ::= <int16> - <partition> ::= <int32> - <offset> ::= <int64> - <size> ::= <int32> - - The request-key (2) is encoded as a short (int16). - """ - req = struct.pack('>HH', KafkaClient.MULTIFETCH_KEY, len(fetchRequests)) - for (topic, partition, offset, size) in fetchRequests: - req += struct.pack('>H%dsiqi' % len(topic), len(topic), topic, partition, offset, size) - return req + payloads: list(bytes), a list of payload to send be sent to Kafka + key: bytes, a key used for partition routing (optional) + """ + message_set = KafkaProtocol._encode_message_set( + [KafkaProtocol.create_message(payload) for payload in payloads]) + gzipped = gzip_encode(message_set) + return Message(0, 0x00 | (KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_GZIP), key, gzipped) @classmethod - def encode_offset_request(cls, offsetRequest): + def create_snappy_message(cls, payloads, key=None): """ - Encode an OffsetRequest message + Construct a Snappy Message containing multiple Messages - Wire Format - =========== - <OffsetRequest> ::= <request-key> <topic> <partition> <time> <max-offsets> - <request-key> ::= 4 - <topic> ::= <topic-length><string> - <topic-length> ::= <int16> - <partition> ::= <int32> - <time> ::= <epoch> - <epoch> ::= <int64> - <max-offsets> ::= <int32> + The given payloads will be encoded, compressed, and sent as a single atomic + message to Kafka. - The request-key (4) is encoded as a short (int16). + Params + ====== + payloads: list(bytes), a list of payload to send be sent to Kafka + key: bytes, a key used for partition routing (optional) """ - (topic, partition, offset, maxOffsets) = offsetRequest - req = struct.pack('>HH%dsiqi' % len(topic), KafkaClient.OFFSET_KEY, len(topic), topic, partition, offset, maxOffsets) - return req + message_set = KafkaProtocol._encode_message_set( + [KafkaProtocol.create_message(payload) for payload in payloads]) + snapped = snappy_encode(message_set) + return Message(0, 0x00 | (KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_SNAPPY), key, snapped) @classmethod - def decode_message(cls, data): + def encode_produce_request(cls, client_id, correlation_id, payloads=[], acks=1, timeout=1000): """ - Decode a Message - - Verify crc and decode the Message. A compressed Message's payload is actually - an encoded MessageSet. This allows Messages to be nested within Messages and - as such, this method will recurse. + Encode some ProduceRequest structs Params ====== - data, bytes - - Returns - ======= - Generator of Messages (depth-first) - """ - N = len(data) - (magic,) = struct.unpack('>B', data[0:1]) - if magic == 0: - # version 0 - (crc,) = struct.unpack('>i', data[1:5]) - payload = data[5:N] - assert zlib.crc32(payload) == crc - msg = Message(magic, None, crc, payload) - log.debug("Got v0 Message, %s", msg) - yield msg - elif magic == 1: - # version 1 - (att, crc) = struct.unpack('>Bi', data[1:6]) - payload = data[6:N] - assert zlib.crc32(payload) == crc - if att & KafkaClient.ATTRIBUTE_CODEC_MASK == 0: - # Uncompressed, just a single Message - msg = Message(magic, att, crc, payload) - log.debug("Got v1 Message, %s", msg) - yield msg - elif att & KafkaClient.ATTRIBUTE_CODEC_MASK == 1: - # Gzip encoded Message - gz = gzip_decode(payload) - (msgs, _) = cls.read_message_set(gz) - for msg in msgs: - yield msg - elif att & KafkaClient.ATTRIBUTE_CODEC_MASK == 2: - # Snappy encoded Message - snp = snappy_decode(payload) - (msgs, _) = cls.read_message_set(snp) - for msg in msgs: - yield msg - else: - raise RuntimeError("Unsupported compression type: %d" % (att & KafkaClient.ATTRIBUTE_CODEC_MASK)) + client_id: string + correlation_id: string + payloads: list of ProduceRequest + acks: How "acky" you want the request to be + 0: immediate response + 1: written to disk by the leader + 2+: waits for this many number of replicas to sync + -1: waits for all replicas to be in sync + timeout: Maximum time the server will wait for acks from replicas. This is _not_ a socket timeout + """ + grouped_payloads = group_by_topic_and_partition(payloads) + message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.PRODUCE_KEY) + message += struct.pack('>hii', acks, timeout, len(grouped_payloads)) + for topic, topic_payloads in grouped_payloads.items(): + message += struct.pack('>h%dsi' % len(topic), len(topic), topic, len(topic_payloads)) + for partition, payload in topic_payloads.items(): + message_set = KafkaProtocol._encode_message_set(payload.messages) + message += struct.pack('>ii%ds' % len(message_set), partition, len(message_set), message_set) + return struct.pack('>i%ds' % len(message), len(message), message) @classmethod - def read_message_set(cls, data): + def decode_produce_response(cls, data): """ - Read a MessageSet + Decode bytes to a ProduceResponse - This method reads through the given bytes and decodes the length-prefixed Messages. It will short - circuit if there are insufficient bytes to read a whole Message. This allows callers to determine - the next valid offset - Params ====== - data: bytes to read - - Returns - ======= - tuple of (list(Message), read), where read is how many bytes were read - - Wire Format - =========== - <MessageSet> ::= <len> <Message> [ <MessageSet> ] - <len> ::= <int32> - - len is the length of the proceeding Message - """ - - # Read the MessageSet - cur = 0 - msgs = [] - size = len(data) - while cur < size: - # Make sure we can read this Message - if (cur + 4) > size: - # Not enough bytes to read the length - if len(msgs) == 0: - raise Exception("Message underflow. Did not request enough bytes to consume a single message") - else: - log.debug("Not enough data to read header of next message") - break - (N,) = struct.unpack('>i', data[cur:(cur+4)]) - if (cur + N + 4) > size: - # Not enough bytes to read this whole Message - log.debug("Not enough data to read next message") - break - else: - cur += 4 - - # Decode the message(s) - for m in cls.decode_message(data[cur:cur+N]): - msgs.append(m) - - # Advance the cursor - cur += N - - # Return the retrieved messages and the cursor position - return (msgs, cur) - - ######################### - # Advanced User API # - ######################### + data: bytes to decode + """ + ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + for i in range(num_topics): + ((strlen,), cur) = relative_unpack('>h', data, cur) + topic = data[cur:cur+strlen] + cur += strlen + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + for i in range(num_partitions): + ((partition, error, offset), cur) = relative_unpack('>ihq', data, cur) + yield ProduceResponse(topic, partition, error, offset) @classmethod - def create_message(cls, payload): + def encode_fetch_request(cls, client_id, correlation_id, payloads=[], max_wait_time=100, min_bytes=4096): """ - Create a standard Message + Encodes some FetchRequest structs Params ====== - payload, bytes - - Returns - ======= - A Message tuple + client_id: string + correlation_id: string + payloads: list of FetchRequest + max_wait_time: int, how long to block waiting on min_bytes of data + min_bytes: int, the minimum number of bytes to accumulate before returning the response """ - return Message(1, 0, zlib.crc32(payload), payload) + + grouped_payloads = group_by_topic_and_partition(payloads) + message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.FETCH_KEY) + message += struct.pack('>iiii', -1, max_wait_time, min_bytes, len(grouped_payloads)) # -1 is the replica id + for topic, topic_payloads in grouped_payloads.items(): + message += write_short_string(topic) + message += struct.pack('>i', len(topic_payloads)) + for partition, payload in topic_payloads.items(): + message += struct.pack('>iqi', partition, payload.offset, payload.max_bytes) + return struct.pack('>i%ds' % len(message), len(message), message) @classmethod - def create_gzip_message(cls, *payloads): + def decode_fetch_response_iter(cls, data): """ - Create a Gzip encoded Message + Decode bytes to a FetchResponse Params ====== - payloads, list of messages (bytes) to be encoded + data: bytes to decode + """ + ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + for i in range(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + for i in range(num_partitions): + ((partition, error, highwater_mark_offset), cur) = relative_unpack('>ihq', data, cur) + (message_set, cur) = read_int_string(data, cur) + yield FetchResponse(topic, partition, error, highwater_mark_offset, + KafkaProtocol._decode_message_set_iter(message_set)) - Returns - ======= - A Message tuple - """ - messages = [cls.create_message(payload) for payload in payloads] - message_set = cls.encode_message_set(messages) - gzipped = gzip_encode(message_set) - return Message(1, 0x00 | (KafkaClient.ATTRIBUTE_CODEC_MASK & 0x01), zlib.crc32(gzipped), gzipped) + @classmethod + def encode_offset_request(cls, client_id, correlation_id, payloads=[]): + grouped_payloads = group_by_topic_and_partition(payloads) + message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.OFFSET_KEY) + message += struct.pack('>ii', -1, len(grouped_payloads)) # -1 is the replica id + for topic, topic_payloads in grouped_payloads.items(): + message += write_short_string(topic) + message += struct.pack('>i', len(topic_payloads)) + for partition, payload in topic_payloads.items(): + message += struct.pack('>iqi', partition, payload.time, payload.max_offsets) + return struct.pack('>i%ds' % len(message), len(message), message) @classmethod - def create_snappy_message(cls, *payloads): + def decode_offset_response(cls, data): """ - Create a Snappy encoded Message + Decode bytes to an OffsetResponse Params ====== - payloads, list of messages (bytes) to be encoded + data: bytes to decode + """ + ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + for i in range(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + for i in range(num_partitions): + ((partition, error, num_offsets,), cur) = relative_unpack('>ihi', data, cur) + offsets = [] + for j in range(num_offsets): + ((offset,), cur) = relative_unpack('>q', data, cur) + offsets.append(offset) + yield OffsetResponse(topic, partition, error, tuple(offsets)) - Returns - ======= - A Message tuple + @classmethod + def encode_metadata_request(cls, client_id, correlation_id, topics=[]): """ - messages = [cls.create_message(payload) for payload in payloads] - message_set = cls.encode_message_set(messages) - snapped = snappy_encode(message_set) - return Message(1, 0x00 | (KafkaClient.ATTRIBUTE_CODEC_MASK & 0x02), zlib.crc32(snapped), snapped) + Encode a MetadataRequest - def send_message_set(self, produceRequest): - """ - Send a ProduceRequest - Params ====== - produceRequest: ProduceRequest + client_id: string + correlation_id: string + topics: list of strings """ - req = length_prefix_message(self.encode_produce_request(produceRequest)) - log.debug("Sending %d bytes to Kafka: %r", len(req), req) - sent = self._sock.sendall(req) - if sent == 0: - raise RuntimeError("Kafka went away") + message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.METADATA_KEY) + message += struct.pack('>i', len(topics)) + for topic in topics: + message += struct.pack('>h%ds' % len(topic), len(topic), topic) + return write_int_string(message) - def send_multi_message_set(self, produceRequests): + @classmethod + def decode_metadata_response(cls, data): """ - Send a MultiProduceRequest - + Decode bytes to a MetadataResponse + Params ====== - produceRequests: list of ProduceRequest - """ - req = length_prefix_message(self.encode_multi_produce_request(produceRequests)) - log.debug("Sending %d bytes to Kafka", len(req)) - sent = self._sock.sendall(req) - if sent == 0: - raise RuntimeError("Kafka went away") + data: bytes to decode + """ + ((correlation_id, numBrokers), cur) = relative_unpack('>ii', data, 0) + + # Broker info + brokers = {} + for i in range(numBrokers): + ((nodeId, ), cur) = relative_unpack('>i', data, cur) + (host, cur) = read_short_string(data, cur) + ((port,), cur) = relative_unpack('>i', data, cur) + brokers[nodeId] = BrokerMetadata(nodeId, host, port) + + # Topic info + ((num_topics,), cur) = relative_unpack('>i', data, cur) + topicMetadata = {} + for i in range(num_topics): + ((topicError,), cur) = relative_unpack('>h', data, cur) + (topicName, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + partitionMetadata = {} + for j in range(num_partitions): + ((partitionErrorCode, partition, leader, numReplicas), cur) = relative_unpack('>hiii', data, cur) + (replicas, cur) = relative_unpack('>%di' % numReplicas, data, cur) + ((numIsr,), cur) = relative_unpack('>i', data, cur) + (isr, cur) = relative_unpack('>%di' % numIsr, data, cur) + partitionMetadata[partition] = PartitionMetadata(topicName, partition, leader, replicas, isr) + topicMetadata[topicName] = partitionMetadata + return (brokers, topicMetadata) - def get_message_set(self, fetchRequest): + @classmethod + def encode_offset_commit_request(cls, client_id, correlation_id, group, payloads): """ - Send a FetchRequest and return the Messages + Encode some OffsetCommitRequest structs Params ====== - fetchRequest: FetchRequest named tuple + client_id: string + correlation_id: string + group: string, the consumer group you are committing offsets for + payloads: list of OffsetCommitRequest + """ + grouped_payloads= group_by_topic_and_partition(payloads) + message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.OFFSET_COMMIT_KEY) + message += write_short_string(group) + message += struct.pack('>i', len(grouped_payloads)) + for topic, topic_payloads in grouped_payloads.items(): + message += write_short_string(topic) + message += struct.pack('>i', len(topic_payloads)) + for partition, payload in topic_payloads.items(): + message += struct.pack('>iq', partition, payload.offset) + message += write_short_string(payload.metadata) + return struct.pack('>i%ds' % len(message), len(message), message) - Returns - ======= - A tuple of (list(Message), FetchRequest). This FetchRequest will have the offset - starting at the next message. + @classmethod + def decode_offset_commit_response(cls, data): """ + Decode bytes to an OffsetCommitResponse - req = length_prefix_message(self.encode_fetch_request(fetchRequest)) - log.debug("Sending %d bytes to Kafka", len(req)) - sent = self._sock.sendall(req) - if sent == 0: - raise RuntimeError("Kafka went away") - data = self._consume_response() - (messages, read) = self.read_message_set(data) - - # Return the retrieved messages and the next FetchRequest - return (messages, FetchRequest(fetchRequest.topic, fetchRequest.partition, (fetchRequest.offset + read), fetchRequest.size)) + Params + ====== + data: bytes to decode + """ + data = data[2:] # TODO remove me when versionId is removed + ((correlation_id,), cur) = relative_unpack('>i', data, 0) + (client_id, cur) = read_short_string(data, cur) + ((num_topics,), cur) = relative_unpack('>i', data, cur) + for i in xrange(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + for i in xrange(num_partitions): + ((partition, error), cur) = relative_unpack('>ih', data, cur) + yield OffsetCommitResponse(topic, partition, error) - def get_multi_message_set(self, fetchRequests): + @classmethod + def encode_offset_fetch_request(cls, client_id, correlation_id, group, payloads): """ - Send several FetchRequests in a single pipelined request. + Encode some OffsetFetchRequest structs Params ====== - fetchRequests: list of FetchRequest - - Returns - ======= - list of tuples of (list(Message), FetchRequest). This FetchRequest will have the offset - starting at the next message. - - Wire Format - =========== - <MultiFetchResponse> ::= <MultiMessageSet> - <MultiMessageSet> ::= <MultiMessage> [ <MultiMessageSet> ] - <MultiMessage> ::= <len> 0 <MessageSet> - <len> ::= <int32> - """ - req = length_prefix_message(self.encode_multi_fetch_request(fetchRequests)) - log.debug("Sending %d bytes to Kafka", len(req)) - sent = self._sock.sendall(req) - if sent == 0: - raise RuntimeError("Kafka went away") - data = self._consume_response() - cur = 0 - responses = [] - for request in fetchRequests: - (size, _) = struct.unpack('>iH', data[cur:(cur+6)]) - cur += 6 - (messages, read) = self.read_message_set(data[cur:(cur+size-2)]) - cur += size-2 - responses.append((messages, FetchRequest(request.topic, request.partition, request.offset+read, request.size))) - return responses + client_id: string + correlation_id: string + group: string, the consumer group you are fetching offsets for + payloads: list of OffsetFetchRequest + """ + grouped_payloads = group_by_topic_and_partition(payloads) + message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.OFFSET_FETCH_KEY) + message += write_short_string(group) + message += struct.pack('>i', len(grouped_payloads)) + for topic, topic_payloads in grouped_payloads.items(): + message += write_short_string(topic) + message += struct.pack('>i', len(topic_payloads)) + for partition, payload in topic_payloads.items(): + message += struct.pack('>i', partition) + return struct.pack('>i%ds' % len(message), len(message), message) - def get_offsets(self, offsetRequest): + @classmethod + def decode_offset_fetch_response(cls, data): """ - Get the offsets for a topic + Decode bytes to an OffsetFetchResponse Params ====== - offsetRequest: OffsetRequest - - Returns - ======= - offsets: tuple of offsets - - Wire Format - =========== - <OffsetResponse> ::= <num> [ <offsets> ] - <num> ::= <int32> - <offsets> ::= <offset> [ <offsets> ] - <offset> ::= <int64> + data: bytes to decode + """ + + data = data[2:] # TODO remove me when versionId is removed + ((correlation_id,), cur) = relative_unpack('>i', data, 0) + (client_id, cur) = read_short_string(data, cur) + ((num_topics,), cur) = relative_unpack('>i', data, cur) + for i in range(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + for i in range(num_partitions): + ((partition, offset), cur) = relative_unpack('>iq', data, cur) + (metadata, cur) = read_short_string(data, cur) + ((error,), cur) = relative_unpack('>h', data, cur) + yield OffsetFetchResponse(topic, partition, offset, metadata, error) + + +class KafkaConnection(object): + """ + A socket connection to a single Kafka broker + + This class is _not_ thread safe. Each call to `send` must be followed + by a call to `recv` in order to get the correct response. Eventually, + we can do something in here to facilitate multiplexed requests/responses + since the Kafka API includes a correlation id. + """ + def __init__(self, host, port, bufsize=4096): + self.host = host + self.port = port + self.bufsize = bufsize + self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + self._sock.connect((host, port)) + self._sock.settimeout(10) + + def __str__(self): + return "<KafkaConnection host=%s port=%d>" % (self.host, self.port) + + ################### + # Private API # + ################### + + def _consume_response(self): + """ + Fully consumer the response iterator + """ + data = "" + for chunk in self._consume_response_iter(): + data += chunk + return data + def _consume_response_iter(self): + """ + This method handles the response header and error messages. It + then returns an iterator for the chunks of the response """ - req = length_prefix_message(self.encode_offset_request(offsetRequest)) - log.debug("Sending OffsetRequest of %d bytes to Kafka", len(req)) - sent = self._sock.sendall(req) + log.debug("Handling response from Kafka") + + # Read the size off of the header + resp = self._sock.recv(4) + if resp == "": + raise Exception("Got no response from Kafka") + (size,) = struct.unpack('>i', resp) + + messageSize = size - 4 + log.debug("About to read %d bytes from Kafka", messageSize) + + # Read the remainder of the response + total = 0 + while total < messageSize: + resp = self._sock.recv(self.bufsize) + log.debug("Read %d bytes from Kafka", len(resp)) + if resp == "": + raise BufferUnderflowError("Not enough data to read this response") + total += len(resp) + yield resp + + ################## + # Public API # + ################## + + # TODO multiplex socket communication to allow for multi-threaded clients + + def send(self, requestId, payload): + "Send a request to Kafka" + sent = self._sock.sendall(payload) if sent == 0: raise RuntimeError("Kafka went away") + self.data = self._consume_response() + + def recv(self, requestId): + "Get a response from Kafka" + return self.data + + def close(self): + "Close this connection" + self._sock.close() + +class KafkaClient(object): + + CLIENT_ID = "kafka-python" + ID_GEN = count() + + def __init__(self, host, port, bufsize=4096): + # We need one connection to bootstrap + self.bufsize = bufsize + self.conns = { # (host, port) -> KafkaConnection + (host, port): KafkaConnection(host, port, bufsize) + } + self.brokers = {} # broker_id -> BrokerMetadata + self.topics_to_brokers = {} # topic_id -> broker_id + self.load_metadata_for_topics() - data = self._consume_response() - (num,) = struct.unpack('>i', data[0:4]) - offsets = struct.unpack('>%dq' % num, data[4:]) - return offsets + def close(self): + for conn in self.conns.values(): + conn.close() + + def get_conn_for_broker(self, broker): + "Get or create a connection to a broker" + if (broker.host, broker.port) not in self.conns: + self.conns[(broker.host, broker.port)] = KafkaConnection(broker.host, broker.port, self.bufsize) + return self.conns[(broker.host, broker.port)] + + def next_id(self): + "Generate a new correlation id" + return KafkaClient.ID_GEN.next() + + def load_metadata_for_topics(self, *topics): + """ + Discover brokers and metadata for a set of topics. This method will + recurse in the event of a retry. + """ + requestId = self.next_id() + request = KafkaProtocol.encode_metadata_request(KafkaClient.CLIENT_ID, requestId, topics) + response = self.try_send_request(requestId, request) + if response is None: + raise Exception("All servers failed to process request") + (brokers, topics) = KafkaProtocol.decode_metadata_response(response) + log.debug("Broker metadata: %s", brokers) + log.debug("Topic metadata: %s", topics) + self.brokers.update(brokers) + self.topics_to_brokers = {} + for topic, partitions in topics.items(): + for partition, meta in partitions.items(): + if meta.leader == -1: + log.info("Partition is unassigned, delay for 1s and retry") + time.sleep(1) + self.load_metadata_for_topics(topic) + else: + self.topics_to_brokers[TopicAndPartition(topic, partition)] = brokers[meta.leader] - ####################### - # Simple User API # - ####################### + def get_leader_for_partition(self, topic, partition): + key = TopicAndPartition(topic, partition) + if key not in self.topics_to_brokers: + self.load_metadata_for_topics(topic) + if key not in self.topics_to_brokers: + raise Exception("Partition does not exist: %s" % str(key)) + return self.topics_to_brokers[key] - def send_messages_simple(self, topic, *payloads): + def send_produce_request(self, payloads=[], fail_on_error=True, callback=None): """ - Send one or more strings to Kafka + Encode and send some ProduceRequests + + ProduceRequests will be grouped by (topic, partition) and then sent to a specific + broker. Output is a list of responses in the same order as the list of payloads + specified Params ====== - topic: string - payloads: strings - """ - messages = tuple([self.create_message(payload) for payload in payloads]) - self.send_message_set(ProduceRequest(topic, -1, messages)) + payloads: list of ProduceRequest + fail_on_error: boolean, should we raise an Exception if we encounter an API error? + callback: function, instead of returning the ProduceResponse, first pass it through this function - def iter_messages(self, topic, partition, offset, size, auto=True): + Return + ====== + list of ProduceResponse or callback(ProduceResponse), in the order of input payloads + """ + # Group the produce requests by which broker they go to + original_keys = [] + payloads_by_broker = defaultdict(list) + for payload in payloads: + payloads_by_broker[self.get_leader_for_partition(payload.topic, payload.partition)] += payloads + original_keys.append((payload.topic, payload.partition)) + + # Accumulate the responses in a dictionary + acc = {} + + # For each broker, send the list of request payloads + for broker, payloads in payloads_by_broker.items(): + conn = self.get_conn_for_broker(broker) + requestId = self.next_id() + request = KafkaProtocol.encode_produce_request(KafkaClient.CLIENT_ID, requestId, payloads) + # Send the request + conn.send(requestId, request) + response = conn.recv(requestId) + for produce_response in KafkaProtocol.decode_produce_response(response): + # Check for errors + if fail_on_error == True and produce_response.error != 0: + raise Exception("ProduceRequest for %s failed with errorcode=%d", + (TopicAndPartition(produce_response.topic, produce_response.partition), produce_response.error)) + # Run the callback + if callback is not None: + acc[(produce_response.topic, produce_response.partition)] = callback(produce_response) + else: + acc[(produce_response.topic, produce_response.partition)] = produce_response + + # Order the accumulated responses by the original key order + return (acc[k] for k in original_keys) + + def send_fetch_request(self, payloads=[], fail_on_error=True, callback=None): """ - Helper method that iterates through all messages starting at the offset - in the given FetchRequest + Encode and send a FetchRequest + + Payloads are grouped by topic and partition so they can be pipelined to the same + brokers. + """ + # Group the produce requests by which broker they go to + original_keys = [] + payloads_by_broker = defaultdict(list) + for payload in payloads: + payloads_by_broker[self.get_leader_for_partition(payload.topic, payload.partition)].append(payload) + original_keys.append((payload.topic, payload.partition)) + + # Accumulate the responses in a dictionary, keyed by topic+partition + acc = {} + + # For each broker, send the list of request payloads + for broker, payloads in payloads_by_broker.items(): + conn = self.get_conn_for_broker(broker) + requestId = self.next_id() + request = KafkaProtocol.encode_fetch_request(KafkaClient.CLIENT_ID, requestId, payloads) + # Send the request + conn.send(requestId, request) + response = conn.recv(requestId) + for fetch_response in KafkaProtocol.decode_fetch_response_iter(response): + # Check for errors + if fail_on_error == True and fetch_response.error != 0: + raise Exception("FetchRequest %s failed with errorcode=%d" % + (TopicAndPartition(fetch_response.topic, fetch_response.partition), fetch_response.error)) + # Run the callback + if callback is not None: + acc[(fetch_response.topic, fetch_response.partition)] = callback(fetch_response) + else: + acc[(fetch_response.topic, fetch_response.partition)] = fetch_response + + # Order the accumulated responses by the original key order + return (acc[k] for k in original_keys) + + def try_send_request(self, requestId, request): + """ + Attempt to send a broker-agnostic request to one of the available brokers. + Keep trying until you succeed. + """ + for conn in self.conns.values(): + try: + conn.send(requestId, request) + response = conn.recv(requestId) + return response + except Exception, e: + log.warning("Could not send request [%r] to server %s, trying next server: %s" % (request, conn, e)) + continue + return None + + def send_offset_request(self, payloads=[], fail_on_error=True, callback=None): + requestId = self.next_id() + request = KafkaProtocol.encode_offset_request(KafkaClient.CLIENT_ID, requestId, payloads) + response = self.try_send_request(requestId, request) + if response is None: + if fail_on_error is True: + raise Exception("All servers failed to process request") + else: + return None + out = [] + for offset_response in KafkaProtocol.decode_offset_response(response): + if fail_on_error == True and offset_response.error != 0: + raise Exception("OffsetRequest failed with errorcode=%s", offset_response.error) + if callback is not None: + out.append(callback(offset_response)) + else: + out.append(offset_response) + return out + + def send_offset_commit_request(self, group, payloads=[], fail_on_error=True, callback=None): + requestId = self.next_id() + request = KafkaProtocol.encode_offset_commit_request(KafkaClient.CLIENT_ID, requestId, group, payloads) + response = self.try_send_request(requestId, request) + if response is None: + if fail_on_error is True: + raise Exception("All servers failed to process request") + else: + return None + out = [] + for offset_commit_response in KafkaProtocol.decode_offset_commit_response(response): + log.debug(offset_commit_response) + if fail_on_error == True and offset_commit_response.error != 0: + raise Exception("OffsetCommitRequest failed with errorcode=%s", offset_commit_response.error) + if callback is not None: + out.append(callback(offset_commit_response)) + else: + out.append(offset_commit_response) + return out + + def send_offset_fetch_request(self, group, payloads=[], fail_on_error=True, callback=None): + requestId = self.next_id() + request = KafkaProtocol.encode_offset_fetch_request(KafkaClient.CLIENT_ID, requestId, group, payloads) + response = self.try_send_request(requestId, request) + if response is None: + if fail_on_error is True: + raise Exception("All servers failed to process request") + else: + return None + out = [] + for offset_fetch_response in KafkaProtocol.decode_offset_fetch_response(response): + if fail_on_error == True and offset_fetch_response.error != 0: + raise Exception("OffsetFetchRequest for topic=%s, partition=%d failed with errorcode=%s" % ( + offset_fetch_response.topic, offset_fetch_response.partition, offset_fetch_response.error)) + if callback is not None: + out.append(callback(offset_fetch_response)) + else: + out.append(offset_fetch_response) + return out - Params - ====== - topic: string - partition: int - offset: int, offset to start consuming from - size: number of bytes to initially fetch - auto: boolean, indicates whether or not to automatically make the next - FetchRequest for more messages - - Returns - ======= - A generator of Messages - """ - fetchRequest = FetchRequest(topic, partition, offset, size) - while True: - lastOffset = fetchRequest.offset - (messages, fetchRequest) = self.get_message_set(fetchRequest) - if fetchRequest.offset == lastOffset: - break - for message in messages: - yield message - if auto == False: - break - def close(self): - self._sock.close() +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + + topic = "foo8" + # Bootstrap connection + conn = KafkaClient("localhost", 49720) + + # Create some Messages + messages = (KafkaProtocol.create_gzip_message(["GZIPPed"]), + KafkaProtocol.create_message("not-gzipped")) + + produce1 = ProduceRequest(topic=topic, partition=0, messages=messages) + produce2 = ProduceRequest(topic=topic, partition=1, messages=messages) + + # Send the ProduceRequest + produce_resp = conn.send_produce_request(payloads=[produce1, produce2]) + + # Check for errors + for resp in produce_resp: + if resp.error != 0: + raise Exception("ProduceRequest failed with errorcode=%d", resp.error) + print resp + + # Offset commit/fetch + #conn.send_offset_commit_request(group="group", payloads=[OffsetCommitRequest("topic-1", 0, 42, "METADATA?")]) + #conn.send_offset_fetch_request(group="group", payloads=[OffsetFetchRequest("topic-1", 0)]) + + def init_offsets(offset_response): + if offset_response.error not in (ErrorMapping.NO_ERROR, ErrorMapping.UNKNOWN_TOPIC_OR_PARTITON): + raise Exception("OffsetFetch failed: %s" % (offset_response)) + elif offset_response.error == ErrorMapping.UNKNOWN_TOPIC_OR_PARTITON: + return 0 + else: + return offset_response.offset + # Load offsets + (offset1, offset2) = conn.send_offset_fetch_request( + group="group1", + payloads=[OffsetFetchRequest(topic, 0),OffsetFetchRequest(topic, 1)], + fail_on_error=False, + callback=init_offsets + ) + print offset1, offset2 + + while True: + for resp in conn.send_fetch_request(payloads=[FetchRequest(topic=topic, partition=0, offset=offset1, max_bytes=4096)]): + i = 0 + for msg in resp.messages: + print msg + offset1 = msg.offset+1 + print offset1, conn.send_offset_commit_request(group="group1", payloads=[OffsetCommitRequest(topic, 0, offset1, "")]) + i += 1 + if i == 0: + raise StopIteration("no more messages") + + for resp in conn.send_fetch_request(payloads=[FetchRequest(topic=topic, partition=1, offset=offset2, max_bytes=4096)]): + i = 0 + for msg in resp.messages: + print msg + offset2 = msg.offset+1 + print offset2, conn.send_offset_commit_request(group="group1", payloads=[OffsetCommitRequest(topic, 1, offset2, "")]) + i += 1 + if i == 0: + raise StopIteration("no more messages") + diff --git a/kafka/client08.py b/kafka/client08.py deleted file mode 100644 index 49d786f..0000000 --- a/kafka/client08.py +++ /dev/null @@ -1,872 +0,0 @@ -import base64 -from collections import namedtuple, defaultdict -from functools import partial -from itertools import groupby, count -import logging -from operator import attrgetter -import socket -import struct -import time -import zlib - -from .codec import gzip_encode, gzip_decode -from .codec import snappy_encode, snappy_decode -from .util import read_short_string, read_int_string -from .util import relative_unpack -from .util import write_short_string, write_int_string -from .util import group_by_topic_and_partition -from .util import BufferUnderflowError, ChecksumError - -log = logging.getLogger("kafka") - -############### -# Structs # -############### - -# Request payloads -ProduceRequest = namedtuple("ProduceRequest", ["topic", "partition", "messages"]) -FetchRequest = namedtuple("FetchRequest", ["topic", "partition", "offset", "max_bytes"]) -OffsetRequest = namedtuple("OffsetRequest", ["topic", "partition", "time", "max_offsets"]) -OffsetCommitRequest = namedtuple("OffsetCommitRequest", ["topic", "partition", "offset", "metadata"]) -OffsetFetchRequest = namedtuple("OffsetFetchRequest", ["topic", "partition"]) - -# Response payloads -ProduceResponse = namedtuple("ProduceResponse", ["topic", "partition", "error", "offset"]) -FetchResponse = namedtuple("FetchResponse", ["topic", "partition", "error", "highwaterMark", "messages"]) -OffsetResponse = namedtuple("OffsetResponse", ["topic", "partition", "error", "offsets"]) -OffsetCommitResponse = namedtuple("OffsetCommitResponse", ["topic", "partition", "error"]) -OffsetFetchResponse = namedtuple("OffsetFetchResponse", ["topic", "partition", "offset", "metadata", "error"]) -BrokerMetadata = namedtuple("BrokerMetadata", ["nodeId", "host", "port"]) -PartitionMetadata = namedtuple("PartitionMetadata", ["topic", "partition", "leader", "replicas", "isr"]) - -# Other useful structs -OffsetAndMessage = namedtuple("OffsetAndMessage", ["offset", "message"]) -Message = namedtuple("Message", ["magic", "attributes", "key", "value"]) -TopicAndPartition = namedtuple("TopicAndPartition", ["topic", "partition"]) - -class ErrorMapping(object): - # Many of these are not actually used by the client - UNKNOWN = -1 - NO_ERROR = 0 - OFFSET_OUT_OF_RANGE = 1 - INVALID_MESSAGE = 2 - UNKNOWN_TOPIC_OR_PARTITON = 3 - INVALID_FETCH_SIZE = 4 - LEADER_NOT_AVAILABLE = 5 - NOT_LEADER_FOR_PARTITION = 6 - REQUEST_TIMED_OUT = 7 - BROKER_NOT_AVAILABLE = 8 - REPLICA_NOT_AVAILABLE = 9 - MESSAGE_SIZE_TO_LARGE = 10 - STALE_CONTROLLER_EPOCH = 11 - OFFSET_METADATA_TOO_LARGE = 12 - -class KafkaProtocol(object): - """ - Class to encapsulate all of the protocol encoding/decoding. This class does not - have any state associated with it, it is purely for organization. - """ - PRODUCE_KEY = 0 - FETCH_KEY = 1 - OFFSET_KEY = 2 - METADATA_KEY = 3 - OFFSET_COMMIT_KEY = 6 - OFFSET_FETCH_KEY = 7 - - ATTRIBUTE_CODEC_MASK = 0x03 - CODEC_NONE = 0x00 - CODEC_GZIP = 0x01 - CODEC_SNAPPY = 0x02 - - ################### - # Private API # - ################### - - @classmethod - def _encode_message_header(cls, client_id, correlation_id, request_key): - """ - Encode the common request envelope - """ - return struct.pack('>hhih%ds' % len(client_id), - request_key, # ApiKey - 0, # ApiVersion - correlation_id, # CorrelationId - len(client_id), # - client_id) # ClientId - - @classmethod - def _encode_message_set(cls, messages): - """ - Encode a MessageSet. Unlike other arrays in the protocol, MessageSets are - not length-prefixed - - Format - ====== - MessageSet => [Offset MessageSize Message] - Offset => int64 - MessageSize => int32 - """ - message_set = "" - for message in messages: - encoded_message = KafkaProtocol._encode_message(message) - message_set += struct.pack('>qi%ds' % len(encoded_message), 0, len(encoded_message), encoded_message) - return message_set - - @classmethod - def _encode_message(cls, message): - """ - Encode a single message. - - The magic number of a message is a format version number. The only supported - magic number right now is zero - - Format - ====== - Message => Crc MagicByte Attributes Key Value - Crc => int32 - MagicByte => int8 - Attributes => int8 - Key => bytes - Value => bytes - """ - if message.magic == 0: - msg = struct.pack('>BB', message.magic, message.attributes) - msg += write_int_string(message.key) - msg += write_int_string(message.value) - crc = zlib.crc32(msg) - msg = struct.pack('>i%ds' % len(msg), crc, msg) - else: - raise Exception("Unexpected magic number: %d" % message.magic) - return msg - - - @classmethod - def _decode_message_set_iter(cls, data): - """ - Iteratively decode a MessageSet - - Reads repeated elements of (offset, message), calling decode_message to decode a - single message. Since compressed messages contain futher MessageSets, these two methods - have been decoupled so that they may recurse easily. - """ - cur = 0 - while cur < len(data): - try: - ((offset, ), cur) = relative_unpack('>q', data, cur) - (msg, cur) = read_int_string(data, cur) - for (offset, message) in KafkaProtocol._decode_message(msg, offset): - yield OffsetAndMessage(offset, message) - except BufferUnderflowError: # If we get a partial read of a message, stop - raise StopIteration() - - @classmethod - def _decode_message(cls, data, offset): - """ - Decode a single Message - - The only caller of this method is decode_message_set_iter. They are decoupled to - support nested messages (compressed MessageSets). The offset is actually read from - decode_message_set_iter (it is part of the MessageSet payload). - """ - ((crc, magic, att), cur) = relative_unpack('>iBB', data, 0) - if crc != zlib.crc32(data[4:]): - raise ChecksumError("Message checksum failed") - - (key, cur) = read_int_string(data, cur) - (value, cur) = read_int_string(data, cur) - if att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_NONE: - yield (offset, Message(magic, att, key, value)) - elif att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_GZIP: - gz = gzip_decode(value) - for (offset, message) in KafkaProtocol._decode_message_set_iter(gz): - yield (offset, message) - elif att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_SNAPPY: - snp = snappy_decode(value) - for (offset, message) in KafkaProtocol._decode_message_set_iter(snp): - yield (offset, message) - - ################## - # Public API # - ################## - - @classmethod - def create_message(cls, payload, key=None): - """ - Construct a Message - - Params - ====== - payload: bytes, the payload to send to Kafka - key: bytes, a key used for partition routing (optional) - """ - return Message(0, 0, key, payload) - - @classmethod - def create_gzip_message(cls, payloads, key=None): - """ - Construct a Gzipped Message containing multiple Messages - - The given payloads will be encoded, compressed, and sent as a single atomic - message to Kafka. - - Params - ====== - payloads: list(bytes), a list of payload to send be sent to Kafka - key: bytes, a key used for partition routing (optional) - """ - message_set = KafkaProtocol._encode_message_set( - [KafkaProtocol.create_message(payload) for payload in payloads]) - gzipped = gzip_encode(message_set) - return Message(0, 0x00 | (KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_GZIP), key, gzipped) - - @classmethod - def create_snappy_message(cls, payloads, key=None): - """ - Construct a Snappy Message containing multiple Messages - - The given payloads will be encoded, compressed, and sent as a single atomic - message to Kafka. - - Params - ====== - payloads: list(bytes), a list of payload to send be sent to Kafka - key: bytes, a key used for partition routing (optional) - """ - message_set = KafkaProtocol._encode_message_set( - [KafkaProtocol.create_message(payload) for payload in payloads]) - snapped = snappy_encode(message_set) - return Message(0, 0x00 | (KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_SNAPPY), key, snapped) - - @classmethod - def encode_produce_request(cls, client_id, correlation_id, payloads=[], acks=1, timeout=1000): - """ - Encode some ProduceRequest structs - - Params - ====== - client_id: string - correlation_id: string - payloads: list of ProduceRequest - acks: How "acky" you want the request to be - 0: immediate response - 1: written to disk by the leader - 2+: waits for this many number of replicas to sync - -1: waits for all replicas to be in sync - timeout: Maximum time the server will wait for acks from replicas. This is _not_ a socket timeout - """ - grouped_payloads = group_by_topic_and_partition(payloads) - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.PRODUCE_KEY) - message += struct.pack('>hii', acks, timeout, len(grouped_payloads)) - for topic, topic_payloads in grouped_payloads.items(): - message += struct.pack('>h%dsi' % len(topic), len(topic), topic, len(topic_payloads)) - for partition, payload in topic_payloads.items(): - message_set = KafkaProtocol._encode_message_set(payload.messages) - message += struct.pack('>ii%ds' % len(message_set), partition, len(message_set), message_set) - return struct.pack('>i%ds' % len(message), len(message), message) - - @classmethod - def decode_produce_response(cls, data): - """ - Decode bytes to a ProduceResponse - - Params - ====== - data: bytes to decode - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - for i in range(num_topics): - ((strlen,), cur) = relative_unpack('>h', data, cur) - topic = data[cur:cur+strlen] - cur += strlen - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): - ((partition, error, offset), cur) = relative_unpack('>ihq', data, cur) - yield ProduceResponse(topic, partition, error, offset) - - @classmethod - def encode_fetch_request(cls, client_id, correlation_id, payloads=[], max_wait_time=100, min_bytes=4096): - """ - Encodes some FetchRequest structs - - Params - ====== - client_id: string - correlation_id: string - payloads: list of FetchRequest - max_wait_time: int, how long to block waiting on min_bytes of data - min_bytes: int, the minimum number of bytes to accumulate before returning the response - """ - - grouped_payloads = group_by_topic_and_partition(payloads) - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.FETCH_KEY) - message += struct.pack('>iiii', -1, max_wait_time, min_bytes, len(grouped_payloads)) # -1 is the replica id - for topic, topic_payloads in grouped_payloads.items(): - message += write_short_string(topic) - message += struct.pack('>i', len(topic_payloads)) - for partition, payload in topic_payloads.items(): - message += struct.pack('>iqi', partition, payload.offset, payload.max_bytes) - return struct.pack('>i%ds' % len(message), len(message), message) - - @classmethod - def decode_fetch_response_iter(cls, data): - """ - Decode bytes to a FetchResponse - - Params - ====== - data: bytes to decode - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - for i in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): - ((partition, error, highwater_mark_offset), cur) = relative_unpack('>ihq', data, cur) - (message_set, cur) = read_int_string(data, cur) - yield FetchResponse(topic, partition, error, highwater_mark_offset, - KafkaProtocol._decode_message_set_iter(message_set)) - - @classmethod - def encode_offset_request(cls, client_id, correlation_id, payloads=[]): - grouped_payloads = group_by_topic_and_partition(payloads) - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.OFFSET_KEY) - message += struct.pack('>ii', -1, len(grouped_payloads)) # -1 is the replica id - for topic, topic_payloads in grouped_payloads.items(): - message += write_short_string(topic) - message += struct.pack('>i', len(topic_payloads)) - for partition, payload in topic_payloads.items(): - message += struct.pack('>iqi', partition, payload.time, payload.max_offsets) - return struct.pack('>i%ds' % len(message), len(message), message) - - @classmethod - def decode_offset_response(cls, data): - """ - Decode bytes to an OffsetResponse - - Params - ====== - data: bytes to decode - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - for i in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): - ((partition, error, num_offsets,), cur) = relative_unpack('>ihi', data, cur) - offsets = [] - for j in range(num_offsets): - ((offset,), cur) = relative_unpack('>q', data, cur) - offsets.append(offset) - yield OffsetResponse(topic, partition, error, tuple(offsets)) - - @classmethod - def encode_metadata_request(cls, client_id, correlation_id, topics=[]): - """ - Encode a MetadataRequest - - Params - ====== - client_id: string - correlation_id: string - topics: list of strings - """ - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.METADATA_KEY) - message += struct.pack('>i', len(topics)) - for topic in topics: - message += struct.pack('>h%ds' % len(topic), len(topic), topic) - return write_int_string(message) - - @classmethod - def decode_metadata_response(cls, data): - """ - Decode bytes to a MetadataResponse - - Params - ====== - data: bytes to decode - """ - ((correlation_id, numBrokers), cur) = relative_unpack('>ii', data, 0) - - # Broker info - brokers = {} - for i in range(numBrokers): - ((nodeId, ), cur) = relative_unpack('>i', data, cur) - (host, cur) = read_short_string(data, cur) - ((port,), cur) = relative_unpack('>i', data, cur) - brokers[nodeId] = BrokerMetadata(nodeId, host, port) - - # Topic info - ((num_topics,), cur) = relative_unpack('>i', data, cur) - topicMetadata = {} - for i in range(num_topics): - ((topicError,), cur) = relative_unpack('>h', data, cur) - (topicName, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - partitionMetadata = {} - for j in range(num_partitions): - ((partitionErrorCode, partition, leader, numReplicas), cur) = relative_unpack('>hiii', data, cur) - (replicas, cur) = relative_unpack('>%di' % numReplicas, data, cur) - ((numIsr,), cur) = relative_unpack('>i', data, cur) - (isr, cur) = relative_unpack('>%di' % numIsr, data, cur) - partitionMetadata[partition] = PartitionMetadata(topicName, partition, leader, replicas, isr) - topicMetadata[topicName] = partitionMetadata - return (brokers, topicMetadata) - - @classmethod - def encode_offset_commit_request(cls, client_id, correlation_id, group, payloads): - """ - Encode some OffsetCommitRequest structs - - Params - ====== - client_id: string - correlation_id: string - group: string, the consumer group you are committing offsets for - payloads: list of OffsetCommitRequest - """ - grouped_payloads= group_by_topic_and_partition(payloads) - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.OFFSET_COMMIT_KEY) - message += write_short_string(group) - message += struct.pack('>i', len(grouped_payloads)) - for topic, topic_payloads in grouped_payloads.items(): - message += write_short_string(topic) - message += struct.pack('>i', len(topic_payloads)) - for partition, payload in topic_payloads.items(): - message += struct.pack('>iq', partition, payload.offset) - message += write_short_string(payload.metadata) - return struct.pack('>i%ds' % len(message), len(message), message) - - @classmethod - def decode_offset_commit_response(cls, data): - """ - Decode bytes to an OffsetCommitResponse - - Params - ====== - data: bytes to decode - """ - data = data[2:] # TODO remove me when versionId is removed - ((correlation_id,), cur) = relative_unpack('>i', data, 0) - (client_id, cur) = read_short_string(data, cur) - ((num_topics,), cur) = relative_unpack('>i', data, cur) - for i in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): - ((partition, error), cur) = relative_unpack('>ih', data, cur) - yield OffsetCommitResponse(topic, partition, error) - - @classmethod - def encode_offset_fetch_request(cls, client_id, correlation_id, group, payloads): - """ - Encode some OffsetFetchRequest structs - - Params - ====== - client_id: string - correlation_id: string - group: string, the consumer group you are fetching offsets for - payloads: list of OffsetFetchRequest - """ - grouped_payloads = group_by_topic_and_partition(payloads) - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.OFFSET_FETCH_KEY) - message += write_short_string(group) - message += struct.pack('>i', len(grouped_payloads)) - for topic, topic_payloads in grouped_payloads.items(): - message += write_short_string(topic) - message += struct.pack('>i', len(topic_payloads)) - for partition, payload in topic_payloads.items(): - message += struct.pack('>i', partition) - return struct.pack('>i%ds' % len(message), len(message), message) - - @classmethod - def decode_offset_fetch_response(cls, data): - """ - Decode bytes to an OffsetFetchResponse - - Params - ====== - data: bytes to decode - """ - ((correlation_id,), cur) = relative_unpack('>i', data, 0) - (client_id, cur) = read_short_string(data, cur) - ((num_topics,), cur) = relative_unpack('>i', data, cur) - for i in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - for i in range(num_partitions): - ((partition, offset), cur) = relative_unpack('>iq', data, cur) - (metadata, cur) = read_short_string(data, cur) - ((error,), cur) = relative_unpack('>h', data, cur) - yield OffsetFetchResponse(topic, partition, offset, metadata, error) - - -class KafkaConnection(object): - """ - A socket connection to a single Kafka broker - - This class is _not_ thread safe. Each call to `send` must be followed - by a call to `recv` in order to get the correct response. Eventually, - we can do something in here to facilitate multiplexed requests/responses - since the Kafka API includes a correlation id. - """ - def __init__(self, host, port, bufsize=4096): - self.host = host - self.port = port - self.bufsize = bufsize - self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self._sock.connect((host, port)) - self._sock.settimeout(10) - - def __str__(self): - return "<KafkaConnection host=%s port=%d>" % (self.host, self.port) - - ################### - # Private API # - ################### - - def _consume_response(self): - """ - Fully consumer the response iterator - """ - data = "" - for chunk in self._consume_response_iter(): - data += chunk - return data - - def _consume_response_iter(self): - """ - This method handles the response header and error messages. It - then returns an iterator for the chunks of the response - """ - log.debug("Handling response from Kafka") - - # Read the size off of the header - resp = self._sock.recv(4) - if resp == "": - raise Exception("Got no response from Kafka") - (size,) = struct.unpack('>i', resp) - - messageSize = size - 4 - log.debug("About to read %d bytes from Kafka", messageSize) - - # Read the remainder of the response - total = 0 - while total < messageSize: - resp = self._sock.recv(self.bufsize) - log.debug("Read %d bytes from Kafka", len(resp)) - if resp == "": - raise BufferUnderflowError("Not enough data to read this response") - total += len(resp) - yield resp - - ################## - # Public API # - ################## - - # TODO multiplex socket communication to allow for multi-threaded clients - - def send(self, requestId, payload): - "Send a request to Kafka" - sent = self._sock.sendall(payload) - if sent == 0: - raise RuntimeError("Kafka went away") - self.data = self._consume_response() - - def recv(self, requestId): - "Get a response from Kafka" - return self.data - - def close(self): - "Close this connection" - self._sock.close() - -class KafkaClient(object): - - CLIENT_ID = "kafka-python" - ID_GEN = count() - - def __init__(self, host, port, bufsize=4096): - # We need one connection to bootstrap - self.bufsize = bufsize - self.conns = { # (host, port) -> KafkaConnection - (host, port): KafkaConnection(host, port, bufsize) - } - self.brokers = {} # broker_id -> BrokerMetadata - self.topics_to_brokers = {} # topic_id -> broker_id - self.load_metadata_for_topics() - - def close(self): - for conn in self.conns.values(): - conn.close() - - def get_conn_for_broker(self, broker): - "Get or create a connection to a broker" - if (broker.host, broker.port) not in self.conns: - self.conns[(broker.host, broker.port)] = KafkaConnection(broker.host, broker.port, self.bufsize) - return self.conns[(broker.host, broker.port)] - - def next_id(self): - "Generate a new correlation id" - return KafkaClient.ID_GEN.next() - - def load_metadata_for_topics(self, *topics): - """ - Discover brokers and metadata for a set of topics. This method will - recurse in the event of a retry. - """ - requestId = self.next_id() - request = KafkaProtocol.encode_metadata_request(KafkaClient.CLIENT_ID, requestId, topics) - response = self.try_send_request(requestId, request) - if response is None: - raise Exception("All servers failed to process request") - (brokers, topics) = KafkaProtocol.decode_metadata_response(response) - log.debug("Broker metadata: %s", brokers) - log.debug("Topic metadata: %s", topics) - self.brokers.update(brokers) - self.topics_to_brokers = {} - for topic, partitions in topics.items(): - for partition, meta in partitions.items(): - if meta.leader == -1: - log.info("Partition is unassigned, delay for 1s and retry") - time.sleep(1) - self.load_metadata_for_topics(topic) - else: - self.topics_to_brokers[TopicAndPartition(topic, partition)] = brokers[meta.leader] - - def get_leader_for_partition(self, topic, partition): - key = TopicAndPartition(topic, partition) - if key not in self.topics_to_brokers: - self.load_metadata_for_topics(topic) - if key not in self.topics_to_brokers: - raise Exception("Partition does not exist: %s" % str(key)) - return self.topics_to_brokers[key] - - def send_produce_request(self, payloads=[], fail_on_error=True, callback=None): - """ - Encode and send some ProduceRequests - - ProduceRequests will be grouped by (topic, partition) and then sent to a specific - broker. Output is a list of responses in the same order as the list of payloads - specified - - Params - ====== - payloads: list of ProduceRequest - fail_on_error: boolean, should we raise an Exception if we encounter an API error? - callback: function, instead of returning the ProduceResponse, first pass it through this function - - Return - ====== - list of ProduceResponse or callback(ProduceResponse), in the order of input payloads - """ - # Group the produce requests by which broker they go to - original_keys = [] - payloads_by_broker = defaultdict(list) - for payload in payloads: - payloads_by_broker[self.get_leader_for_partition(payload.topic, payload.partition)] += payloads - original_keys.append((payload.topic, payload.partition)) - - # Accumulate the responses in a dictionary - acc = {} - - # For each broker, send the list of request payloads - for broker, payloads in payloads_by_broker.items(): - conn = self.get_conn_for_broker(broker) - requestId = self.next_id() - request = KafkaProtocol.encode_produce_request(KafkaClient.CLIENT_ID, requestId, payloads) - # Send the request - conn.send(requestId, request) - response = conn.recv(requestId) - for produce_response in KafkaProtocol.decode_produce_response(response): - # Check for errors - if fail_on_error == True and produce_response.error != 0: - raise Exception("ProduceRequest for %s failed with errorcode=%d", - (TopicAndPartition(produce_response.topic, produce_response.partition), produce_response.error)) - # Run the callback - if callback is not None: - acc[(produce_response.topic, produce_response.partition)] = callback(produce_response) - else: - acc[(produce_response.topic, produce_response.partition)] = produce_response - - # Order the accumulated responses by the original key order - return (acc[k] for k in original_keys) - - def send_fetch_request(self, payloads=[], fail_on_error=True, callback=None): - """ - Encode and send a FetchRequest - - Payloads are grouped by topic and partition so they can be pipelined to the same - brokers. - """ - # Group the produce requests by which broker they go to - original_keys = [] - payloads_by_broker = defaultdict(list) - for payload in payloads: - payloads_by_broker[self.get_leader_for_partition(payload.topic, payload.partition)].append(payload) - original_keys.append((payload.topic, payload.partition)) - - # Accumulate the responses in a dictionary, keyed by topic+partition - acc = {} - - # For each broker, send the list of request payloads - for broker, payloads in payloads_by_broker.items(): - conn = self.get_conn_for_broker(broker) - requestId = self.next_id() - request = KafkaProtocol.encode_fetch_request(KafkaClient.CLIENT_ID, requestId, payloads) - # Send the request - conn.send(requestId, request) - response = conn.recv(requestId) - for fetch_response in KafkaProtocol.decode_fetch_response_iter(response): - # Check for errors - if fail_on_error == True and fetch_response.error != 0: - raise Exception("FetchRequest %s failed with errorcode=%d" % - (TopicAndPartition(fetch_response.topic, fetch_response.partition), fetch_response.error)) - # Run the callback - if callback is not None: - acc[(fetch_response.topic, fetch_response.partition)] = callback(fetch_response) - else: - acc[(fetch_response.topic, fetch_response.partition)] = fetch_response - - # Order the accumulated responses by the original key order - return (acc[k] for k in original_keys) - - def try_send_request(self, requestId, request): - """ - Attempt to send a broker-agnostic request to one of the available brokers. - Keep trying until you succeed. - """ - for conn in self.conns.values(): - try: - conn.send(requestId, request) - response = conn.recv(requestId) - return response - except Exception, e: - log.warning("Could not send request [%r] to server %s, trying next server: %s" % (request, conn, e)) - continue - return None - - def send_offset_request(self, payloads=[], fail_on_error=True, callback=None): - requestId = self.next_id() - request = KafkaProtocol.encode_offset_request(KafkaClient.CLIENT_ID, requestId, payloads) - response = self.try_send_request(requestId, request) - if response is None: - if fail_on_error is True: - raise Exception("All servers failed to process request") - else: - return None - out = [] - for offset_response in KafkaProtocol.decode_offset_response(response): - if fail_on_error == True and offset_response.error != 0: - raise Exception("OffsetRequest failed with errorcode=%s", offset_response.error) - if callback is not None: - out.append(callback(offset_response)) - else: - out.append(offset_response) - return out - - def send_offset_commit_request(self, group, payloads=[], fail_on_error=True, callback=None): - requestId = self.next_id() - request = KafkaProtocol.encode_offset_commit_request(KafkaClient.CLIENT_ID, requestId, group, payloads) - response = self.try_send_request(requestId, request) - if response is None: - if fail_on_error is True: - raise Exception("All servers failed to process request") - else: - return None - out = [] - for offset_commit_response in KafkaProtocol.decode_offset_commit_response(response): - if fail_on_error == True and offset_commit_response.error != 0: - print(offset_commit_response) - raise Exception("OffsetCommitRequest failed with errorcode=%s", offset_commit_response.error) - if callback is not None: - out.append(callback(offset_commit_response)) - else: - out.append(offset_commit_response) - return out - - def send_offset_fetch_request(self, group, payloads=[], fail_on_error=True, callback=None): - requestId = self.next_id() - request = KafkaProtocol.encode_offset_fetch_request(KafkaClient.CLIENT_ID, requestId, group, payloads) - response = self.try_send_request(requestId, request) - if response is None: - if fail_on_error is True: - raise Exception("All servers failed to process request") - else: - return None - out = [] - for offset_fetch_response in KafkaProtocol.decode_offset_fetch_response(response): - if fail_on_error == True and offset_fetch_response.error != 0: - raise Exception("OffsetFetchRequest for topic=%s, partition=%d failed with errorcode=%s" % ( - offset_fetch_response.topic, offset_fetch_response.partition, offset_fetch_response.error)) - if callback is not None: - out.append(callback(offset_fetch_response)) - else: - out.append(offset_fetch_response) - return out - - -if __name__ == "__main__": - logging.basicConfig(level=logging.DEBUG) - - topic = "foo8" - # Bootstrap connection - conn = KafkaClient("localhost", 49720) - - # Create some Messages - messages = (KafkaProtocol.create_gzip_message(["GZIPPed"]), - KafkaProtocol.create_message("not-gzipped")) - - produce1 = ProduceRequest(topic=topic, partition=0, messages=messages) - produce2 = ProduceRequest(topic=topic, partition=1, messages=messages) - - # Send the ProduceRequest - produce_resp = conn.send_produce_request(payloads=[produce1, produce2]) - - # Check for errors - for resp in produce_resp: - if resp.error != 0: - raise Exception("ProduceRequest failed with errorcode=%d", resp.error) - print resp - - # Offset commit/fetch - #conn.send_offset_commit_request(group="group", payloads=[OffsetCommitRequest("topic-1", 0, 42, "METADATA?")]) - #conn.send_offset_fetch_request(group="group", payloads=[OffsetFetchRequest("topic-1", 0)]) - - def init_offsets(offset_response): - if offset_response.error not in (ErrorMapping.NO_ERROR, ErrorMapping.UNKNOWN_TOPIC_OR_PARTITON): - raise Exception("OffsetFetch failed: %s" % (offset_response)) - elif offset_response.error == ErrorMapping.UNKNOWN_TOPIC_OR_PARTITON: - return 0 - else: - return offset_response.offset - # Load offsets - (offset1, offset2) = conn.send_offset_fetch_request( - group="group1", - payloads=[OffsetFetchRequest(topic, 0),OffsetFetchRequest(topic, 1)], - fail_on_error=False, - callback=init_offsets - ) - print offset1, offset2 - - while True: - for resp in conn.send_fetch_request(payloads=[FetchRequest(topic=topic, partition=0, offset=offset1, max_bytes=4096)]): - i = 0 - for msg in resp.messages: - print msg - offset1 = msg.offset+1 - print offset1, conn.send_offset_commit_request(group="group1", payloads=[OffsetCommitRequest(topic, 0, offset1, "")]) - i += 1 - if i == 0: - raise StopIteration("no more messages") - - for resp in conn.send_fetch_request(payloads=[FetchRequest(topic=topic, partition=1, offset=offset2, max_bytes=4096)]): - i = 0 - for msg in resp.messages: - print msg - offset2 = msg.offset+1 - print offset2, conn.send_offset_commit_request(group="group1", payloads=[OffsetCommitRequest(topic, 1, offset2, "")]) - i += 1 - if i == 0: - raise StopIteration("no more messages") - diff --git a/kafka/util.py b/kafka/util.py index 509c5b8..715fb28 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -4,13 +4,13 @@ import struct def write_int_string(s): if s is None: - return struct.pack('>i', 0) # TODO change this to -1 when KAFKA-771 is accepted + return struct.pack('>i', -1) else: return struct.pack('>i%ds' % len(s), len(s), s) def write_short_string(s): if s is None: - return struct.pack('>h', 0) # TODO change this to -1 when KAFKA-771 is accepted + return struct.pack('>h', -1) else: return struct.pack('>h%ds' % len(s), len(s), s) @@ -2,7 +2,7 @@ from distutils.core import setup setup( name="kafka-python", - version="0.1-alpha", + version="0.2-alpha", author="David Arthur", author_email="mumrah@gmail.com", url="https://github.com/mumrah/kafka-python", diff --git a/test/integration.py b/test/integration.py index 598b17a..9fa8538 100644 --- a/test/integration.py +++ b/test/integration.py @@ -10,8 +10,9 @@ import tempfile from threading import Thread, Event import time import unittest +from urlparse import urlparse -from kafka.client08 import * +from kafka.client import * def get_open_port(): sock = socket.socket() @@ -49,16 +50,26 @@ class KafkaFixture(Thread): stdout = open(os.path.join(logDir, 'stdout'), 'w') # Create the config file + zkChroot = "kafka-python_%s" % self.tmpDir.replace("/", "_") logConfig = "test/resources/log4j.properties" configFile = os.path.join(self.tmpDir, 'server.properties') f = open('test/resources/server.properties', 'r') props = f.read() f = open(configFile, 'w') - f.write(props % {'kafka.port': self.port, 'kafka.tmp.dir': logDir, 'kafka.partitions': 2}) + f.write(props % {'kafka.port': self.port, 'kafka.tmp.dir': logDir, 'kafka.partitions': 2, 'zk.chroot': zkChroot}) f.close() + cp = build_kafka_classpath() + + # Create the Zookeeper chroot + args = shlex.split("java -cp %s org.apache.zookeeper.ZooKeeperMain create /%s kafka-python" % (cp, zkChroot)) + proc = subprocess.Popen(args) + ret = proc.wait() + assert ret == 0 + + # Start Kafka - args = shlex.split("java -Xmx256M -server -Dlog4j.configuration=%s -cp %s kafka.Kafka %s" % (logConfig, build_kafka_classpath(), configFile)) + args = shlex.split("java -Xmx256M -server -Dlog4j.configuration=%s -cp %s kafka.Kafka %s" % (logConfig, cp, configFile)) proc = subprocess.Popen(args, bufsize=1, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, env={"JMX_PORT":"%d" % get_open_port()}) killed = False @@ -334,11 +345,7 @@ class TestKafkaClient(unittest.TestCase): (resp,) = self.client.send_offset_fetch_request("group", [req]) self.assertEquals(resp.error, 0) self.assertEquals(resp.offset, 42) - self.assertEquals(resp.metadata, "metadata") - - - - + self.assertEquals(resp.metadata, "") # Metadata isn't stored for now if __name__ == "__main__": logging.basicConfig(level=logging.INFO) diff --git a/test/resources/server.properties b/test/resources/server.properties index 0d01fca..85bee88 100644 --- a/test/resources/server.properties +++ b/test/resources/server.properties @@ -102,7 +102,7 @@ log.cleanup.interval.mins=1 # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. -zk.connect=localhost:2181/kafka-python +zk.connect=localhost:2181/%(zk.chroot)s # Timeout in ms for connecting to zookeeper zk.connection.timeout.ms=1000000 |