summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorDavid Arthur <mumrah@gmail.com>2013-03-01 15:12:21 -0500
committerDavid Arthur <mumrah@gmail.com>2013-04-02 20:19:30 -0400
commiteac51e9c68c50f15962b6c785ede92cb3d512a17 (patch)
treeec814fa65e96a8a4db1d2e6aceb319ebb23a781f
parent2a3d231aa61642c57537bc2128dd4f2bd30f35dd (diff)
downloadkafka-python-eac51e9c68c50f15962b6c785ede92cb3d512a17.tar.gz
Integration tests passing
m---------kafka-src0
-rw-r--r--kafka/__init__.py3
-rw-r--r--kafka/client.py1239
-rw-r--r--kafka/client08.py872
-rw-r--r--kafka/util.py4
-rw-r--r--setup.py2
-rw-r--r--test/integration.py23
-rw-r--r--test/resources/server.properties2
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)
diff --git a/setup.py b/setup.py
index 90608a2..18f9fcc 100644
--- a/setup.py
+++ b/setup.py
@@ -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