From c593217815ac82a1b918ce1d16c980e65ae3ef6a Mon Sep 17 00:00:00 2001 From: Mahendra M Date: Wed, 29 May 2013 14:20:58 +0530 Subject: [PATCH 1/2] PEP8-ify most of the files consumer.py and conn.py will be done later after pending merges --- kafka/__init__.py | 2 +- kafka/codec.py | 8 +- kafka/common.py | 45 ++++++--- kafka/producer.py | 4 +- kafka/protocol.py | 231 +++++++++++++++++++++++++++++++++------------- kafka/queue.py | 96 ++++++++++++------- kafka/util.py | 51 ++++++---- 7 files changed, 309 insertions(+), 128 deletions(-) diff --git a/kafka/__init__.py b/kafka/__init__.py index 5198d2fe2..d2291692f 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -4,7 +4,7 @@ __license__ = 'Apache License 2.0' __copyright__ = 'Copyright 2012, David Arthur under Apache License, v2.0' -from kafka.client import KafkaClient +from kafka.client import KafkaClient from kafka.conn import KafkaConnection from kafka.protocol import ( create_message, create_gzip_message, create_snappy_message diff --git a/kafka/codec.py b/kafka/codec.py index 83f3c0b56..c7d39920c 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -6,10 +6,11 @@ try: import snappy - hasSnappy=True + hasSnappy = True except ImportError: log.warn("Snappy codec not available") - hasSnappy=False + hasSnappy = False + def gzip_encode(payload): buf = StringIO() @@ -21,6 +22,7 @@ def gzip_encode(payload): buf.close() return out + def gzip_decode(payload): buf = StringIO(payload) f = gzip.GzipFile(fileobj=buf, mode='r') @@ -29,11 +31,13 @@ def gzip_decode(payload): buf.close() return out + def snappy_encode(payload): if not hasSnappy: raise NotImplementedError("Snappy codec not available") return snappy.compress(payload) + def snappy_decode(payload): if not hasSnappy: raise NotImplementedError("Snappy codec not available") diff --git a/kafka/common.py b/kafka/common.py index 447684f4c..9aab8fc17 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -5,26 +5,49 @@ ############### # 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"]) +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"]) +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 +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 diff --git a/kafka/producer.py b/kafka/producer.py index 47e690b1f..589eb118a 100644 --- a/kafka/producer.py +++ b/kafka/producer.py @@ -6,6 +6,7 @@ log = logging.getLogger("kafka") + class SimpleProducer(object): """ A simple, round-robbin producer. Each message goes to exactly one partition @@ -18,6 +19,7 @@ def __init__(self, client, topic): def send_messages(self, *msg): req = ProduceRequest(self.topic, self.next_partition.next(), - messages=[create_message(m) for m in msg]) + messages=[create_message(m) for m in msg]) + resp = self.client.send_produce_request([req])[0] assert resp.error == 0 diff --git a/kafka/protocol.py b/kafka/protocol.py index 94a7f2a8b..6bd5c7341 100644 --- a/kafka/protocol.py +++ b/kafka/protocol.py @@ -18,10 +18,12 @@ log = logging.getLogger("kafka") + 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. + 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 @@ -44,18 +46,18 @@ def _encode_message_header(cls, client_id, correlation_id, request_key): """ Encode the common request envelope """ - return struct.pack('>hhih%ds' % len(client_id), + return struct.pack('>hhih%ds' % len(client_id), request_key, # ApiKey 0, # ApiVersion correlation_id, # CorrelationId - len(client_id), # + 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 + Encode a MessageSet. Unlike other arrays in the protocol, + MessageSets are not length-prefixed Format ====== @@ -66,7 +68,8 @@ def _encode_message_set(cls, messages): 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) + message_set += struct.pack('>qi%ds' % len(encoded_message), 0, + len(encoded_message), encoded_message) return message_set @classmethod @@ -74,10 +77,10 @@ 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 + The magic number of a message is a format version number. + The only supported magic number right now is zero - Format + Format ====== Message => Crc MagicByte Attributes Key Value Crc => int32 @@ -96,24 +99,27 @@ def _encode_message(cls, message): 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. + 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): + 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 + + except BufferUnderflowError: + # If we get a partial read of a message, stop raise StopIteration() @classmethod @@ -121,9 +127,10 @@ 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). + 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:]): @@ -131,23 +138,29 @@ def _decode_message(cls, data, offset): (key, cur) = read_int_string(data, cur) (value, cur) = read_int_string(data, cur) - if att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_NONE: + + codec = att & KafkaProtocol.ATTRIBUTE_CODEC_MASK + + if codec == KafkaProtocol.CODEC_NONE: yield (offset, Message(magic, att, key, value)) - elif att & KafkaProtocol.ATTRIBUTE_CODEC_MASK == KafkaProtocol.CODEC_GZIP: + + elif codec == 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: + for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz): + yield (offset, msg) + + elif codec == KafkaProtocol.CODEC_SNAPPY: snp = snappy_decode(value) - for (offset, message) in KafkaProtocol._decode_message_set_iter(snp): - yield (offset, message) + for (offset, msg) in KafkaProtocol._decode_message_set_iter(snp): + yield (offset, msg) ################## # Public API # ################## @classmethod - def encode_produce_request(cls, client_id, correlation_id, payloads=[], acks=1, timeout=1000): + def encode_produce_request(cls, client_id, correlation_id, + payloads=None, acks=1, timeout=1000): """ Encode some ProduceRequest structs @@ -161,39 +174,53 @@ def encode_produce_request(cls, client_id, correlation_id, payloads=[], acks=1, 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 + timeout: Maximum time the server will wait for acks from replicas. + This is _not_ a socket timeout """ + payloads = [] if payloads is None else payloads grouped_payloads = group_by_topic_and_partition(payloads) - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.PRODUCE_KEY) + + 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)) + 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) + msg_set = KafkaProtocol._encode_message_set(payload.messages) + message += struct.pack('>ii%ds' % len(msg_set), partition, + len(msg_set), msg_set) + return struct.pack('>i%ds' % len(message), len(message), message) @classmethod def decode_produce_response(cls, data): """ - Decode bytes to a ProduceResponse + 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] + 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) + ((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): + def encode_fetch_request(cls, client_id, correlation_id, payloads=None, + max_wait_time=100, min_bytes=4096): """ Encodes some FetchRequest structs @@ -203,17 +230,27 @@ def encode_fetch_request(cls, client_id, correlation_id, payloads=[], max_wait_t 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 + min_bytes: int, the minimum number of bytes to accumulate before + returning the response """ - + + payloads = [] if payloads is None else payloads 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 + + message = cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.FETCH_KEY) + + # -1 is the replica id + message += struct.pack('>iiii', -1, max_wait_time, min_bytes, + 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('>iqi', partition, payload.offset, payload.max_bytes) + message += struct.pack('>iqi', partition, payload.offset, + payload.max_bytes) + return struct.pack('>i%ds' % len(message), len(message), message) @classmethod @@ -226,25 +263,41 @@ def decode_fetch_response(cls, data): 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) + ((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, + + 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=[]): + def encode_offset_request(cls, client_id, correlation_id, payloads=None): + payloads = [] if payloads is None else 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 + + message = cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.OFFSET_KEY) + + # -1 is the replica id + message += struct.pack('>ii', -1, 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('>iqi', partition, payload.time, payload.max_offsets) + message += struct.pack('>iqi', partition, payload.time, + payload.max_offsets) + return struct.pack('>i%ds' % len(message), len(message), message) @classmethod @@ -257,19 +310,24 @@ def decode_offset_response(cls, data): 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) + ((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=[]): + def encode_metadata_request(cls, client_id, correlation_id, topics=None): """ Encode a MetadataRequest @@ -279,10 +337,15 @@ def encode_metadata_request(cls, client_id, correlation_id, topics=[]): correlation_id: string topics: list of strings """ - message = cls._encode_message_header(client_id, correlation_id, KafkaProtocol.METADATA_KEY) + topics = [] if topics is None else topics + 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 @@ -307,22 +370,34 @@ def decode_metadata_response(cls, data): # 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) + ((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) + + 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): + def encode_offset_commit_request(cls, client_id, correlation_id, + group, payloads): """ Encode some OffsetCommitRequest structs @@ -333,16 +408,21 @@ def encode_offset_commit_request(cls, client_id, correlation_id, group, payloads 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) + 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 @@ -357,15 +437,18 @@ def decode_offset_commit_response(cls, data): ((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) @classmethod - def encode_offset_fetch_request(cls, client_id, correlation_id, group, payloads): + def encode_offset_fetch_request(cls, client_id, correlation_id, + group, payloads): """ Encode some OffsetFetchRequest structs @@ -377,14 +460,19 @@ def encode_offset_fetch_request(cls, client_id, correlation_id, group, payloads) 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 = 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 @@ -400,14 +488,19 @@ def decode_offset_fetch_response(cls, data): ((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) + + yield OffsetFetchResponse(topic, partition, offset, + metadata, error) + def create_message(payload, key=None): """ @@ -420,6 +513,7 @@ def create_message(payload, key=None): """ return Message(0, 0, key, payload) + def create_gzip_message(payloads, key=None): """ Construct a Gzipped Message containing multiple Messages @@ -433,9 +527,13 @@ def create_gzip_message(payloads, key=None): key: bytes, a key used for partition routing (optional) """ message_set = KafkaProtocol._encode_message_set( - [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) + [create_message(payload) for payload in payloads]) + + gzipped = gzip_encode(message_set) + codec = KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_GZIP + + return Message(0, 0x00 | codec, key, gzipped) + def create_snappy_message(payloads, key=None): """ @@ -450,6 +548,9 @@ def create_snappy_message(payloads, key=None): key: bytes, a key used for partition routing (optional) """ message_set = KafkaProtocol._encode_message_set( - [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) + [create_message(payload) for payload in payloads]) + + snapped = snappy_encode(message_set) + codec = KafkaProtocol.ATTRIBUTE_CODEC_MASK & KafkaProtocol.CODEC_SNAPPY + + return Message(0, 0x00 | codec, key, snapped) diff --git a/kafka/queue.py b/kafka/queue.py index 6fe9eaa34..3bd7dca2c 100644 --- a/kafka/queue.py +++ b/kafka/queue.py @@ -10,8 +10,10 @@ raise NotImplementedError("Still need to refactor this class") + class KafkaConsumerProcess(Process): - def __init__(self, client, topic, partition, out_queue, barrier, consumer_fetch_size=1024, consumer_sleep=200): + def __init__(self, client, topic, partition, out_queue, barrier, + consumer_fetch_size=1024, consumer_sleep=200): self.client = copy(client) self.topic = topic self.partition = partition @@ -23,29 +25,40 @@ def __init__(self, client, topic, partition, out_queue, barrier, consumer_fetch_ Process.__init__(self) def __str__(self): - return "[KafkaConsumerProcess: topic=%s, partition=%s, sleep=%s]" % ( - self.topic, self.partition, self.consumer_sleep) + return "[KafkaConsumerProcess: topic=%s, partition=%s, sleep=%s]" % \ + (self.topic, self.partition, self.consumer_sleep) def run(self): self.barrier.wait() log.info("Starting %s" % self) - fetchRequest = FetchRequest(self.topic, self.partition, offset=0, size=self.consumer_fetch_size) + fetchRequest = FetchRequest(self.topic, self.partition, + offset=0, size=self.consumer_fetch_size) + while True: - if self.barrier.is_set() == False: + if self.barrier.is_set() is False: log.info("Shutdown %s" % self) self.client.close() break + lastOffset = fetchRequest.offset (messages, fetchRequest) = self.client.get_message_set(fetchRequest) + if fetchRequest.offset == lastOffset: - log.debug("No more data for this partition, sleeping a bit (200ms)") + log.debug("No more data for this partition, " + "sleeping a bit (200ms)") time.sleep(self.consumer_sleep) continue + for message in messages: self.out_queue.put(message) + class KafkaProducerProcess(Process): - def __init__(self, client, topic, in_queue, barrier, producer_flush_buffer=500, producer_flush_timeout=2000, producer_timeout=100): + def __init__(self, client, topic, in_queue, barrier, + producer_flush_buffer=500, + producer_flush_timeout=2000, + producer_timeout=100): + self.client = copy(client) self.topic = topic self.in_queue = in_queue @@ -57,8 +70,10 @@ def __init__(self, client, topic, in_queue, barrier, producer_flush_buffer=500, Process.__init__(self) def __str__(self): - return "[KafkaProducerProcess: topic=%s, flush_buffer=%s, flush_timeout=%s, timeout=%s]" % ( - self.topic, self.producer_flush_buffer, self.producer_flush_timeout, self.producer_timeout) + return "[KafkaProducerProcess: topic=%s, flush_buffer=%s, \ + flush_timeout=%s, timeout=%s]" % ( + self.topic, self.producer_flush_buffer, + self.producer_flush_timeout, self.producer_timeout) def run(self): self.barrier.wait() @@ -67,36 +82,47 @@ def run(self): last_produce = time.time() def flush(messages): - self.client.send_message_set(ProduceRequest(self.topic, -1, messages)) + self.client.send_message_set(ProduceRequest(self.topic, -1, + messages)) del messages[:] while True: - if self.barrier.is_set() == False: + if self.barrier.is_set() is False: log.info("Shutdown %s, flushing messages" % self) flush(messages) self.client.close() break + if len(messages) > self.producer_flush_buffer: - log.debug("Message count threashold reached. Flushing messages") + log.debug("Message count threshold reached. Flushing messages") flush(messages) last_produce = time.time() + elif (time.time() - last_produce) > self.producer_flush_timeout: log.debug("Producer timeout reached. Flushing messages") flush(messages) last_produce = time.time() + try: - messages.append(KafkaClient.create_message(self.in_queue.get(True, self.producer_timeout))) + msg = KafkaClient.create_message(self.in_queue.get(True, + self.producer_timeout)) + messages.append(msg) + except Empty: continue + class KafkaQueue(object): - def __init__(self, client, topic, partitions, producer_config={}, consumer_config={}): + def __init__(self, client, topic, partitions, + producer_config=None, consumer_config=None): """ - KafkaQueue a Queue-like object backed by a Kafka producer and some number of consumers + KafkaQueue a Queue-like object backed by a Kafka producer and some + number of consumers - Messages are eagerly loaded by the consumer in batches of size consumer_fetch_size. - Messages are buffered in the producer thread until producer_flush_timeout or - producer_flush_buffer is reached. + Messages are eagerly loaded by the consumer in batches of size + consumer_fetch_size. + Messages are buffered in the producer thread until + producer_flush_timeout or producer_flush_buffer is reached. Params ====== @@ -108,21 +134,26 @@ def __init__(self, client, topic, partitions, producer_config={}, consumer_confi Consumer Config =============== - consumer_fetch_size: int, number of bytes to fetch in one call to Kafka. Default - is 1024 - consumer_sleep: int, time in milliseconds a consumer should sleep when it reaches - the end of a partition. Default is 200 + consumer_fetch_size: int, number of bytes to fetch in one call + to Kafka. Default is 1024 + consumer_sleep: int, time in milliseconds a consumer should sleep + when it reaches the end of a partition. Default is 200 Producer Config =============== - producer_timeout: int, time in milliseconds a producer should wait for messages to - enqueue for producing. Default is 100 - producer_flush_timeout: int, time in milliseconds a producer should allow messages - to accumulate before sending to Kafka. Default is 2000 - producer_flush_buffer: int, number of messages a producer should allow to accumulate. - Default is 500 - + producer_timeout: int, time in milliseconds a producer should + wait for messages to enqueue for producing. + Default is 100 + producer_flush_timeout: int, time in milliseconds a producer + should allow messages to accumulate before + sending to Kafka. Default is 2000 + producer_flush_buffer: int, number of messages a producer should + allow to accumulate. Default is 500 + """ + producer_config = {} if producer_config is None else producer_config + consumer_config = {} if consumer_config is None else consumer_config + self.in_queue = Queue() self.out_queue = Queue() self.consumers = [] @@ -130,12 +161,15 @@ def __init__(self, client, topic, partitions, producer_config={}, consumer_confi # Initialize and start consumer threads for partition in partitions: - consumer = KafkaConsumerProcess(client, topic, partition, self.in_queue, self.barrier, **consumer_config) + consumer = KafkaConsumerProcess(client, topic, partition, + self.in_queue, self.barrier, + **consumer_config) consumer.start() self.consumers.append(consumer) # Initialize and start producer thread - self.producer = KafkaProducerProcess(client, topic, self.out_queue, self.barrier, **producer_config) + self.producer = KafkaProducerProcess(client, topic, self.out_queue, + self.barrier, **producer_config) self.producer.start() # Trigger everything to start diff --git a/kafka/util.py b/kafka/util.py index 8c02cb29a..10bf8381a 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -3,64 +3,81 @@ import struct from threading import Timer + def write_int_string(s): if s is None: 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', -1) else: return struct.pack('>h%ds' % len(s), len(s), s) + def read_short_string(data, cur): - if len(data) < cur+2: + if len(data) < cur + 2: raise BufferUnderflowError("Not enough data left") - (strLen,) = struct.unpack('>h', data[cur:cur+2]) + + (strLen,) = struct.unpack('>h', data[cur:cur + 2]) if strLen == -1: - return (None, cur+2) + return (None, cur + 2) + cur += 2 - if len(data) < cur+strLen: + if len(data) < cur + strLen: raise BufferUnderflowError("Not enough data left") - out = data[cur:cur+strLen] - return (out, cur+strLen) + + out = data[cur:cur + strLen] + return (out, cur + strLen) + def read_int_string(data, cur): - if len(data) < cur+4: + if len(data) < cur + 4: raise BufferUnderflowError("Not enough data left") - (strLen,) = struct.unpack('>i', data[cur:cur+4]) + + (strLen,) = struct.unpack('>i', data[cur:cur + 4]) if strLen == -1: - return (None, cur+4) + return (None, cur + 4) + cur += 4 - if len(data) < cur+strLen: + if len(data) < cur + strLen: raise BufferUnderflowError("Not enough data left") - out = data[cur:cur+strLen] - return (out, cur+strLen) + + out = data[cur:cur + strLen] + return (out, cur + strLen) + def relative_unpack(fmt, data, cur): size = struct.calcsize(fmt) - if len(data) < cur+size: + if len(data) < cur + size: raise BufferUnderflowError("Not enough data left") - out = struct.unpack(fmt, data[cur:cur+size]) - return (out, cur+size) + + out = struct.unpack(fmt, data[cur:cur + size]) + return (out, cur + size) + def group_by_topic_and_partition(tuples): out = defaultdict(dict) for t in tuples: out[t.topic][t.partition] = t - return out + return out + class BufferUnderflowError(Exception): pass + class ChecksumError(Exception): pass + class ReentrantTimer(object): """ - A timer that can be restarted, unlike threading.Timer (although this uses threading.Timer) + A timer that can be restarted, unlike threading.Timer + (although this uses threading.Timer) t: timer interval in milliseconds fn: a callable to invoke From b829c60e23a888b68b604bb5144fc7fe5885a1e3 Mon Sep 17 00:00:00 2001 From: Mahendra M Date: Fri, 31 May 2013 14:57:57 +0530 Subject: [PATCH 2/2] Finish making remaining files pep8 ready --- kafka/client.py | 169 ++++++++++++++++++++++++++++++---------------- kafka/consumer.py | 93 +++++++++++++++---------- 2 files changed, 168 insertions(+), 94 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index eb2c25c6b..5595d49ae 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -10,43 +10,47 @@ from kafka.common import * from kafka.conn import KafkaConnection -from kafka.protocol import KafkaProtocol +from kafka.protocol import KafkaProtocol log = logging.getLogger("kafka") + class KafkaClient(object): CLIENT_ID = "kafka-python" - ID_GEN = count() + ID_GEN = count() def __init__(self, host, port, bufsize=4096): - # We need one connection to bootstrap + # We need one connection to bootstrap self.bufsize = bufsize - self.conns = { # (host, port) -> KafkaConnection + 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.topic_partitions = defaultdict(list) # topic_id -> [0, 1, 2, ...] + } + self.brokers = {} # broker_id -> BrokerMetadata + self.topics_to_brokers = {} # topic_id -> broker_id + self.topic_partitions = defaultdict(list) # topic_id -> [0, 1, 2, ...] self._load_metadata_for_topics() ################## # Private API # ################## - 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) + self.conns[(broker.host, broker.port)] = \ + KafkaConnection(broker.host, broker.port, self.bufsize) + return self.conns[(broker.host, broker.port)] 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 _load_metadata_for_topics(self, *topics): @@ -55,13 +59,18 @@ def _load_metadata_for_topics(self, *topics): recurse in the event of a retry. """ requestId = self._next_id() - request = KafkaProtocol.encode_metadata_request(KafkaClient.CLIENT_ID, requestId, topics) + request = KafkaProtocol.encode_metadata_request(KafkaClient.CLIENT_ID, + requestId, topics) + response = self._send_broker_unaware_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(): @@ -77,7 +86,8 @@ def _load_metadata_for_topics(self, *topics): time.sleep(1) self._load_metadata_for_topics(topic) else: - self.topics_to_brokers[TopicAndPartition(topic, partition)] = brokers[meta.leader] + topic_part = TopicAndPartition(topic, partition) + self.topics_to_brokers[topic_part] = brokers[meta.leader] self.topic_partitions[topic].append(partition) def _next_id(self): @@ -86,8 +96,8 @@ def _next_id(self): def _send_broker_unaware_request(self, requestId, request): """ - Attempt to send a broker-agnostic request to one of the available brokers. - Keep trying until you succeed. + 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: @@ -95,32 +105,43 @@ def _send_broker_unaware_request(self, 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)) + log.warning("Could not send request [%r] to server %s, " + "trying next server: %s" % (request, conn, e)) continue + return None def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): """ - Group a list of request payloads by topic+partition and send them to the - leader broker for that partition using the supplied encode/decode functions + Group a list of request payloads by topic+partition and send them to + the leader broker for that partition using the supplied encode/decode + functions Params ====== - payloads: list of object-like entities with a topic and partition attribute - encode_fn: a method to encode the list of payloads to a request body, must accept - client_id, correlation_id, and payloads as keyword arguments - decode_fn: a method to decode a response body into response objects. The response - objects must be object-like and have topic and partition attributes + payloads: list of object-like entities with a topic and + partition attribute + encode_fn: a method to encode the list of payloads to a request body, + must accept client_id, correlation_id, and payloads as + keyword arguments + decode_fn: a method to decode a response body into response objects. + The response objects must be object-like and have topic + and partition attributes Return ====== List of response objects in the same order as the supplied payloads """ + # Group the requests by topic+partition 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) + leader = self._get_leader_for_partition(payload.topic, + payload.partition) + + payloads_by_broker[leader].append(payload) original_keys.append((payload.topic, payload.partition)) # Accumulate the responses in a dictionary @@ -130,7 +151,8 @@ def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): for broker, payloads in payloads_by_broker.items(): conn = self._get_conn_for_broker(broker) requestId = self._next_id() - request = encoder_fn(client_id=KafkaClient.CLIENT_ID, correlation_id=requestId, payloads=payloads) + request = encoder_fn(client_id=KafkaClient.CLIENT_ID, + correlation_id=requestId, payloads=payloads) # Send the request, recv the response conn.send(requestId, request) @@ -149,33 +171,43 @@ def close(self): for conn in self.conns.values(): conn.close() - def send_produce_request(self, payloads=[], acks=1, timeout=1000, fail_on_error=True, callback=None): + def send_produce_request(self, payloads=[], acks=1, timeout=1000, + 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 + 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 + 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 + list of ProduceResponse or callback(ProduceResponse), in the + order of input payloads """ - resps = self._send_broker_aware_request(payloads, - partial(KafkaProtocol.encode_produce_request, acks=acks, timeout=timeout), - KafkaProtocol.decode_produce_response) + + encoder = partial(KafkaProtocol.encode_produce_request, + acks=acks, timeout=timeout) + decoder = KafkaProtocol.decode_produce_response + resps = self._send_broker_aware_request(payloads, encoder, decoder) + out = [] for resp in resps: # Check for errors - if fail_on_error == True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("ProduceRequest for %s failed with errorcode=%d" % - (TopicAndPartition(resp.topic, resp.partition), resp.error)) + if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: + raise Exception("ProduceRequest for %s failed with " + "errorcode=%d" % ( + TopicAndPartition(resp.topic, resp.partition), + resp.error)) + # Run the callback if callback is not None: out.append(callback(resp)) @@ -183,22 +215,27 @@ def send_produce_request(self, payloads=[], acks=1, timeout=1000, fail_on_error= out.append(resp) return out - def send_fetch_request(self, payloads=[], fail_on_error=True, callback=None): + 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. + + Payloads are grouped by topic and partition so they can be pipelined + to the same brokers. """ resps = self._send_broker_aware_request(payloads, KafkaProtocol.encode_fetch_request, KafkaProtocol.decode_fetch_response) + out = [] for resp in resps: # Check for errors - if fail_on_error == True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("FetchRequest for %s failed with errorcode=%d" % - (TopicAndPartition(resp.topic, resp.partition), resp.error)) + if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: + raise Exception("FetchRequest for %s failed with " + "errorcode=%d" % ( + TopicAndPartition(resp.topic, resp.partition), + resp.error)) + # Run the callback if callback is not None: out.append(callback(resp)) @@ -206,43 +243,55 @@ def send_fetch_request(self, payloads=[], fail_on_error=True, callback=None): out.append(resp) return out - - def send_offset_request(self, payloads=[], fail_on_error=True, callback=None): + def send_offset_request(self, payloads=[], fail_on_error=True, + callback=None): resps = self._send_broker_aware_request(payloads, KafkaProtocol.encode_offset_request, KafkaProtocol.decode_offset_response) + out = [] for resp in resps: - if fail_on_error == True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("OffsetRequest failed with errorcode=%s", resp.error) + if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: + raise Exception("OffsetRequest failed with errorcode=%s", + resp.error) if callback is not None: out.append(callback(resp)) else: out.append(resp) return out - def send_offset_commit_request(self, group, payloads=[], fail_on_error=True, callback=None): - resps = self._send_broker_aware_request(payloads, - partial(KafkaProtocol.encode_offset_commit_request, group=group), - KafkaProtocol.decode_offset_commit_response) + def send_offset_commit_request(self, group, payloads=[], + fail_on_error=True, callback=None): + encoder = partial(KafkaProtocol.encode_offset_commit_request, + group=group) + decoder = KafkaProtocol.decode_offset_commit_response + resps = self._send_broker_aware_request(payloads, encoder, decoder) + out = [] for resp in resps: - if fail_on_error == True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("OffsetCommitRequest failed with errorcode=%s", resp.error) + if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: + raise Exception("OffsetCommitRequest failed with " + "errorcode=%s", resp.error) + if callback is not None: out.append(callback(resp)) else: out.append(resp) return out - def send_offset_fetch_request(self, group, payloads=[], fail_on_error=True, callback=None): - resps = self._send_broker_aware_request(payloads, - partial(KafkaProtocol.encode_offset_fetch_request, group=group), - KafkaProtocol.decode_offset_fetch_response) + def send_offset_fetch_request(self, group, payloads=[], + fail_on_error=True, callback=None): + + encoder = partial(KafkaProtocol.encode_offset_fetch_request, + group=group) + decoder = KafkaProtocol.decode_offset_fetch_response + resps = self._send_broker_aware_request(payloads, encoder, decoder) + out = [] for resp in resps: - if fail_on_error == True and resp.error != ErrorMapping.NO_ERROR: - raise Exception("OffsetCommitRequest failed with errorcode=%s", resp.error) + if fail_on_error is True and resp.error != ErrorMapping.NO_ERROR: + raise Exception("OffsetCommitRequest failed with errorcode=%s", + resp.error) if callback is not None: out.append(callback(resp)) else: diff --git a/kafka/consumer.py b/kafka/consumer.py index d09803a04..3b64571c5 100644 --- a/kafka/consumer.py +++ b/kafka/consumer.py @@ -16,6 +16,7 @@ AUTO_COMMIT_MSG_COUNT = 100 AUTO_COMMIT_INTERVAL = 5000 + class SimpleConsumer(object): """ A simple consumer implementation that consumes all partitions for a topic @@ -25,13 +26,16 @@ class SimpleConsumer(object): topic: the topic to consume auto_commit: default True. Whether or not to auto commit the offsets - auto_commit_every_n: default 100. How many messages to consume before a commit - auto_commit_every_t: default 5000. How much time (in milliseconds) to wait before commit + auto_commit_every_n: default 100. How many messages to consume + before a commit + auto_commit_every_t: default 5000. How much time (in milliseconds) to + wait before commit Auto commit details: - If both auto_commit_every_n and auto_commit_every_t are set, they will reset one another - when one is triggered. These triggers simply call the commit method on this class. A - manual call to commit will also reset these triggers + If both auto_commit_every_n and auto_commit_every_t are set, they will + reset one another when one is triggered. These triggers simply call the + commit method on this class. A manual call to commit will also reset + these triggers """ def __init__(self, client, group, topic, auto_commit=True, @@ -63,17 +67,19 @@ def get_or_init_offset_callback(resp): elif resp.error == ErrorMapping.UNKNOWN_TOPIC_OR_PARTITON: return 0 else: - raise Exception("OffsetFetchRequest for topic=%s, partition=%d failed with errorcode=%s" % ( - resp.topic, resp.partition, resp.error)) + raise Exception("OffsetFetchRequest for topic=%s, " + "partition=%d failed with errorcode=%s" % ( + resp.topic, resp.partition, resp.error)) # Uncomment for 0.8.1 # #for partition in self.client.topic_partitions[topic]: # req = OffsetFetchRequest(topic, partition) # (offset,) = self.client.send_offset_fetch_request(group, [req], - # callback=get_or_init_offset_callback, fail_on_error=False) + # callback=get_or_init_offset_callback, + # fail_on_error=False) # self.offsets[partition] = offset - + for partition in self.client.topic_partitions[topic]: self.offsets[partition] = 0 @@ -87,14 +93,16 @@ def seek(self, offset, whence): 1 is relative to the current offset 2 is relative to the latest known offset (tail) """ - if whence == 1: # relative to current position + if whence == 1: # relative to current position for partition, _offset in self.offsets.items(): self.offsets[partition] = _offset + offset - elif whence in (0, 2): # relative to beginning or end - # divide the request offset by number of partitions, distribute the remained evenly + elif whence in (0, 2): # relative to beginning or end + # divide the request offset by number of partitions, + # distribute the remained evenly (delta, rem) = divmod(offset, len(self.offsets)) deltas = {} - for partition, r in izip_longest(self.offsets.keys(), repeat(1, rem), fillvalue=0): + for partition, r in izip_longest(self.offsets.keys(), + repeat(1, rem), fillvalue=0): deltas[partition] = delta + r reqs = [] @@ -108,7 +116,8 @@ def seek(self, offset, whence): resps = self.client.send_offset_request(reqs) for resp in resps: - self.offsets[resp.partition] = resp.offsets[0] + deltas[resp.partition] + self.offsets[resp.partition] = resp.offsets[0] + \ + deltas[resp.partition] else: raise ValueError("Unexpected value for `whence`, %d" % whence) @@ -149,24 +158,24 @@ def commit(self, partitions=[]): """ Commit offsets for this consumer - partitions: list of partitions to commit, default is to commit all of them + partitions: list of partitions to commit, default is to commit + all of them """ - # short circuit if nothing happened if self.count_since_commit == 0: return with self.commit_lock: reqs = [] - if len(partitions) == 0: # commit all partitions + if len(partitions) == 0: # commit all partitions partitions = self.offsets.keys() for partition in partitions: offset = self.offsets[partition] log.debug("Commit offset %d in SimpleConsumer: " "group=%s, topic=%s, partition=%s" % - (offset, self.group, self.topic, partition)) + (offset, self.group, self.topic, partition)) reqs.append(OffsetCommitRequest(self.topic, partition, offset, None)) @@ -177,10 +186,27 @@ def commit(self, partitions=[]): self.count_since_commit = 0 + def _auto_commit(self): + """ + Check if we have to commit based on number of messages and commit + """ + + # Check if we are supposed to do an auto-commit + if not self.auto_commit or self.auto_commit_every_n is None: + return + + if self.count_since_commit > self.auto_commit_every_n: + if self.commit_timer is not None: + self.commit_timer.stop() + self.commit() + self.commit_timer.start() + else: + self.commit() + def __iter__(self): """ - Create an iterate per partition. Iterate through them calling next() until they are - all exhausted. + Create an iterate per partition. Iterate through them calling next() + until they are all exhausted. """ iters = {} for partition, offset in self.offsets.items(): @@ -199,31 +225,30 @@ def __iter__(self): except StopIteration: log.debug("Done iterating over partition %s" % partition) del iters[partition] - continue # skip auto-commit since we didn't yield anything - # auto commit logic + # skip auto-commit since we didn't yield anything + continue + + # Count, check and commit messages if necessary self.count_since_commit += 1 - if self.auto_commit is True: - if self.auto_commit_every_n is not None and self.count_since_commit > self.auto_commit_every_n: - if self.commit_timer is not None: - self.commit_timer.stop() - self.commit() - self.commit_timer.start() - else: - self.commit() + self._auto_commit() def __iter_partition__(self, partition, offset): """ - Iterate over the messages in a partition. Create a FetchRequest to get back - a batch of messages, yield them one at a time. After a batch is exhausted, - start a new batch unless we've reached the end of ths partition. + Iterate over the messages in a partition. Create a FetchRequest + to get back a batch of messages, yield them one at a time. + After a batch is exhausted, start a new batch unless we've reached + the end of this partition. """ while True: - req = FetchRequest(self.topic, partition, offset, 1024) # TODO configure fetch size + # TODO: configure fetch size + req = FetchRequest(self.topic, partition, offset, 1024) (resp,) = self.client.send_fetch_request([req]) + assert resp.topic == self.topic assert resp.partition == partition + next_offset = None for message in resp.messages: next_offset = message.offset