diff options
-rw-r--r-- | LICENSE | 203 | ||||
-rw-r--r-- | README.md | 75 | ||||
-rw-r--r-- | example.py | 29 | ||||
-rw-r--r-- | kafka.py | 475 |
4 files changed, 782 insertions, 0 deletions
@@ -0,0 +1,203 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + diff --git a/README.md b/README.md new file mode 100644 index 0000000..d8aa6c2 --- /dev/null +++ b/README.md @@ -0,0 +1,75 @@ +# Kakfa Python client + +This module provides low-level protocol support Apache Kafka. It implements the five basic request types (and their responses): Produce, Fetch, MultiFetch, MultiProduce, and Offsets. + +Compatible with Apache Kafka 0.7x. + +# License + +Copyright 2012, David Arthur under Apache License, v2.0. See `LICENSE` + +# Usage + +## Send a message to a topic + +You need to specify the topic and partition + + kafka = KafkaClient("localhost", 9092) + kafka.send_messages_simple("my-topic", 0, "some message") + kafka.close() + +## Send several messages to a topic + +Same as before, just add more arguments to `send_simple` + + kafka = KafkaClient("localhost", 9092) + kafka.send_messages_simple("my-topic", 0, "some message", "another message", "and another") + kafka.close() + +## Recieve some messages from a topic + +Supply `get_message_set` with a `FetchRequest`, get back the messages and new `FetchRequest` + + kafka = KafkaClient("localhost", 9092) + req = FetchRequest("my-topic", 0, 0, 1024*1024) + (messages, req1) = kafka.get_message_set(req) + kafka.close() + +The returned `FetchRequest` includes the offset of the next message. This makes +paging through the queue very simple. + +## Send multiple messages to multiple topics + +For this we use the `send_multi_message_set` method along with `ProduceRequest` objects. + + kafka = KafkaClient("localhost", 9092) + req1 = ProduceRequest("my-topic-1", 0, [ + create_message_from_string("message one"), + create_message_from_string("message two") + ]) + req2 = ProduceRequest("my-topic-2", 0, [ + create_message_from_string("nachricht ein"), + create_message_from_string("nachricht zwei") + ]) + kafka.sent_multi_message_set([req1, req1]) + kafka.close() + +## Iterate through all messages from an offset + +The `iter_messages` method will make the underlying calls to `get_message_set` +to provide a generator that returns every message available. + + kafka = KafkaClient("localhost", 9092) + for msg in kafka.iter_messages(FetchRequest("my-topic", 0, 0, 1024*1024)): + print(msg.payload) + kafka.close() + +An optional `auto` argument will control auto-paging through results + + kafka = KafkaClient("localhost", 9092) + for msg in kafka.iter_messages(FetchRequest("my-topic", 0, 0, 1024*1024), False): + print(msg.payload) + kafka.close() + +This will only iterate through messages in the first byte range of +(0, 1024\*1024) diff --git a/example.py b/example.py new file mode 100644 index 0000000..c58751b --- /dev/null +++ b/example.py @@ -0,0 +1,29 @@ +import logging + +from kafka import KafkaClient, FetchRequest, ProduceRequest +from kafka import create_message_from_string + +def produce_example(kafka): + message = create_message_from_string("test") + request = ProduceRequest("my-topic", 0, [message]) + print("Sending %s" % str(request)) + kafka.send_message_set(request) + +def consume_example(kafka): + request = FetchRequest("my-topic", 0, 0, 64) + print("Sending %s" % str(request)) + (messages, nextRequest) = kafka.get_message_set(request) + print("Got %d messages:" % len(messages)) + for message in messages: + print("\t%s" % message.payload) + print("Next request %s" % str(nextRequest)) + +def main(): + kafka = KafkaClient("localhost", 9092) + produce_example(kafka) + consume_example(kafka) + + +if __name__ == "__main__": + logging.basicConfig(level=logging.DEBUG) + main() diff --git a/kafka.py b/kafka.py new file mode 100644 index 0000000..3439238 --- /dev/null +++ b/kafka.py @@ -0,0 +1,475 @@ +from collections import namedtuple +import logging +import select +import socket +import struct +import zlib + +log = logging.getLogger("org.apache.kafka") + +def length_prefix_message(msg): + """ + Prefix a message with it's length as an int + """ + return struct.pack('>i', len(msg)) + msg + + +def create_message_from_string(payload): + return Message(1, 0, zlib.crc32(payload), payload) + +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(errorType) + + +Message = namedtuple("Message", ["magic", "attributes", "crc", "payload"]) +FetchRequest = namedtuple("FetchRequest", ["topic", "partition", "offset", "size"]) +ProduceRequest = namedtuple("ProduceRequest", ["topic", "partition", "messages"]) +OffsetRequest = namedtuple("OffsetRequest", ["topic", "partition", "time", "maxOffsets"]) + +class KafkaClient(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 + """ + + PRODUCE_KEY = 0 + FETCH_KEY = 1 + MULTIFETCH_KEY = 2 + MULTIPRODUCE_KEY = 3 + OFFSET_KEY = 4 + + def __init__(self, host, port): + self.host = host + self.port = port + self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + self._sock.connect((host, port)) + log.debug("Connected to %s on %d", host, port) + + ###################### + # Protocol Stuff # + ###################### + + 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") + # 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(1024) + log.debug("Read %d bytes from Kafka", len(resp)) + if resp == "": + raise Exception("Underflow") + total += len(resp) + yield resp + + def _consume_response(self): + """ + Fully consumer the response iterator + """ + data = "" + for chunk in self._consume_response_iter(): + data += chunk + return data + + + def create_message(self, message): + """ + Create a Message from a Message tuple + + Params + ====== + 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. + """ + if message.magic == 0: + return struct.pack('>Bi%ds' % len(message.payload), + message.magic, message.crc, message.payload) + elif message.magic == 1: + return struct.pack('>BBi%ds' % len(message.payload), + message.magic, message.attributes, message.crc, message.payload) + else: + raise Exception("Unknown message version: %d" % message.magic) + + def create_message_set(self, messages): + message_set = "" + for message in messages: + encoded_message = self.create_message(message) + message_set += length_prefix_message(encoded_message) + return message_set + + def create_produce_request(self, produceRequest): + """ + Create 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 = self.create_message_set(messages) + 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 create_multi_produce_request(self, produceRequests): + req = struct.pack('>HH', KafkaClient.MULTIPRODUCE_KEY, len(produceRequests)) + for (topic, partition, messages) in produceRequests: + message_set = self.create_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 + + def create_fetch_request(self, fetchRequest): + """ + Create 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 + + def create_multi_fetch_request(self, fetchRequests): + """ + Create the MultiFetchRequest message from a list of FetchRequest objects + + 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 + + def create_offset_request(self, offsetRequest): + """ + Create an OffsetRequest message + + 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 request-key (4) is encoded as a short (int16). + """ + (topic, partition, offset, maxOffsets) = offsetRequest + req = struct.pack('>HH%dsiqi' % len(topic), KafkaClient.OFFSET_KEY, len(topic), topic, partition, offset, maxOffsets) + return req + + def read_message_set(self, data): + """ + Read a MessageSet + + 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: + if (cur + 5) > size: + # Underflow for the Header + 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 + # Read a Message header (length, magic byte) + (N, magic) = struct.unpack('>iB', data[cur:(cur+5)]) + + if (cur + N + 4) > size: + # Underflow for this Message + log.debug("Not enough data to read next message") + break + cur += 5 + + if magic == 0: # v0 Message + # Read crc; check the crc; append the message + (crc,) = struct.unpack('>i', data[cur:(cur+4)]) + cur += 4 + payload = data[cur:(cur+N-5)] + assert zlib.crc32(payload) == crc + cur += (N-5) + log.debug("Got v0 Message, %d bytes", len(payload)) + msgs.append(Message(magic, None, crc, payload)) + elif magic == 1: # v1 Message + # Read attributes, crc; check the crc; append the message + (att, crc) = struct.unpack('>Bi', data[cur:(cur+5)]) + cur += 5 + payload = data[cur:(cur+N-6)] + assert zlib.crc32(payload) == crc + cur += (N-6) + log.debug("Got v1 Message, %d bytes", len(payload)) + msgs.append(Message(magic, att, crc, payload)) + + # Return the retrieved messages and the cursor position + return (msgs, cur) + + ######################### + # Advanced User API # + ######################### + + def send_message_set(self, produceRequest): + """ + Send a ProduceRequest + + Params + ====== + produceRequest: ProduceRequest + """ + req = length_prefix_message(self.create_produce_request(produceRequest)) + log.debug("Sending %d bytes to Kafka", len(req)) + self._sock.send(req) + + def send_multi_message_set(self, produceRequests): + """ + Send a MultiProduceRequest + + Params + ====== + produceRequests: list of ProduceRequest + """ + req = length_prefix_message(self.create_multi_produce_request(produceRequests)) + log.debug("Sending %d bytes to Kafka", len(req)) + self._sock.send(req) + + def get_message_set(self, fetchRequest): + """ + Send a FetchRequest and return the Messages + + Params + ====== + fetchRequest: FetchRequest named tuple + + Returns + ======= + A tuple of (list(Message), FetchRequest). This FetchRequest will have the offset + starting at the next message. + """ + + req = length_prefix_message(self.create_fetch_request(fetchRequest)) + log.debug("Sending %d bytes to Kafka", len(req)) + self._sock.send(req) + 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)) + + def get_multi_message_set(self, fetchRequests): + """ + Send several FetchRequests in a single pipelined request. + + 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.create_multi_fetch_request(fetchRequests)) + log.debug("Sending %d bytes to Kafka", len(req)) + self._sock.send(req) + 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 + + def get_offsets(self, offsetRequest): + """ + Get the offsets for a topic + + Params + ====== + offsetRequest: OffsetRequest + + Returns + ======= + offsets: tuple of offsets + + Wire Format + =========== + <OffsetResponse> ::= <num> [ <offsets> ] + <num> ::= <int32> + <offsets> ::= <offset> [ <offsets> ] + <offset> ::= <int64> + + """ + req = length_prefix_message(create_offset_request(offsetRequest)) + log.debug("Sending %d bytes to Kafka", len(req)) + self._sock.send(req) + + data = self._consume_response() + (num,) = struct.unpack('>i', data[0:4]) + offsets = struct.unpack('>%dq' % num, data[4:]) + return offsets + + ####################### + # Simple User API # + ####################### + + def send_messages_simple(self, topic, partition, *payloads): + """ + Send one or more strings to Kafka + + Params + ====== + topic: string + partition: int + payloads: strings + """ + messages = tuple([create_message_from_string(payload) for payload in payloads]) + self.send_message_set(ProduceRequest(topic, partition, messages)) + + def iter_messages(self, topic, partition, offset, size, auto=True): + """ + Helper method that iterates through all messages starting at the offset + in the given FetchRequest + + 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() |