From f0cd6d4082d7abe95693f63b4697cb4ed2b8a6d8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Nov 2015 04:01:20 +0800 Subject: Rename kafka.protocol -> kafka.protocol.legacy --- kafka/protocol.py | 646 --------------------------------------------- kafka/protocol/__init__.py | 6 + kafka/protocol/legacy.py | 646 +++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 652 insertions(+), 646 deletions(-) delete mode 100644 kafka/protocol.py create mode 100644 kafka/protocol/__init__.py create mode 100644 kafka/protocol/legacy.py diff --git a/kafka/protocol.py b/kafka/protocol.py deleted file mode 100644 index 412a957..0000000 --- a/kafka/protocol.py +++ /dev/null @@ -1,646 +0,0 @@ -import logging -import struct - -import six - -from six.moves import xrange - -from kafka.codec import ( - gzip_encode, gzip_decode, snappy_encode, snappy_decode -) -from kafka.common import ( - Message, OffsetAndMessage, TopicAndPartition, - BrokerMetadata, TopicMetadata, PartitionMetadata, - MetadataResponse, ProduceResponse, FetchResponse, - OffsetResponse, OffsetCommitResponse, OffsetFetchResponse, - ProtocolError, BufferUnderflowError, ChecksumError, - ConsumerFetchSizeTooSmall, UnsupportedCodecError, - ConsumerMetadataResponse -) -from kafka.util import ( - crc32, read_short_string, read_int_string, relative_unpack, - write_short_string, write_int_string, group_by_topic_and_partition -) - - -log = logging.getLogger(__name__) - -ATTRIBUTE_CODEC_MASK = 0x03 -CODEC_NONE = 0x00 -CODEC_GZIP = 0x01 -CODEC_SNAPPY = 0x02 -ALL_CODECS = (CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY) - - -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 = 8 - OFFSET_FETCH_KEY = 9 - CONSUMER_METADATA_KEY = 10 - - ################### - # Private API # - ################### - - @classmethod - def _encode_message_header(cls, client_id, correlation_id, request_key, - version=0): - """ - Encode the common request envelope - """ - return struct.pack('>hhih%ds' % len(client_id), - request_key, # ApiKey - version, # ApiVersion - correlation_id, # CorrelationId - len(client_id), # ClientId size - 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.append(struct.pack('>qi%ds' % len(encoded_message), 0, - len(encoded_message), - encoded_message)) - return b''.join(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 = b''.join([ - struct.pack('>BB', message.magic, message.attributes), - write_int_string(message.key), - write_int_string(message.value) - ]) - crc = crc32(msg) - msg = struct.pack('>I%ds' % len(msg), crc, msg) - else: - raise ProtocolError("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 - read_message = False - 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): - read_message = True - yield OffsetAndMessage(offset, message) - except BufferUnderflowError: - # NOTE: Not sure this is correct error handling: - # Is it possible to get a BUE if the message set is somewhere - # in the middle of the fetch response? If so, we probably have - # an issue that's not fetch size too small. - # Aren't we ignoring errors if we fail to unpack data by - # raising StopIteration()? - # If _decode_message() raises a ChecksumError, couldn't that - # also be due to the fetch size being too small? - if read_message is False: - # If we get a partial read of a message, but haven't - # yielded anything there's a problem - raise ConsumerFetchSizeTooSmall() - else: - 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 != crc32(data[4:]): - raise ChecksumError("Message checksum failed") - - (key, cur) = read_int_string(data, cur) - (value, cur) = read_int_string(data, cur) - - codec = att & ATTRIBUTE_CODEC_MASK - - if codec == CODEC_NONE: - yield (offset, Message(magic, att, key, value)) - - elif codec == CODEC_GZIP: - gz = gzip_decode(value) - for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz): - yield (offset, msg) - - elif codec == CODEC_SNAPPY: - snp = snappy_decode(value) - 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=None, acks=1, timeout=1000): - """ - Encode some ProduceRequest structs - - Arguments: - client_id: string - correlation_id: int - 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 - - """ - payloads = [] if payloads is None else payloads - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.PRODUCE_KEY)) - - message.append(struct.pack('>hii', acks, timeout, - len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(struct.pack('>h%dsi' % len(topic), len(topic), topic, - len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - msg_set = KafkaProtocol._encode_message_set(payload.messages) - message.append(struct.pack('>ii%ds' % len(msg_set), partition, - len(msg_set), msg_set)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) - - @classmethod - def decode_produce_response(cls, data): - """ - Decode bytes to a ProduceResponse - - Arguments: - data: bytes to decode - - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - - for _ 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 _ 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=None, - max_wait_time=100, min_bytes=4096): - """ - Encodes some FetchRequest structs - - Arguments: - client_id: string - correlation_id: int - 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 - """ - - payloads = [] if payloads is None else payloads - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.FETCH_KEY)) - - # -1 is the replica id - message.append(struct.pack('>iiii', -1, max_wait_time, min_bytes, - len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>iqi', partition, payload.offset, - payload.max_bytes)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) - - @classmethod - def decode_fetch_response(cls, data): - """ - Decode bytes to a FetchResponse - - Arguments: - data: bytes to decode - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - - for _ in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for j 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=None): - payloads = [] if payloads is None else payloads - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_KEY)) - - # -1 is the replica id - message.append(struct.pack('>ii', -1, len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>iqi', partition, payload.time, - payload.max_offsets)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) - - @classmethod - def decode_offset_response(cls, data): - """ - Decode bytes to an OffsetResponse - - Arguments: - data: bytes to decode - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - - for _ in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for _ in range(num_partitions): - ((partition, error, num_offsets,), cur) = \ - relative_unpack('>ihi', data, cur) - - offsets = [] - for k 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=None, - payloads=None): - """ - Encode a MetadataRequest - - Arguments: - client_id: string - correlation_id: int - topics: list of strings - """ - if payloads is None: - topics = [] if topics is None else topics - else: - topics = payloads - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.METADATA_KEY)) - - message.append(struct.pack('>i', len(topics))) - - for topic in topics: - message.append(struct.pack('>h%ds' % len(topic), len(topic), topic)) - - msg = b''.join(message) - return write_int_string(msg) - - @classmethod - def decode_metadata_response(cls, data): - """ - Decode bytes to a MetadataResponse - - Arguments: - data: bytes to decode - """ - ((correlation_id, numbrokers), cur) = relative_unpack('>ii', data, 0) - - # Broker info - brokers = [] - for _ 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.append(BrokerMetadata(nodeId, host, port)) - - # Topic info - ((num_topics,), cur) = relative_unpack('>i', data, cur) - topic_metadata = [] - - for _ in range(num_topics): - ((topic_error,), cur) = relative_unpack('>h', data, cur) - (topic_name, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - partition_metadata = [] - - for _ in range(num_partitions): - ((partition_error_code, partition, leader, numReplicas), cur) = \ - relative_unpack('>hiii', data, cur) - - (replicas, cur) = relative_unpack( - '>%di' % numReplicas, data, cur) - - ((num_isr,), cur) = relative_unpack('>i', data, cur) - (isr, cur) = relative_unpack('>%di' % num_isr, data, cur) - - partition_metadata.append( - PartitionMetadata(topic_name, partition, leader, - replicas, isr, partition_error_code) - ) - - topic_metadata.append( - TopicMetadata(topic_name, topic_error, partition_metadata) - ) - - return MetadataResponse(brokers, topic_metadata) - - @classmethod - def encode_consumer_metadata_request(cls, client_id, correlation_id, payloads): - """ - Encode a ConsumerMetadataRequest - - Arguments: - client_id: string - correlation_id: int - payloads: string (consumer group) - """ - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.CONSUMER_METADATA_KEY)) - message.append(struct.pack('>h%ds' % len(payloads), len(payloads), payloads)) - - msg = b''.join(message) - return write_int_string(msg) - - @classmethod - def decode_consumer_metadata_response(cls, data): - """ - Decode bytes to a ConsumerMetadataResponse - - Arguments: - data: bytes to decode - """ - ((correlation_id, error, nodeId), cur) = relative_unpack('>ihi', data, 0) - (host, cur) = read_short_string(data, cur) - ((port,), cur) = relative_unpack('>i', data, cur) - - return ConsumerMetadataResponse(error, nodeId, host, port) - - @classmethod - def encode_offset_commit_request(cls, client_id, correlation_id, - group, payloads): - """ - Encode some OffsetCommitRequest structs - - Arguments: - client_id: string - correlation_id: int - group: string, the consumer group you are committing offsets for - payloads: list of OffsetCommitRequest - """ - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_COMMIT_KEY)) - message.append(write_short_string(group)) - message.append(struct.pack('>i', len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>iq', partition, payload.offset)) - message.append(write_short_string(payload.metadata)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) - - @classmethod - def decode_offset_commit_response(cls, data): - """ - Decode bytes to an OffsetCommitResponse - - Arguments: - data: bytes to decode - """ - ((correlation_id,), cur) = relative_unpack('>i', data, 0) - ((num_topics,), cur) = relative_unpack('>i', data, cur) - - for _ in xrange(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for _ 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, from_kafka=False): - """ - Encode some OffsetFetchRequest structs. The request is encoded using - version 0 if from_kafka is false, indicating a request for Zookeeper - offsets. It is encoded using version 1 otherwise, indicating a request - for Kafka offsets. - - Arguments: - client_id: string - correlation_id: int - group: string, the consumer group you are fetching offsets for - payloads: list of OffsetFetchRequest - from_kafka: bool, default False, set True for Kafka-committed offsets - """ - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - reqver = 1 if from_kafka else 0 - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_FETCH_KEY, - version=reqver)) - - message.append(write_short_string(group)) - message.append(struct.pack('>i', len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>i', partition)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) - - @classmethod - def decode_offset_fetch_response(cls, data): - """ - Decode bytes to an OffsetFetchResponse - - Arguments: - data: bytes to decode - """ - - ((correlation_id,), cur) = relative_unpack('>i', data, 0) - ((num_topics,), cur) = relative_unpack('>i', data, cur) - - for _ in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for _ 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) - - -def create_message(payload, key=None): - """ - Construct a Message - - Arguments: - payload: bytes, the payload to send to Kafka - key: bytes, a key used for partition routing (optional) - - """ - return Message(0, 0, key, payload) - - -def create_gzip_message(payloads, key=None, compresslevel=None): - """ - Construct a Gzipped Message containing multiple Messages - - The given payloads will be encoded, compressed, and sent as a single atomic - message to Kafka. - - Arguments: - 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( - [create_message(payload, pl_key) for payload, pl_key in payloads]) - - gzipped = gzip_encode(message_set, compresslevel=compresslevel) - codec = ATTRIBUTE_CODEC_MASK & CODEC_GZIP - - return Message(0, 0x00 | codec, key, gzipped) - - -def create_snappy_message(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. - - Arguments: - 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( - [create_message(payload, pl_key) for payload, pl_key in payloads]) - - snapped = snappy_encode(message_set) - codec = ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY - - return Message(0, 0x00 | codec, key, snapped) - - -def create_message_set(messages, codec=CODEC_NONE, key=None, compresslevel=None): - """Create a message set using the given codec. - - If codec is CODEC_NONE, return a list of raw Kafka messages. Otherwise, - return a list containing a single codec-encoded message. - """ - if codec == CODEC_NONE: - return [create_message(m, k) for m, k in messages] - elif codec == CODEC_GZIP: - return [create_gzip_message(messages, key, compresslevel)] - elif codec == CODEC_SNAPPY: - return [create_snappy_message(messages, key)] - else: - raise UnsupportedCodecError("Codec 0x%02x unsupported" % codec) diff --git a/kafka/protocol/__init__.py b/kafka/protocol/__init__.py new file mode 100644 index 0000000..39cb64a --- /dev/null +++ b/kafka/protocol/__init__.py @@ -0,0 +1,6 @@ +from legacy import ( + create_message, create_gzip_message, + create_snappy_message, create_message_set, + CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY, ALL_CODECS, + ATTRIBUTE_CODEC_MASK, KafkaProtocol, +) diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py new file mode 100644 index 0000000..412a957 --- /dev/null +++ b/kafka/protocol/legacy.py @@ -0,0 +1,646 @@ +import logging +import struct + +import six + +from six.moves import xrange + +from kafka.codec import ( + gzip_encode, gzip_decode, snappy_encode, snappy_decode +) +from kafka.common import ( + Message, OffsetAndMessage, TopicAndPartition, + BrokerMetadata, TopicMetadata, PartitionMetadata, + MetadataResponse, ProduceResponse, FetchResponse, + OffsetResponse, OffsetCommitResponse, OffsetFetchResponse, + ProtocolError, BufferUnderflowError, ChecksumError, + ConsumerFetchSizeTooSmall, UnsupportedCodecError, + ConsumerMetadataResponse +) +from kafka.util import ( + crc32, read_short_string, read_int_string, relative_unpack, + write_short_string, write_int_string, group_by_topic_and_partition +) + + +log = logging.getLogger(__name__) + +ATTRIBUTE_CODEC_MASK = 0x03 +CODEC_NONE = 0x00 +CODEC_GZIP = 0x01 +CODEC_SNAPPY = 0x02 +ALL_CODECS = (CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY) + + +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 = 8 + OFFSET_FETCH_KEY = 9 + CONSUMER_METADATA_KEY = 10 + + ################### + # Private API # + ################### + + @classmethod + def _encode_message_header(cls, client_id, correlation_id, request_key, + version=0): + """ + Encode the common request envelope + """ + return struct.pack('>hhih%ds' % len(client_id), + request_key, # ApiKey + version, # ApiVersion + correlation_id, # CorrelationId + len(client_id), # ClientId size + 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.append(struct.pack('>qi%ds' % len(encoded_message), 0, + len(encoded_message), + encoded_message)) + return b''.join(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 = b''.join([ + struct.pack('>BB', message.magic, message.attributes), + write_int_string(message.key), + write_int_string(message.value) + ]) + crc = crc32(msg) + msg = struct.pack('>I%ds' % len(msg), crc, msg) + else: + raise ProtocolError("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 + read_message = False + 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): + read_message = True + yield OffsetAndMessage(offset, message) + except BufferUnderflowError: + # NOTE: Not sure this is correct error handling: + # Is it possible to get a BUE if the message set is somewhere + # in the middle of the fetch response? If so, we probably have + # an issue that's not fetch size too small. + # Aren't we ignoring errors if we fail to unpack data by + # raising StopIteration()? + # If _decode_message() raises a ChecksumError, couldn't that + # also be due to the fetch size being too small? + if read_message is False: + # If we get a partial read of a message, but haven't + # yielded anything there's a problem + raise ConsumerFetchSizeTooSmall() + else: + 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 != crc32(data[4:]): + raise ChecksumError("Message checksum failed") + + (key, cur) = read_int_string(data, cur) + (value, cur) = read_int_string(data, cur) + + codec = att & ATTRIBUTE_CODEC_MASK + + if codec == CODEC_NONE: + yield (offset, Message(magic, att, key, value)) + + elif codec == CODEC_GZIP: + gz = gzip_decode(value) + for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz): + yield (offset, msg) + + elif codec == CODEC_SNAPPY: + snp = snappy_decode(value) + 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=None, acks=1, timeout=1000): + """ + Encode some ProduceRequest structs + + Arguments: + client_id: string + correlation_id: int + 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 + + """ + payloads = [] if payloads is None else payloads + grouped_payloads = group_by_topic_and_partition(payloads) + + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.PRODUCE_KEY)) + + message.append(struct.pack('>hii', acks, timeout, + len(grouped_payloads))) + + for topic, topic_payloads in grouped_payloads.items(): + message.append(struct.pack('>h%dsi' % len(topic), len(topic), topic, + len(topic_payloads))) + + for partition, payload in topic_payloads.items(): + msg_set = KafkaProtocol._encode_message_set(payload.messages) + message.append(struct.pack('>ii%ds' % len(msg_set), partition, + len(msg_set), msg_set)) + + msg = b''.join(message) + return struct.pack('>i%ds' % len(msg), len(msg), msg) + + @classmethod + def decode_produce_response(cls, data): + """ + Decode bytes to a ProduceResponse + + Arguments: + data: bytes to decode + + """ + ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + + for _ 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 _ 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=None, + max_wait_time=100, min_bytes=4096): + """ + Encodes some FetchRequest structs + + Arguments: + client_id: string + correlation_id: int + 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 + """ + + payloads = [] if payloads is None else payloads + grouped_payloads = group_by_topic_and_partition(payloads) + + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.FETCH_KEY)) + + # -1 is the replica id + message.append(struct.pack('>iiii', -1, max_wait_time, min_bytes, + len(grouped_payloads))) + + for topic, topic_payloads in grouped_payloads.items(): + message.append(write_short_string(topic)) + message.append(struct.pack('>i', len(topic_payloads))) + for partition, payload in topic_payloads.items(): + message.append(struct.pack('>iqi', partition, payload.offset, + payload.max_bytes)) + + msg = b''.join(message) + return struct.pack('>i%ds' % len(msg), len(msg), msg) + + @classmethod + def decode_fetch_response(cls, data): + """ + Decode bytes to a FetchResponse + + Arguments: + data: bytes to decode + """ + ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + + for _ in range(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + + for j 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=None): + payloads = [] if payloads is None else payloads + grouped_payloads = group_by_topic_and_partition(payloads) + + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.OFFSET_KEY)) + + # -1 is the replica id + message.append(struct.pack('>ii', -1, len(grouped_payloads))) + + for topic, topic_payloads in grouped_payloads.items(): + message.append(write_short_string(topic)) + message.append(struct.pack('>i', len(topic_payloads))) + + for partition, payload in topic_payloads.items(): + message.append(struct.pack('>iqi', partition, payload.time, + payload.max_offsets)) + + msg = b''.join(message) + return struct.pack('>i%ds' % len(msg), len(msg), msg) + + @classmethod + def decode_offset_response(cls, data): + """ + Decode bytes to an OffsetResponse + + Arguments: + data: bytes to decode + """ + ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + + for _ in range(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + + for _ in range(num_partitions): + ((partition, error, num_offsets,), cur) = \ + relative_unpack('>ihi', data, cur) + + offsets = [] + for k 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=None, + payloads=None): + """ + Encode a MetadataRequest + + Arguments: + client_id: string + correlation_id: int + topics: list of strings + """ + if payloads is None: + topics = [] if topics is None else topics + else: + topics = payloads + + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.METADATA_KEY)) + + message.append(struct.pack('>i', len(topics))) + + for topic in topics: + message.append(struct.pack('>h%ds' % len(topic), len(topic), topic)) + + msg = b''.join(message) + return write_int_string(msg) + + @classmethod + def decode_metadata_response(cls, data): + """ + Decode bytes to a MetadataResponse + + Arguments: + data: bytes to decode + """ + ((correlation_id, numbrokers), cur) = relative_unpack('>ii', data, 0) + + # Broker info + brokers = [] + for _ 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.append(BrokerMetadata(nodeId, host, port)) + + # Topic info + ((num_topics,), cur) = relative_unpack('>i', data, cur) + topic_metadata = [] + + for _ in range(num_topics): + ((topic_error,), cur) = relative_unpack('>h', data, cur) + (topic_name, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + partition_metadata = [] + + for _ in range(num_partitions): + ((partition_error_code, partition, leader, numReplicas), cur) = \ + relative_unpack('>hiii', data, cur) + + (replicas, cur) = relative_unpack( + '>%di' % numReplicas, data, cur) + + ((num_isr,), cur) = relative_unpack('>i', data, cur) + (isr, cur) = relative_unpack('>%di' % num_isr, data, cur) + + partition_metadata.append( + PartitionMetadata(topic_name, partition, leader, + replicas, isr, partition_error_code) + ) + + topic_metadata.append( + TopicMetadata(topic_name, topic_error, partition_metadata) + ) + + return MetadataResponse(brokers, topic_metadata) + + @classmethod + def encode_consumer_metadata_request(cls, client_id, correlation_id, payloads): + """ + Encode a ConsumerMetadataRequest + + Arguments: + client_id: string + correlation_id: int + payloads: string (consumer group) + """ + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.CONSUMER_METADATA_KEY)) + message.append(struct.pack('>h%ds' % len(payloads), len(payloads), payloads)) + + msg = b''.join(message) + return write_int_string(msg) + + @classmethod + def decode_consumer_metadata_response(cls, data): + """ + Decode bytes to a ConsumerMetadataResponse + + Arguments: + data: bytes to decode + """ + ((correlation_id, error, nodeId), cur) = relative_unpack('>ihi', data, 0) + (host, cur) = read_short_string(data, cur) + ((port,), cur) = relative_unpack('>i', data, cur) + + return ConsumerMetadataResponse(error, nodeId, host, port) + + @classmethod + def encode_offset_commit_request(cls, client_id, correlation_id, + group, payloads): + """ + Encode some OffsetCommitRequest structs + + Arguments: + client_id: string + correlation_id: int + group: string, the consumer group you are committing offsets for + payloads: list of OffsetCommitRequest + """ + grouped_payloads = group_by_topic_and_partition(payloads) + + message = [] + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.OFFSET_COMMIT_KEY)) + message.append(write_short_string(group)) + message.append(struct.pack('>i', len(grouped_payloads))) + + for topic, topic_payloads in grouped_payloads.items(): + message.append(write_short_string(topic)) + message.append(struct.pack('>i', len(topic_payloads))) + + for partition, payload in topic_payloads.items(): + message.append(struct.pack('>iq', partition, payload.offset)) + message.append(write_short_string(payload.metadata)) + + msg = b''.join(message) + return struct.pack('>i%ds' % len(msg), len(msg), msg) + + @classmethod + def decode_offset_commit_response(cls, data): + """ + Decode bytes to an OffsetCommitResponse + + Arguments: + data: bytes to decode + """ + ((correlation_id,), cur) = relative_unpack('>i', data, 0) + ((num_topics,), cur) = relative_unpack('>i', data, cur) + + for _ in xrange(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + + for _ 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, from_kafka=False): + """ + Encode some OffsetFetchRequest structs. The request is encoded using + version 0 if from_kafka is false, indicating a request for Zookeeper + offsets. It is encoded using version 1 otherwise, indicating a request + for Kafka offsets. + + Arguments: + client_id: string + correlation_id: int + group: string, the consumer group you are fetching offsets for + payloads: list of OffsetFetchRequest + from_kafka: bool, default False, set True for Kafka-committed offsets + """ + grouped_payloads = group_by_topic_and_partition(payloads) + + message = [] + reqver = 1 if from_kafka else 0 + message.append(cls._encode_message_header(client_id, correlation_id, + KafkaProtocol.OFFSET_FETCH_KEY, + version=reqver)) + + message.append(write_short_string(group)) + message.append(struct.pack('>i', len(grouped_payloads))) + + for topic, topic_payloads in grouped_payloads.items(): + message.append(write_short_string(topic)) + message.append(struct.pack('>i', len(topic_payloads))) + + for partition, payload in topic_payloads.items(): + message.append(struct.pack('>i', partition)) + + msg = b''.join(message) + return struct.pack('>i%ds' % len(msg), len(msg), msg) + + @classmethod + def decode_offset_fetch_response(cls, data): + """ + Decode bytes to an OffsetFetchResponse + + Arguments: + data: bytes to decode + """ + + ((correlation_id,), cur) = relative_unpack('>i', data, 0) + ((num_topics,), cur) = relative_unpack('>i', data, cur) + + for _ in range(num_topics): + (topic, cur) = read_short_string(data, cur) + ((num_partitions,), cur) = relative_unpack('>i', data, cur) + + for _ 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) + + +def create_message(payload, key=None): + """ + Construct a Message + + Arguments: + payload: bytes, the payload to send to Kafka + key: bytes, a key used for partition routing (optional) + + """ + return Message(0, 0, key, payload) + + +def create_gzip_message(payloads, key=None, compresslevel=None): + """ + Construct a Gzipped Message containing multiple Messages + + The given payloads will be encoded, compressed, and sent as a single atomic + message to Kafka. + + Arguments: + 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( + [create_message(payload, pl_key) for payload, pl_key in payloads]) + + gzipped = gzip_encode(message_set, compresslevel=compresslevel) + codec = ATTRIBUTE_CODEC_MASK & CODEC_GZIP + + return Message(0, 0x00 | codec, key, gzipped) + + +def create_snappy_message(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. + + Arguments: + 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( + [create_message(payload, pl_key) for payload, pl_key in payloads]) + + snapped = snappy_encode(message_set) + codec = ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY + + return Message(0, 0x00 | codec, key, snapped) + + +def create_message_set(messages, codec=CODEC_NONE, key=None, compresslevel=None): + """Create a message set using the given codec. + + If codec is CODEC_NONE, return a list of raw Kafka messages. Otherwise, + return a list containing a single codec-encoded message. + """ + if codec == CODEC_NONE: + return [create_message(m, k) for m, k in messages] + elif codec == CODEC_GZIP: + return [create_gzip_message(messages, key, compresslevel)] + elif codec == CODEC_SNAPPY: + return [create_snappy_message(messages, key)] + else: + raise UnsupportedCodecError("Codec 0x%02x unsupported" % codec) -- cgit v1.2.1 From f6edeafac3f42f5407629dcfb1ddd4357dbf5445 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Nov 2015 04:15:19 +0800 Subject: Add base api type classes w/ encoders in kafka.protocol.types --- kafka/protocol/types.py | 55 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 kafka/protocol/types.py diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py new file mode 100644 index 0000000..6b257d3 --- /dev/null +++ b/kafka/protocol/types.py @@ -0,0 +1,55 @@ +from struct import pack + + +class AbstractField(object): + def __init__(self, name): + self.name = name + + +class Int8(AbstractField): + @classmethod + def encode(cls, value): + return pack('>b', value) + + +class Int16(AbstractField): + @classmethod + def encode(cls, value): + return pack('>h', value) + + +class Int32(AbstractField): + @classmethod + def encode(cls, value): + return pack('>i', value) + + +class Int64(AbstractField): + @classmethod + def encode(cls, value): + return pack('>q', value) + + +class String(AbstractField): + @classmethod + def encode(cls, value): + if value is None: + return Int16.encode(-1) + else: + return Int16.encode(len(value)) + value + + +class Bytes(AbstractField): + @classmethod + def encode(cls, value): + if value is None: + return Int32.encode(-1) + else: + return Int32.encode(len(value)) + value + + +class Array(object): + @classmethod + def encode(cls, values): + # Assume that values are already encoded + return Int32.encode(len(values)) + b''.join(values) -- cgit v1.2.1 From a0be374ce36f00ebb11a1e211ecee715999d9e8b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Nov 2015 04:24:28 +0800 Subject: Basic 0.8 Request protocol classes, with encoding only --- kafka/protocol/api.py | 355 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 355 insertions(+) create mode 100644 kafka/protocol/api.py diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py new file mode 100644 index 0000000..cbaf828 --- /dev/null +++ b/kafka/protocol/api.py @@ -0,0 +1,355 @@ +import struct + +from .types import ( + Int8, Int16, Int32, Int64, Bytes, String, Array +) +from ..util import crc32 + + +class Message(object): + MAGIC_BYTE = 0 + __slots__ = ('magic', 'attributes', 'key', 'value') + + def __init__(self, value, key=None, magic=0, attributes=0): + self.magic = magic + self.attributes = attributes + self.key = key + self.value = value + + def encode(self): + message = ( + Int8.encode(self.magic) + + Int8.encode(self.attributes) + + Bytes.encode(self.key) + + Bytes.encode(self.value) + ) + return ( + struct.pack('>I', crc32(message)) + + message + ) + + +class MessageSet(object): + + @staticmethod + def _encode_one(message): + encoded = message.encode() + return (Int64.encode(0) + Int32.encode(len(encoded)) + encoded) + + @staticmethod + def encode(messages): + return b''.join(map(MessageSet._encode_one, messages)) + + +class AbstractRequestResponse(object): + @classmethod + def encode(cls, message): + return Int32.encode(len(message)) + message + + +class AbstractRequest(AbstractRequestResponse): + @classmethod + def encode(cls, request, correlation_id=0, client_id='kafka-python'): + request = (Int16.encode(cls.API_KEY) + + Int16.encode(cls.API_VERSION) + + Int32.encode(correlation_id) + + String.encode(client_id) + + request) + return super(AbstractRequest, cls).encode(request) + + +class ProduceRequest(AbstractRequest): + API_KEY = 0 + API_VERSION = 0 + __slots__ = ('required_acks', 'timeout', 'topic_partition_messages', 'compression') + + def __init__(self, topic_partition_messages, + required_acks=-1, timeout=1000, compression=None): + """ + topic_partition_messages is a dict of dicts of lists (of messages) + { + "TopicFoo": { + 0: [ + Message('foo'), + Message('bar') + ], + 1: [ + Message('fizz'), + Message('buzz') + ] + } + } + """ + self.required_acks = required_acks + self.timeout = timeout + self.topic_partition_messages = topic_partition_messages + self.compression = compression + + @staticmethod + def _encode_messages(partition, messages, compression): + message_set = MessageSet.encode(messages) + + if compression: + # compress message_set data and re-encode as single message + # then wrap single compressed message in a new message_set + pass + + return (Int32.encode(partition) + + Int32.encode(len(message_set)) + + message_set) + + def encode(self): + request = ( + Int16.encode(self.required_acks) + + Int32.encode(self.timeout) + + Array.encode([( + String.encode(topic) + + Array.encode([ + self._encode_messages(partition, messages, self.compression) + for partition, messages in partitions.iteritems()]) + ) for topic, partitions in self.topic_partition_messages.iteritems()]) + ) + return super(ProduceRequest, self).encode(request) + + +class FetchRequest(AbstractRequest): + API_KEY = 1 + API_VERSION = 0 + __slots__ = ('replica_id', 'max_wait_time', 'min_bytes', 'topic_partition_offsets') + + def __init__(self, topic_partition_offsets, + max_wait_time=-1, min_bytes=0, replica_id=-1): + """ + topic_partition_offsets is a dict of dicts of (offset, max_bytes) tuples + { + "TopicFoo": { + 0: (1234, 1048576), + 1: (1324, 1048576) + } + } + """ + self.topic_partition_offsets = topic_partition_offsets + self.max_wait_time = max_wait_time + self.min_bytes = min_bytes + self.replica_id = replica_id + + def encode(self): + request = ( + Int32.encode(self.replica_id) + + Int32.encode(self.max_wait_time) + + Int32.encode(self.min_bytes) + + Array.encode([( + String.encode(topic) + + Array.encode([( + Int32.encode(partition) + + Int64.encode(offset) + + Int32.encode(max_bytes) + ) for partition, (offset, max_bytes) in partitions.iteritems()]) + ) for topic, partitions in self.topic_partition_offsets.iteritems()])) + return super(FetchRequest, self).encode(request) + + +class OffsetRequest(AbstractRequest): + API_KEY = 2 + API_VERSION = 0 + __slots__ = ('replica_id', 'topic_partition_times') + + def __init__(self, topic_partition_times, replica_id=-1): + """ + topic_partition_times is a dict of dicts of (time, max_offsets) tuples + { + "TopicFoo": { + 0: (-1, 1), + 1: (-1, 1) + } + } + """ + self.topic_partition_times = topic_partition_times + self.replica_id = replica_id + + def encode(self): + request = ( + Int32.encode(self.replica_id) + + Array.encode([( + String.encode(topic) + + Array.encode([( + Int32.encode(partition) + + Int64.encode(time) + + Int32.encode(max_offsets) + ) for partition, (time, max_offsets) in partitions.iteritems()]) + ) for topic, partitions in self.topic_partition_times.iteritems()])) + return super(OffsetRequest, self).encode(request) + + +class MetadataRequest(AbstractRequest): + API_KEY = 3 + API_VERSION = 0 + __slots__ = ('topics') + + def __init__(self, *topics): + self.topics = topics + + def encode(self): + request = Array.encode(map(String.encode, self.topics)) + return super(MetadataRequest, self).encode(request) + + +# Non-user facing control APIs 4-7 + + +class OffsetCommitRequestV0(AbstractRequest): + API_KEY = 8 + API_VERSION = 0 + __slots__ = ('consumer_group_id', 'offsets') + + def __init__(self, consumer_group_id, offsets): + """ + offsets is a dict of dicts of (offset, metadata) tuples + { + "TopicFoo": { + 0: (1234, ""), + 1: (1243, "") + } + } + """ + self.consumer_group_id = consumer_group_id + self.offsets = offsets + + def encode(self): + request = ( + String.encode(self.consumer_group_id) + + Array.encode([( + String.encode(topic) + + Array.encode([( + Int32.encode(partition) + + Int64.encode(offset) + + String.encode(metadata) + ) for partition, (offset, metadata) in partitions.iteritems()]) + ) for topic, partitions in self.offsets.iteritems()])) + return super(OffsetCommitRequestV0, self).encode(request) + + +class OffsetCommitRequestV1(AbstractRequest): + API_KEY = 8 + API_VERSION = 1 + __slots__ = ('consumer_group_id', 'consumer_group_generation_id', + 'consumer_id', 'offsets') + + def __init__(self, consumer_group_id, consumer_group_generation_id, + consumer_id, offsets): + """ + offsets is a dict of dicts of (offset, timestamp, metadata) tuples + { + "TopicFoo": { + 0: (1234, 1448198827, ""), + 1: (1243, 1448198827, "") + } + } + """ + self.consumer_group_id = consumer_group_id + self.consumer_group_generation_id = consumer_group_generation_id + self.consumer_id = consumer_id + self.offsets = offsets + + def encode(self): + request = ( + String.encode(self.consumer_group_id) + + Int32.encode(self.consumer_group_generation_id) + + String.encode(self.consumer_id) + + Array.encode([( + String.encode(topic) + + Array.encode([( + Int32.encode(partition) + + Int64.encode(offset) + + Int64.encode(timestamp) + + String.encode(metadata) + ) for partition, (offset, timestamp, metadata) in partitions.iteritems()]) + ) for topic, partitions in self.offsets.iteritems()])) + return super(OffsetCommitRequestV1, self).encode(request) + + +class OffsetCommitRequest(AbstractRequest): + API_KEY = 8 + API_VERSION = 2 + __slots__ = ('consumer_group_id', 'consumer_group_generation_id', + 'consumer_id', 'retention_time', 'offsets') + + def __init__(self, consumer_group_id, consumer_group_generation_id, + consumer_id, retention_time, offsets): + """ + offsets is a dict of dicts of (offset, metadata) tuples + { + "TopicFoo": { + 0: (1234, ""), + 1: (1243, "") + } + } + """ + self.consumer_group_id = consumer_group_id + self.consumer_group_generation_id = consumer_group_generation_id + self.consumer_id = consumer_id + self.retention_time = retention_time + self.offsets = offsets + + def encode(self): + request = ( + String.encode(self.consumer_group_id) + + Int32.encode(self.consumer_group_generation_id) + + String.encode(self.consumer_id) + + Int64.encode(self.retention_time) + + Array.encode([( + String.encode(topic) + + Array.encode([( + Int32.encode(partition) + + Int64.encode(offset) + + String.encode(metadata) + ) for partition, (offset, timestamp, metadata) in partitions.iteritems()]) + ) for topic, partitions in self.offsets.iteritems()])) + return super(OffsetCommitRequest, self).encode(request) + + +class OffsetFetchRequestV0(AbstractRequest): + API_KEY = 9 + API_VERSION = 0 + __slots__ = ('consumer_group', 'topic_partitions') + + def __init__(self, consumer_group, topic_partitions): + """ + offsets is a dict of lists of partition ints + { + "TopicFoo": [0, 1, 2] + } + """ + self.consumer_group = consumer_group + self.topic_partitions = topic_partitions + + def encode(self): + request = ( + String.encode(self.consumer_group) + + Array.encode([( + String.encode(topic) + + Array.encode([Int32.encode(partition) for partition in partitions]) + ) for topic, partitions in self.topic_partitions.iteritems()]) + ) + return super(OffsetFetchRequest, self).encode(request) + + +class OffsetFetchRequest(OffsetFetchRequestV0): + """Identical to V0, but offsets fetched from kafka storage not zookeeper""" + API_VERSION = 1 + + +class GroupCoordinatorRequest(AbstractRequest): + API_KEY = 10 + API_VERSION = 0 + __slots__ = ('group_id',) + + def __init__(self, group_id): + self.group_id = group_id + + def encode(self): + request = String.encode(self.group_id) + return super(GroupCoordinatorRequest, self).encode(request) + + + -- cgit v1.2.1 From 3f65ff4ab93f2282af442e6bb5e54e3af1d602db Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Nov 2015 05:26:13 +0800 Subject: Move ProduceRequest to kafka.protocol.produce --- kafka/protocol/api.py | 54 ------------------------------------------- kafka/protocol/produce.py | 59 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+), 54 deletions(-) create mode 100644 kafka/protocol/produce.py diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index cbaf828..8ea820b 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -58,60 +58,6 @@ class AbstractRequest(AbstractRequestResponse): return super(AbstractRequest, cls).encode(request) -class ProduceRequest(AbstractRequest): - API_KEY = 0 - API_VERSION = 0 - __slots__ = ('required_acks', 'timeout', 'topic_partition_messages', 'compression') - - def __init__(self, topic_partition_messages, - required_acks=-1, timeout=1000, compression=None): - """ - topic_partition_messages is a dict of dicts of lists (of messages) - { - "TopicFoo": { - 0: [ - Message('foo'), - Message('bar') - ], - 1: [ - Message('fizz'), - Message('buzz') - ] - } - } - """ - self.required_acks = required_acks - self.timeout = timeout - self.topic_partition_messages = topic_partition_messages - self.compression = compression - - @staticmethod - def _encode_messages(partition, messages, compression): - message_set = MessageSet.encode(messages) - - if compression: - # compress message_set data and re-encode as single message - # then wrap single compressed message in a new message_set - pass - - return (Int32.encode(partition) + - Int32.encode(len(message_set)) + - message_set) - - def encode(self): - request = ( - Int16.encode(self.required_acks) + - Int32.encode(self.timeout) + - Array.encode([( - String.encode(topic) + - Array.encode([ - self._encode_messages(partition, messages, self.compression) - for partition, messages in partitions.iteritems()]) - ) for topic, partitions in self.topic_partition_messages.iteritems()]) - ) - return super(ProduceRequest, self).encode(request) - - class FetchRequest(AbstractRequest): API_KEY = 1 API_VERSION = 0 diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py new file mode 100644 index 0000000..b875397 --- /dev/null +++ b/kafka/protocol/produce.py @@ -0,0 +1,59 @@ +from .api import AbstractRequest, AbstractResponse, MessageSet +from .types import Int8, Int16, Int32, Int64, Bytes, String, Array + + +class ProduceRequest(AbstractRequest): + API_KEY = 0 + API_VERSION = 0 + __slots__ = ('required_acks', 'timeout', 'topic_partition_messages', 'compression') + + def __init__(self, topic_partition_messages, + required_acks=-1, timeout=1000, compression=None): + """ + topic_partition_messages is a dict of dicts of lists (of messages) + { + "TopicFoo": { + 0: [ + Message('foo'), + Message('bar') + ], + 1: [ + Message('fizz'), + Message('buzz') + ] + } + } + """ + self.required_acks = required_acks + self.timeout = timeout + self.topic_partition_messages = topic_partition_messages + self.compression = compression + + @staticmethod + def _encode_messages(partition, messages, compression): + message_set = MessageSet.encode(messages) + + if compression: + # compress message_set data and re-encode as single message + # then wrap single compressed message in a new message_set + pass + + return (Int32.encode(partition) + + Int32.encode(len(message_set)) + + message_set) + + def encode(self): + request = ( + Int16.encode(self.required_acks) + + Int32.encode(self.timeout) + + Array.encode([( + String.encode(topic) + + Array.encode([ + self._encode_messages(partition, messages, self.compression) + for partition, messages in partitions.iteritems()]) + ) for topic, partitions in self.topic_partition_messages.iteritems()]) + ) + return super(ProduceRequest, self).encode(request) + + + -- cgit v1.2.1 From dc94b5fe9f3f93bf6f2235d7f65c62fcf0a2a996 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 28 Nov 2015 19:29:32 +0800 Subject: Fix tests broken by legacy module move --- kafka/protocol/legacy.py | 2 ++ test/test_protocol.py | 6 +++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py index 412a957..d7ac50a 100644 --- a/kafka/protocol/legacy.py +++ b/kafka/protocol/legacy.py @@ -1,3 +1,5 @@ +from __future__ import absolute_import + import logging import struct diff --git a/test/test_protocol.py b/test/test_protocol.py index ac7bea6..368c2d0 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -780,11 +780,11 @@ class TestProtocol(unittest.TestCase): @contextmanager def mock_create_message_fns(self): import kafka.protocol - with patch.object(kafka.protocol, "create_message", + with patch.object(kafka.protocol.legacy, "create_message", return_value=sentinel.message): - with patch.object(kafka.protocol, "create_gzip_message", + with patch.object(kafka.protocol.legacy, "create_gzip_message", return_value=sentinel.gzip_message): - with patch.object(kafka.protocol, "create_snappy_message", + with patch.object(kafka.protocol.legacy, "create_snappy_message", return_value=sentinel.snappy_message): yield -- cgit v1.2.1 From f08775a6198cd16a7bc9ec93ffd057f65064ec54 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 28 Nov 2015 19:34:37 +0800 Subject: Switch crc32 back to signed integer -- this is consistent with protocol encoding spec --- kafka/protocol/legacy.py | 4 ++-- kafka/util.py | 8 +++++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py index d7ac50a..db9f3e0 100644 --- a/kafka/protocol/legacy.py +++ b/kafka/protocol/legacy.py @@ -109,7 +109,7 @@ class KafkaProtocol(object): write_int_string(message.value) ]) crc = crc32(msg) - msg = struct.pack('>I%ds' % len(msg), crc, msg) + msg = struct.pack('>i%ds' % len(msg), crc, msg) else: raise ProtocolError("Unexpected magic number: %d" % message.magic) return msg @@ -159,7 +159,7 @@ class KafkaProtocol(object): 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) + ((crc, magic, att), cur) = relative_unpack('>iBB', data, 0) if crc != crc32(data[4:]): raise ChecksumError("Message checksum failed") diff --git a/kafka/util.py b/kafka/util.py index 6d9d307..e95d51d 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -10,7 +10,13 @@ from kafka.common import BufferUnderflowError def crc32(data): - return binascii.crc32(data) & 0xffffffff + crc = binascii.crc32(data) + # py2 and py3 behave a little differently + # CRC is encoded as a signed int in kafka protocol + # so we'll convert the py3 unsigned result to signed + if six.PY3 and crc >= 2**31: + crc -= 2**32 + return crc def write_int_string(s): -- cgit v1.2.1 From e24a4d5f5252d6f97ac586e328b95779ef83f4b6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 28 Nov 2015 19:40:07 +0800 Subject: Fix __init__ legacy relative module import --- kafka/protocol/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/protocol/__init__.py b/kafka/protocol/__init__.py index 39cb64a..7b2a2f3 100644 --- a/kafka/protocol/__init__.py +++ b/kafka/protocol/__init__.py @@ -1,4 +1,4 @@ -from legacy import ( +from .legacy import ( create_message, create_gzip_message, create_snappy_message, create_message_set, CODEC_NONE, CODEC_GZIP, CODEC_SNAPPY, ALL_CODECS, -- cgit v1.2.1 From a85e09df89a43de5b659a0fa4ed35bec37c60e04 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 28 Nov 2015 19:41:06 +0800 Subject: Rework protocol type definition: AbstractType, Schema, Struct --- kafka/protocol/abstract.py | 13 ++ kafka/protocol/api.py | 309 ++------------------------------------------- kafka/protocol/commit.py | 111 ++++++++++++++++ kafka/protocol/fetch.py | 30 +++++ kafka/protocol/message.py | 67 ++++++++++ kafka/protocol/metadata.py | 28 ++++ kafka/protocol/offset.py | 32 +++++ kafka/protocol/produce.py | 81 ++++-------- kafka/protocol/struct.py | 52 ++++++++ kafka/protocol/types.py | 109 +++++++++++++--- 10 files changed, 461 insertions(+), 371 deletions(-) create mode 100644 kafka/protocol/abstract.py create mode 100644 kafka/protocol/commit.py create mode 100644 kafka/protocol/fetch.py create mode 100644 kafka/protocol/message.py create mode 100644 kafka/protocol/metadata.py create mode 100644 kafka/protocol/offset.py create mode 100644 kafka/protocol/struct.py diff --git a/kafka/protocol/abstract.py b/kafka/protocol/abstract.py new file mode 100644 index 0000000..9c53c8c --- /dev/null +++ b/kafka/protocol/abstract.py @@ -0,0 +1,13 @@ +import abc + + +class AbstractType(object): + __metaclass__ = abc.ABCMeta + + @abc.abstractmethod + def encode(cls, value): + pass + + @abc.abstractmethod + def decode(cls, data): + pass diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index 8ea820b..0c23437 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -1,301 +1,16 @@ -import struct +from .struct import Struct +from .types import Int16, Int32, String, Schema -from .types import ( - Int8, Int16, Int32, Int64, Bytes, String, Array -) -from ..util import crc32 +class RequestHeader(Struct): + SCHEMA = Schema( + ('api_key', Int16), + ('api_version', Int16), + ('correlation_id', Int32), + ('client_id', String('utf-8')) + ) -class Message(object): - MAGIC_BYTE = 0 - __slots__ = ('magic', 'attributes', 'key', 'value') - - def __init__(self, value, key=None, magic=0, attributes=0): - self.magic = magic - self.attributes = attributes - self.key = key - self.value = value - - def encode(self): - message = ( - Int8.encode(self.magic) + - Int8.encode(self.attributes) + - Bytes.encode(self.key) + - Bytes.encode(self.value) + def __init__(self, request, correlation_id=0, client_id='kafka-python'): + super(RequestHeader, self).__init__( + request.API_KEY, request.API_VERSION, correlation_id, client_id ) - return ( - struct.pack('>I', crc32(message)) + - message - ) - - -class MessageSet(object): - - @staticmethod - def _encode_one(message): - encoded = message.encode() - return (Int64.encode(0) + Int32.encode(len(encoded)) + encoded) - - @staticmethod - def encode(messages): - return b''.join(map(MessageSet._encode_one, messages)) - - -class AbstractRequestResponse(object): - @classmethod - def encode(cls, message): - return Int32.encode(len(message)) + message - - -class AbstractRequest(AbstractRequestResponse): - @classmethod - def encode(cls, request, correlation_id=0, client_id='kafka-python'): - request = (Int16.encode(cls.API_KEY) + - Int16.encode(cls.API_VERSION) + - Int32.encode(correlation_id) + - String.encode(client_id) + - request) - return super(AbstractRequest, cls).encode(request) - - -class FetchRequest(AbstractRequest): - API_KEY = 1 - API_VERSION = 0 - __slots__ = ('replica_id', 'max_wait_time', 'min_bytes', 'topic_partition_offsets') - - def __init__(self, topic_partition_offsets, - max_wait_time=-1, min_bytes=0, replica_id=-1): - """ - topic_partition_offsets is a dict of dicts of (offset, max_bytes) tuples - { - "TopicFoo": { - 0: (1234, 1048576), - 1: (1324, 1048576) - } - } - """ - self.topic_partition_offsets = topic_partition_offsets - self.max_wait_time = max_wait_time - self.min_bytes = min_bytes - self.replica_id = replica_id - - def encode(self): - request = ( - Int32.encode(self.replica_id) + - Int32.encode(self.max_wait_time) + - Int32.encode(self.min_bytes) + - Array.encode([( - String.encode(topic) + - Array.encode([( - Int32.encode(partition) + - Int64.encode(offset) + - Int32.encode(max_bytes) - ) for partition, (offset, max_bytes) in partitions.iteritems()]) - ) for topic, partitions in self.topic_partition_offsets.iteritems()])) - return super(FetchRequest, self).encode(request) - - -class OffsetRequest(AbstractRequest): - API_KEY = 2 - API_VERSION = 0 - __slots__ = ('replica_id', 'topic_partition_times') - - def __init__(self, topic_partition_times, replica_id=-1): - """ - topic_partition_times is a dict of dicts of (time, max_offsets) tuples - { - "TopicFoo": { - 0: (-1, 1), - 1: (-1, 1) - } - } - """ - self.topic_partition_times = topic_partition_times - self.replica_id = replica_id - - def encode(self): - request = ( - Int32.encode(self.replica_id) + - Array.encode([( - String.encode(topic) + - Array.encode([( - Int32.encode(partition) + - Int64.encode(time) + - Int32.encode(max_offsets) - ) for partition, (time, max_offsets) in partitions.iteritems()]) - ) for topic, partitions in self.topic_partition_times.iteritems()])) - return super(OffsetRequest, self).encode(request) - - -class MetadataRequest(AbstractRequest): - API_KEY = 3 - API_VERSION = 0 - __slots__ = ('topics') - - def __init__(self, *topics): - self.topics = topics - - def encode(self): - request = Array.encode(map(String.encode, self.topics)) - return super(MetadataRequest, self).encode(request) - - -# Non-user facing control APIs 4-7 - - -class OffsetCommitRequestV0(AbstractRequest): - API_KEY = 8 - API_VERSION = 0 - __slots__ = ('consumer_group_id', 'offsets') - - def __init__(self, consumer_group_id, offsets): - """ - offsets is a dict of dicts of (offset, metadata) tuples - { - "TopicFoo": { - 0: (1234, ""), - 1: (1243, "") - } - } - """ - self.consumer_group_id = consumer_group_id - self.offsets = offsets - - def encode(self): - request = ( - String.encode(self.consumer_group_id) + - Array.encode([( - String.encode(topic) + - Array.encode([( - Int32.encode(partition) + - Int64.encode(offset) + - String.encode(metadata) - ) for partition, (offset, metadata) in partitions.iteritems()]) - ) for topic, partitions in self.offsets.iteritems()])) - return super(OffsetCommitRequestV0, self).encode(request) - - -class OffsetCommitRequestV1(AbstractRequest): - API_KEY = 8 - API_VERSION = 1 - __slots__ = ('consumer_group_id', 'consumer_group_generation_id', - 'consumer_id', 'offsets') - - def __init__(self, consumer_group_id, consumer_group_generation_id, - consumer_id, offsets): - """ - offsets is a dict of dicts of (offset, timestamp, metadata) tuples - { - "TopicFoo": { - 0: (1234, 1448198827, ""), - 1: (1243, 1448198827, "") - } - } - """ - self.consumer_group_id = consumer_group_id - self.consumer_group_generation_id = consumer_group_generation_id - self.consumer_id = consumer_id - self.offsets = offsets - - def encode(self): - request = ( - String.encode(self.consumer_group_id) + - Int32.encode(self.consumer_group_generation_id) + - String.encode(self.consumer_id) + - Array.encode([( - String.encode(topic) + - Array.encode([( - Int32.encode(partition) + - Int64.encode(offset) + - Int64.encode(timestamp) + - String.encode(metadata) - ) for partition, (offset, timestamp, metadata) in partitions.iteritems()]) - ) for topic, partitions in self.offsets.iteritems()])) - return super(OffsetCommitRequestV1, self).encode(request) - - -class OffsetCommitRequest(AbstractRequest): - API_KEY = 8 - API_VERSION = 2 - __slots__ = ('consumer_group_id', 'consumer_group_generation_id', - 'consumer_id', 'retention_time', 'offsets') - - def __init__(self, consumer_group_id, consumer_group_generation_id, - consumer_id, retention_time, offsets): - """ - offsets is a dict of dicts of (offset, metadata) tuples - { - "TopicFoo": { - 0: (1234, ""), - 1: (1243, "") - } - } - """ - self.consumer_group_id = consumer_group_id - self.consumer_group_generation_id = consumer_group_generation_id - self.consumer_id = consumer_id - self.retention_time = retention_time - self.offsets = offsets - - def encode(self): - request = ( - String.encode(self.consumer_group_id) + - Int32.encode(self.consumer_group_generation_id) + - String.encode(self.consumer_id) + - Int64.encode(self.retention_time) + - Array.encode([( - String.encode(topic) + - Array.encode([( - Int32.encode(partition) + - Int64.encode(offset) + - String.encode(metadata) - ) for partition, (offset, timestamp, metadata) in partitions.iteritems()]) - ) for topic, partitions in self.offsets.iteritems()])) - return super(OffsetCommitRequest, self).encode(request) - - -class OffsetFetchRequestV0(AbstractRequest): - API_KEY = 9 - API_VERSION = 0 - __slots__ = ('consumer_group', 'topic_partitions') - - def __init__(self, consumer_group, topic_partitions): - """ - offsets is a dict of lists of partition ints - { - "TopicFoo": [0, 1, 2] - } - """ - self.consumer_group = consumer_group - self.topic_partitions = topic_partitions - - def encode(self): - request = ( - String.encode(self.consumer_group) + - Array.encode([( - String.encode(topic) + - Array.encode([Int32.encode(partition) for partition in partitions]) - ) for topic, partitions in self.topic_partitions.iteritems()]) - ) - return super(OffsetFetchRequest, self).encode(request) - - -class OffsetFetchRequest(OffsetFetchRequestV0): - """Identical to V0, but offsets fetched from kafka storage not zookeeper""" - API_VERSION = 1 - - -class GroupCoordinatorRequest(AbstractRequest): - API_KEY = 10 - API_VERSION = 0 - __slots__ = ('group_id',) - - def __init__(self, group_id): - self.group_id = group_id - - def encode(self): - request = String.encode(self.group_id) - return super(GroupCoordinatorRequest, self).encode(request) - - - diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py new file mode 100644 index 0000000..5ba0227 --- /dev/null +++ b/kafka/protocol/commit.py @@ -0,0 +1,111 @@ +from .struct import Struct +from .types import Array, Int16, Int32, Int64, Schema, String + + +class OffsetCommitRequest_v2(Struct): + API_KEY = 8 + API_VERSION = 2 # added retention_time, dropped timestamp + SCHEMA = Schema( + ('consumer_group', String('utf-8')), + ('consumer_group_generation_id', Int32), + ('consumer_id', String('utf-8')), + ('retention_time', Int64), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('metadata', String('utf-8')))))) + ) + + +class OffsetCommitRequest_v1(Struct): + API_KEY = 8 + API_VERSION = 1 # Kafka-backed storage + SCHEMA = Schema( + ('consumer_group', String('utf-8')), + ('consumer_group_generation_id', Int32), + ('consumer_id', String('utf-8')), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('timestamp', Int64), + ('metadata', String('utf-8')))))) + ) + + +class OffsetCommitRequest_v0(Struct): + API_KEY = 8 + API_VERSION = 0 # Zookeeper-backed storage + SCHEMA = Schema( + ('consumer_group', String('utf-8')), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('metadata', String('utf-8')))))) + ) + + +class OffsetCommitResponse(Struct): + SCHEMA = Schema( + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16))))) + ) + + +class OffsetFetchRequest_v1(Struct): + API_KEY = 9 + API_VERSION = 1 # kafka-backed storage + SCHEMA = Schema( + ('consumer_group', String('utf-8')), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array(Int32)))) + ) + + +class OffsetFetchRequest_v0(Struct): + API_KEY = 9 + API_VERSION = 0 # zookeeper-backed storage + SCHEMA = Schema( + ('consumer_group', String('utf-8')), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array(Int32)))) + ) + + +class OffsetFetchResponse(Struct): + SCHEMA = Schema( + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('metadata', String('utf-8')), + ('error_code', Int16))))) + ) + + +class GroupCoordinatorRequest(Struct): + API_KEY = 10 + API_VERSION = 0 + SCHEMA = Schema( + ('consumer_group', String('utf-8')) + ) + + +class GroupCoordinatorResponse(Struct): + SCHEMA = Schema( + ('error_code', Int16), + ('coordinator_id', Int32), + ('host', String('utf-8')), + ('port', Int32) + ) diff --git a/kafka/protocol/fetch.py b/kafka/protocol/fetch.py new file mode 100644 index 0000000..c6d60cc --- /dev/null +++ b/kafka/protocol/fetch.py @@ -0,0 +1,30 @@ +from .message import MessageSet +from .struct import Struct +from .types import Array, Int16, Int32, Int64, Schema, String + + +class FetchRequest(Struct): + API_KEY = 1 + API_VERSION = 0 + SCHEMA = Schema( + ('replica_id', Int32), + ('max_wait_time', Int32), + ('min_bytes', Int32), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('max_bytes', Int32))))) + ) + +class FetchResponse(Struct): + SCHEMA = Schema( + ('topics', Array( + ('topics', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16), + ('highwater_offset', Int64), + ('message_set', MessageSet))))) + ) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py new file mode 100644 index 0000000..26f5ef6 --- /dev/null +++ b/kafka/protocol/message.py @@ -0,0 +1,67 @@ +from .struct import Struct +from .types import ( + Int8, Int16, Int32, Int64, Bytes, String, Array, Schema, AbstractType +) +from ..util import crc32 + + +class Message(Struct): + SCHEMA = Schema( + ('crc', Int32), + ('magic', Int8), + ('attributes', Int8), + ('key', Bytes), + ('value', Bytes) + ) + + def __init__(self, value, key=None, magic=0, attributes=0, crc=0): + self.crc = crc + self.magic = magic + self.attributes = attributes + self.key = key + self.value = value + self.encode = self._encode_self + + def _encode_self(self, recalc_crc=True): + message = Message.SCHEMA.encode( + (self.crc, self.magic, self.attributes, self.key, self.value) + ) + if not recalc_crc: + return message + self.crc = crc32(message[4:]) + return self.SCHEMA.fields[0].encode(self.crc) + message[4:] + + +class MessageSet(AbstractType): + ITEM = Schema( + ('offset', Int64), + ('message_size', Int32), + ('message', Message.SCHEMA) + ) + + @classmethod + def encode(cls, items, size=True, recalc_message_size=True): + encoded_values = [] + for (offset, message_size, message) in items: + if isinstance(message, Message): + encoded_message = message.encode() + else: + encoded_message = cls.ITEM.fields[2].encode(message) + if recalc_message_size: + message_size = len(encoded_message) + encoded_values.append(cls.ITEM.fields[0].encode(offset)) + encoded_values.append(cls.ITEM.fields[1].encode(message_size)) + encoded_values.append(encoded_message) + encoded = b''.join(encoded_values) + if not size: + return encoded + return Int32.encode(len(encoded)) + encoded + + @classmethod + def decode(cls, data): + size = Int32.decode(data) + end = data.tell() + size + items = [] + while data.tell() < end: + items.append(cls.ITEM.decode(data)) + return items diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py new file mode 100644 index 0000000..b35e7ef --- /dev/null +++ b/kafka/protocol/metadata.py @@ -0,0 +1,28 @@ +from .struct import Struct +from .types import Array, Int16, Int32, Schema, String + + +class MetadataRequest(Struct): + API_KEY = 3 + API_VERSION = 0 + SCHEMA = Schema( + ('topics', Array(String('utf-8'))) + ) + + +class MetadataResponse(Struct): + SCHEMA = Schema( + ('brokers', Array( + ('node_id', Int32), + ('host', String('utf-8')), + ('port', Int32))), + ('topics', Array( + ('error_code', Int16), + ('topic', String('utf-8')), + ('partitions', Array( + ('error_code', Int16), + ('partition', Int32), + ('leader', Int32), + ('replicas', Array(Int32)), + ('isr', Array(Int32)))))) + ) diff --git a/kafka/protocol/offset.py b/kafka/protocol/offset.py new file mode 100644 index 0000000..942bdbf --- /dev/null +++ b/kafka/protocol/offset.py @@ -0,0 +1,32 @@ +from .struct import Struct +from .types import Array, Int16, Int32, Int64, Schema, String + + +class OffsetRequest(Struct): + API_KEY = 2 + API_VERSION = 0 + SCHEMA = Schema( + ('replica_id', Int32), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('time', Int64), + ('max_offsets', Int32))))) + ) + DEFAULTS = { + 'replica_id': -1 + } + + +class OffsetResponse(Struct): + API_KEY = 2 + API_VERSION = 0 + SCHEMA = Schema( + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16), + ('offsets', Array(Int64)))))) + ) diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py index b875397..532a702 100644 --- a/kafka/protocol/produce.py +++ b/kafka/protocol/produce.py @@ -1,59 +1,30 @@ -from .api import AbstractRequest, AbstractResponse, MessageSet -from .types import Int8, Int16, Int32, Int64, Bytes, String, Array +from .message import MessageSet +from .struct import Struct +from .types import Int8, Int16, Int32, Int64, Bytes, String, Array, Schema -class ProduceRequest(AbstractRequest): +class ProduceRequest(Struct): API_KEY = 0 API_VERSION = 0 - __slots__ = ('required_acks', 'timeout', 'topic_partition_messages', 'compression') - - def __init__(self, topic_partition_messages, - required_acks=-1, timeout=1000, compression=None): - """ - topic_partition_messages is a dict of dicts of lists (of messages) - { - "TopicFoo": { - 0: [ - Message('foo'), - Message('bar') - ], - 1: [ - Message('fizz'), - Message('buzz') - ] - } - } - """ - self.required_acks = required_acks - self.timeout = timeout - self.topic_partition_messages = topic_partition_messages - self.compression = compression - - @staticmethod - def _encode_messages(partition, messages, compression): - message_set = MessageSet.encode(messages) - - if compression: - # compress message_set data and re-encode as single message - # then wrap single compressed message in a new message_set - pass - - return (Int32.encode(partition) + - Int32.encode(len(message_set)) + - message_set) - - def encode(self): - request = ( - Int16.encode(self.required_acks) + - Int32.encode(self.timeout) + - Array.encode([( - String.encode(topic) + - Array.encode([ - self._encode_messages(partition, messages, self.compression) - for partition, messages in partitions.iteritems()]) - ) for topic, partitions in self.topic_partition_messages.iteritems()]) - ) - return super(ProduceRequest, self).encode(request) - - - + SCHEMA = Schema( + ('required_acks', Int16), + ('timeout', Int32), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('messages', MessageSet))))) + ) + + +class ProduceResponse(Struct): + API_KEY = 0 + API_VERSION = 0 + SCHEMA = Schema( + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16), + ('offset', Int64))))) + ) diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py new file mode 100644 index 0000000..77f5fe7 --- /dev/null +++ b/kafka/protocol/struct.py @@ -0,0 +1,52 @@ +from collections import namedtuple +from io import BytesIO + +from .abstract import AbstractType +from .types import Schema + + +class Struct(AbstractType): + SCHEMA = Schema() + + def __init__(self, *args, **kwargs): + if len(args) == len(self.SCHEMA.fields): + for i, name in enumerate(self.SCHEMA.names): + self.__dict__[name] = args[i] + elif len(args) > 0: + raise ValueError('Args must be empty or mirror schema') + else: + self.__dict__.update(kwargs) + + # overloading encode() to support both class and instance + self.encode = self._encode_self + + @classmethod + def encode(cls, item): + bits = [] + for i, field in enumerate(cls.SCHEMA.fields): + bits.append(field.encode(item[i])) + return b''.join(bits) + + def _encode_self(self): + return self.SCHEMA.encode( + [self.__dict__[name] for name in self.SCHEMA.names] + ) + + @classmethod + def decode(cls, data): + if isinstance(data, bytes): + data = BytesIO(data) + return cls(*[field.decode(data) for field in cls.SCHEMA.fields]) + + def __repr__(self): + key_vals =['%s=%r' % (name, self.__dict__[name]) + for name in self.SCHEMA.names] + return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' + +""" +class MetaStruct(type): + def __new__(cls, clsname, bases, dct): + nt = namedtuple(clsname, [name for (name, _) in dct['SCHEMA']]) + bases = tuple([Struct, nt] + list(bases)) + return super(MetaStruct, cls).__new__(cls, clsname, bases, dct) +""" diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 6b257d3..5aa2e41 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -1,45 +1,73 @@ -from struct import pack +from __future__ import absolute_import +import abc +from struct import pack, unpack -class AbstractField(object): - def __init__(self, name): - self.name = name +from .abstract import AbstractType -class Int8(AbstractField): +class Int8(AbstractType): @classmethod def encode(cls, value): return pack('>b', value) + @classmethod + def decode(cls, data): + (value,) = unpack('>b', data.read(1)) + return value + -class Int16(AbstractField): +class Int16(AbstractType): @classmethod def encode(cls, value): return pack('>h', value) + @classmethod + def decode(cls, data): + (value,) = unpack('>h', data.read(2)) + return value -class Int32(AbstractField): + +class Int32(AbstractType): @classmethod def encode(cls, value): return pack('>i', value) + @classmethod + def decode(cls, data): + (value,) = unpack('>i', data.read(4)) + return value + -class Int64(AbstractField): +class Int64(AbstractType): @classmethod def encode(cls, value): return pack('>q', value) - -class String(AbstractField): @classmethod - def encode(cls, value): + def decode(cls, data): + (value,) = unpack('>q', data.read(8)) + return value + + +class String(AbstractType): + def __init__(self, encoding='utf-8'): + self.encoding = encoding + + def encode(self, value): if value is None: return Int16.encode(-1) - else: - return Int16.encode(len(value)) + value + value = str(value).encode(self.encoding) + return Int16.encode(len(value)) + value + + def decode(self, data): + length = Int16.decode(data) + if length < 0: + return None + return data.read(length).decode(self.encoding) -class Bytes(AbstractField): +class Bytes(AbstractType): @classmethod def encode(cls, value): if value is None: @@ -47,9 +75,52 @@ class Bytes(AbstractField): else: return Int32.encode(len(value)) + value - -class Array(object): @classmethod - def encode(cls, values): - # Assume that values are already encoded - return Int32.encode(len(values)) + b''.join(values) + def decode(cls, data): + length = Int32.decode(data) + if length < 0: + return None + return data.read(length) + + +class Schema(AbstractType): + def __init__(self, *fields): + if fields: + self.names, self.fields = zip(*fields) + else: + self.names, self.fields = (), () + + def encode(self, item): + if len(item) != len(self.fields): + raise ValueError('Item field count does not match Schema') + return b''.join([ + field.encode(item[i]) + for i, field in enumerate(self.fields) + ]) + + def decode(self, data): + return tuple([field.decode(data) for field in self.fields]) + + def __len__(self): + return len(self.fields) + + +class Array(AbstractType): + def __init__(self, *array_of): + if len(array_of) > 1: + self.array_of = Schema(*array_of) + elif len(array_of) == 1 and (isinstance(array_of[0], AbstractType) or + issubclass(array_of[0], AbstractType)): + self.array_of = array_of[0] + else: + raise ValueError('Array instantiated with no array_of type') + + def encode(self, items): + return b''.join( + [Int32.encode(len(items))] + + [self.array_of.encode(item) for item in items] + ) + + def decode(self, data): + length = Int32.decode(data) + return [self.array_of.decode(data) for _ in range(length)] -- cgit v1.2.1 From 058567912e8d82c1da5e5ead9e30be532573a173 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Nov 2015 10:00:50 +0800 Subject: Add simple BrokerConnection class; add request.RESPONSE_TYPE class vars --- kafka/conn.py | 36 ++++++++++++++++++++++++++++++++++++ kafka/protocol/commit.py | 42 ++++++++++++++++++++++++------------------ kafka/protocol/fetch.py | 24 +++++++++++++----------- kafka/protocol/metadata.py | 17 +++++++++-------- kafka/protocol/offset.py | 23 +++++++++++------------ kafka/protocol/produce.py | 17 ++++++++--------- 6 files changed, 101 insertions(+), 58 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 9514e48..0602d70 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -8,6 +8,8 @@ from threading import local import six from kafka.common import ConnectionError +from kafka.protocol.api import RequestHeader +from kafka.protocol.types import Int32 log = logging.getLogger(__name__) @@ -16,6 +18,40 @@ DEFAULT_SOCKET_TIMEOUT_SECONDS = 120 DEFAULT_KAFKA_PORT = 9092 +class BrokerConnection(local): + def __init__(self, host, port, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): + super(BrokerConnection, self).__init__() + self.host = host + self.port = port + self.timeout = timeout + self._sock = socket.create_connection((host, port), timeout) + self.fd = self._sock.makefile(mode='+') + self.correlation_id = 0 + + def close(self): + self.fd.close() + self._sock.close() + + def send(self, request): + self.correlation_id += 1 + header = RequestHeader(request, correlation_id=self.correlation_id) + message = b''.join([header.encode(), request.encode()]) + size = Int32.encode(len(message)) + self.fd.write(size) + self.fd.write(message) + self.fd.flush() + + size = Int32.decode(self.fd) + correlation_id = Int32.decode(self.fd) + return request.RESPONSE_TYPE.decode(self.fd) + + def __getnewargs__(self): + return (self.host, self.port, self.timeout) + + def __repr__(self): + return "" % (self.host, self.port) + + def collect_hosts(hosts, randomize=True): """ Collects a comma-separated set of hosts (host:port) and optionally diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py index 5ba0227..2955de1 100644 --- a/kafka/protocol/commit.py +++ b/kafka/protocol/commit.py @@ -2,9 +2,20 @@ from .struct import Struct from .types import Array, Int16, Int32, Int64, Schema, String +class OffsetCommitResponse(Struct): + SCHEMA = Schema( + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16))))) + ) + + class OffsetCommitRequest_v2(Struct): API_KEY = 8 API_VERSION = 2 # added retention_time, dropped timestamp + RESPONSE_TYPE = OffsetCommitResponse SCHEMA = Schema( ('consumer_group', String('utf-8')), ('consumer_group_generation_id', Int32), @@ -22,6 +33,7 @@ class OffsetCommitRequest_v2(Struct): class OffsetCommitRequest_v1(Struct): API_KEY = 8 API_VERSION = 1 # Kafka-backed storage + RESPONSE_TYPE = OffsetCommitResponse SCHEMA = Schema( ('consumer_group', String('utf-8')), ('consumer_group_generation_id', Int32), @@ -39,6 +51,7 @@ class OffsetCommitRequest_v1(Struct): class OffsetCommitRequest_v0(Struct): API_KEY = 8 API_VERSION = 0 # Zookeeper-backed storage + RESPONSE_TYPE = OffsetCommitResponse SCHEMA = Schema( ('consumer_group', String('utf-8')), ('topics', Array( @@ -50,12 +63,14 @@ class OffsetCommitRequest_v0(Struct): ) -class OffsetCommitResponse(Struct): +class OffsetFetchResponse(Struct): SCHEMA = Schema( ('topics', Array( ('topic', String('utf-8')), ('partitions', Array( ('partition', Int32), + ('offset', Int64), + ('metadata', String('utf-8')), ('error_code', Int16))))) ) @@ -63,6 +78,7 @@ class OffsetCommitResponse(Struct): class OffsetFetchRequest_v1(Struct): API_KEY = 9 API_VERSION = 1 # kafka-backed storage + RESPONSE_TYPE = OffsetFetchResponse SCHEMA = Schema( ('consumer_group', String('utf-8')), ('topics', Array( @@ -74,6 +90,7 @@ class OffsetFetchRequest_v1(Struct): class OffsetFetchRequest_v0(Struct): API_KEY = 9 API_VERSION = 0 # zookeeper-backed storage + RESPONSE_TYPE = OffsetFetchResponse SCHEMA = Schema( ('consumer_group', String('utf-8')), ('topics', Array( @@ -82,30 +99,19 @@ class OffsetFetchRequest_v0(Struct): ) -class OffsetFetchResponse(Struct): +class GroupCoordinatorResponse(Struct): SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')), - ('error_code', Int16))))) + ('error_code', Int16), + ('coordinator_id', Int32), + ('host', String('utf-8')), + ('port', Int32) ) class GroupCoordinatorRequest(Struct): API_KEY = 10 API_VERSION = 0 + RESPONSE_TYPE = GroupCoordinatorResponse SCHEMA = Schema( ('consumer_group', String('utf-8')) ) - - -class GroupCoordinatorResponse(Struct): - SCHEMA = Schema( - ('error_code', Int16), - ('coordinator_id', Int32), - ('host', String('utf-8')), - ('port', Int32) - ) diff --git a/kafka/protocol/fetch.py b/kafka/protocol/fetch.py index c6d60cc..e00c9ab 100644 --- a/kafka/protocol/fetch.py +++ b/kafka/protocol/fetch.py @@ -3,9 +3,22 @@ from .struct import Struct from .types import Array, Int16, Int32, Int64, Schema, String +class FetchResponse(Struct): + SCHEMA = Schema( + ('topics', Array( + ('topics', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16), + ('highwater_offset', Int64), + ('message_set', MessageSet))))) + ) + + class FetchRequest(Struct): API_KEY = 1 API_VERSION = 0 + RESPONSE_TYPE = FetchResponse SCHEMA = Schema( ('replica_id', Int32), ('max_wait_time', Int32), @@ -17,14 +30,3 @@ class FetchRequest(Struct): ('offset', Int64), ('max_bytes', Int32))))) ) - -class FetchResponse(Struct): - SCHEMA = Schema( - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('message_set', MessageSet))))) - ) diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py index b35e7ef..810f1b8 100644 --- a/kafka/protocol/metadata.py +++ b/kafka/protocol/metadata.py @@ -2,14 +2,6 @@ from .struct import Struct from .types import Array, Int16, Int32, Schema, String -class MetadataRequest(Struct): - API_KEY = 3 - API_VERSION = 0 - SCHEMA = Schema( - ('topics', Array(String('utf-8'))) - ) - - class MetadataResponse(Struct): SCHEMA = Schema( ('brokers', Array( @@ -26,3 +18,12 @@ class MetadataResponse(Struct): ('replicas', Array(Int32)), ('isr', Array(Int32)))))) ) + + +class MetadataRequest(Struct): + API_KEY = 3 + API_VERSION = 0 + RESPONSE_TYPE = MetadataResponse + SCHEMA = Schema( + ('topics', Array(String('utf-8'))) + ) diff --git a/kafka/protocol/offset.py b/kafka/protocol/offset.py index 942bdbf..776de39 100644 --- a/kafka/protocol/offset.py +++ b/kafka/protocol/offset.py @@ -2,31 +2,30 @@ from .struct import Struct from .types import Array, Int16, Int32, Int64, Schema, String -class OffsetRequest(Struct): - API_KEY = 2 - API_VERSION = 0 +class OffsetResponse(Struct): SCHEMA = Schema( - ('replica_id', Int32), ('topics', Array( ('topic', String('utf-8')), ('partitions', Array( ('partition', Int32), - ('time', Int64), - ('max_offsets', Int32))))) + ('error_code', Int16), + ('offsets', Array(Int64)))))) ) - DEFAULTS = { - 'replica_id': -1 - } -class OffsetResponse(Struct): +class OffsetRequest(Struct): API_KEY = 2 API_VERSION = 0 + RESPONSE_TYPE = OffsetResponse SCHEMA = Schema( + ('replica_id', Int32), ('topics', Array( ('topic', String('utf-8')), ('partitions', Array( ('partition', Int32), - ('error_code', Int16), - ('offsets', Array(Int64)))))) + ('time', Int64), + ('max_offsets', Int32))))) ) + DEFAULTS = { + 'replica_id': -1 + } diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py index 532a702..ef2f96e 100644 --- a/kafka/protocol/produce.py +++ b/kafka/protocol/produce.py @@ -3,28 +3,27 @@ from .struct import Struct from .types import Int8, Int16, Int32, Int64, Bytes, String, Array, Schema -class ProduceRequest(Struct): - API_KEY = 0 - API_VERSION = 0 +class ProduceResponse(Struct): SCHEMA = Schema( - ('required_acks', Int16), - ('timeout', Int32), ('topics', Array( ('topic', String('utf-8')), ('partitions', Array( ('partition', Int32), - ('messages', MessageSet))))) + ('error_code', Int16), + ('offset', Int64))))) ) -class ProduceResponse(Struct): +class ProduceRequest(Struct): API_KEY = 0 API_VERSION = 0 + RESPONSE_TYPE = ProduceResponse SCHEMA = Schema( + ('required_acks', Int16), + ('timeout', Int32), ('topics', Array( ('topic', String('utf-8')), ('partitions', Array( ('partition', Int32), - ('error_code', Int16), - ('offset', Int64))))) + ('messages', MessageSet))))) ) -- cgit v1.2.1 From 51227eae4f71565cf7abae51a91a6c037f705076 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Nov 2015 10:05:58 +0800 Subject: Use simply counting instead of buffer.tell() -- socket.makefile does not support tell() --- kafka/protocol/message.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 26f5ef6..4024a5c 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -60,8 +60,10 @@ class MessageSet(AbstractType): @classmethod def decode(cls, data): size = Int32.decode(data) - end = data.tell() + size + bytes_read = 0 items = [] - while data.tell() < end: + while bytes_read < size: items.append(cls.ITEM.decode(data)) + msg_size = items[-1][1] + bytes_read += (8 + 4 + msg_size) return items -- cgit v1.2.1 From 389fa20423eb7c553298aab47d79bfec989c72e2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Nov 2015 23:53:23 +0800 Subject: Recursive repr, display all field names and values for Structs --- kafka/protocol/abstract.py | 4 ++++ kafka/protocol/message.py | 6 +++++- kafka/protocol/struct.py | 8 +++++--- kafka/protocol/types.py | 14 +++++++++++++- 4 files changed, 27 insertions(+), 5 deletions(-) diff --git a/kafka/protocol/abstract.py b/kafka/protocol/abstract.py index 9c53c8c..532d15e 100644 --- a/kafka/protocol/abstract.py +++ b/kafka/protocol/abstract.py @@ -11,3 +11,7 @@ class AbstractType(object): @abc.abstractmethod def decode(cls, data): pass + + @classmethod + def repr(cls, value): + return repr(value) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 4024a5c..4f84c43 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -1,6 +1,6 @@ from .struct import Struct from .types import ( - Int8, Int16, Int32, Int64, Bytes, String, Array, Schema, AbstractType + Int8, Int32, Int64, Bytes, Schema, AbstractType ) from ..util import crc32 @@ -67,3 +67,7 @@ class MessageSet(AbstractType): msg_size = items[-1][1] bytes_read += (8 + 4 + msg_size) return items + + @classmethod + def repr(cls, messages): + return '[' + ', '.join([cls.ITEM.repr(m) for m in messages]) + ']' diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index 77f5fe7..30e233c 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -1,4 +1,4 @@ -from collections import namedtuple +#from collections import namedtuple from io import BytesIO from .abstract import AbstractType @@ -39,10 +39,12 @@ class Struct(AbstractType): return cls(*[field.decode(data) for field in cls.SCHEMA.fields]) def __repr__(self): - key_vals =['%s=%r' % (name, self.__dict__[name]) - for name in self.SCHEMA.names] + key_vals = [] + for name, field in zip(self.SCHEMA.names, self.SCHEMA.fields): + key_vals.append('%s=%s' % (name, field.repr(self.__dict__[name]))) return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' + """ class MetaStruct(type): def __new__(cls, clsname, bases, dct): diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 5aa2e41..99d89a6 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -1,6 +1,5 @@ from __future__ import absolute_import -import abc from struct import pack, unpack from .abstract import AbstractType @@ -104,6 +103,16 @@ class Schema(AbstractType): def __len__(self): return len(self.fields) + def repr(self, value): + key_vals = [] + for i in range(len(self)): + try: + field_val = getattr(value, self.names[i]) + except AttributeError: + field_val = value[i] + key_vals.append('%s=%s' % (self.names[i], self.fields[i].repr(field_val))) + return '(' + ', '.join(key_vals) + ')' + class Array(AbstractType): def __init__(self, *array_of): @@ -124,3 +133,6 @@ class Array(AbstractType): def decode(self, data): length = Int32.decode(data) return [self.array_of.decode(data) for _ in range(length)] + + def repr(self, list_of_items): + return '[' + ', '.join([self.array_of.repr(item) for item in list_of_items]) + ']' -- cgit v1.2.1 From ec323bcd0af675a6bd4acc61718a089321abd116 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Nov 2015 11:22:03 -0800 Subject: BrokerConnection: separate send / recv in BrokerConnection improve connection and error handling use different read and write fds for py3 --- kafka/conn.py | 76 ++++++++++++++++++++++++++++++++++++++++++++++++++--------- 1 file changed, 65 insertions(+), 11 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 0602d70..fee44c4 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -1,6 +1,8 @@ +from collections import deque import copy import logging from random import shuffle +from select import select import socket import struct from threading import local @@ -24,26 +26,78 @@ class BrokerConnection(local): self.host = host self.port = port self.timeout = timeout - self._sock = socket.create_connection((host, port), timeout) - self.fd = self._sock.makefile(mode='+') + self._write_fd = None + self._read_fd = None self.correlation_id = 0 + self.in_flight_requests = deque() + + def connect(self): + if self.connected(): + self.close() + try: + sock = socket.create_connection((self.host, self.port), self.timeout) + self._write_fd = sock.makefile('wb') + self._read_fd = sock.makefile('rb') + except socket.error as e: + log.exception("Error in BrokerConnection.connect()") + return None + self.in_flight_requests.clear() + return True + + def connected(self): + return (self._read_fd is not None and self._write_fd is not None) def close(self): - self.fd.close() - self._sock.close() + if self.connected(): + try: + self._read_fd.close() + self._write_fd.close() + except socket.error as e: + log.exception("Error in BrokerConnection.close()") + pass + self._read_fd = None + self._write_fd = None + self.in_flight_requests.clear() def send(self, request): + if not self.connected() and not self.connect(): + return None self.correlation_id += 1 header = RequestHeader(request, correlation_id=self.correlation_id) message = b''.join([header.encode(), request.encode()]) size = Int32.encode(len(message)) - self.fd.write(size) - self.fd.write(message) - self.fd.flush() - - size = Int32.decode(self.fd) - correlation_id = Int32.decode(self.fd) - return request.RESPONSE_TYPE.decode(self.fd) + try: + self._write_fd.write(size) + self._write_fd.write(message) + self._write_fd.flush() + except socket.error as e: + log.exception("Error in BrokerConnection.send()") + self.close() + return None + self.in_flight_requests.append((self.correlation_id, request.RESPONSE_TYPE)) + return self.correlation_id + + def recv(self, timeout=None): + if not self.connected(): + return None + readable, _, _ = select([self._read_fd], [], [], timeout) + if not readable: + return None + correlation_id, response_type = self.in_flight_requests.popleft() + # Current implementation does not use size + # instead we read directly from the socket fd buffer + # alternatively, we could read size bytes into a separate buffer + # and decode from that buffer (and verify buffer is empty afterwards) + size = Int32.decode(self._read_fd) + recv_correlation_id = Int32.decode(self._read_fd) + assert correlation_id == recv_correlation_id + try: + response = response_type.decode(self._read_fd) + except socket.error as e: + log.exception("Error in BrokerConnection.recv()") + self.close() + return None + return response def __getnewargs__(self): return (self.host, self.port, self.timeout) -- cgit v1.2.1 From c94cb620292f93a4cd3cfc0bb57c5fa38d95a717 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Nov 2015 11:24:35 -0800 Subject: Add simple Cluster class to manage broker metadata --- kafka/cluster.py | 91 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 91 insertions(+) create mode 100644 kafka/cluster.py diff --git a/kafka/cluster.py b/kafka/cluster.py new file mode 100644 index 0000000..3cd0a3c --- /dev/null +++ b/kafka/cluster.py @@ -0,0 +1,91 @@ +import logging +import random + +from .conn import BrokerConnection, collect_hosts +from .protocol.metadata import MetadataRequest + +logger = logging.getLogger(__name__) + + +class Cluster(object): + def __init__(self, **kwargs): + if 'bootstrap_servers' not in kwargs: + kargs['bootstrap_servers'] = 'localhost' + + self._brokers = {} + self._topics = {} + self._groups = {} + + self._bootstrap(collect_hosts(kwargs['bootstrap_servers']), + timeout=kwargs.get('bootstrap_timeout', 2)) + + def brokers(self): + brokers = list(self._brokers.values()) + return random.sample(brokers, len(brokers)) + + def random_broker(self): + for broker in self.brokers(): + if broker.connected() or broker.connect(): + return broker + return None + + def broker_by_id(self, broker_id): + return self._brokers.get(broker_id) + + def topics(self): + return list(self._topics.keys()) + + def partitions_for_topic(self, topic): + if topic not in self._topics: + return None + return list(self._topics[topic].keys()) + + def broker_for_partition(self, topic, partition): + if topic not in self._topics or partition not in self._topics[topic]: + return None + broker_id = self._topics[topic][partition] + return self.broker_by_id(broker_id) + + def refresh_metadata(self): + broker = self.random_broker() + if not broker.send(MetadataRequest([])): + return None + metadata = broker.recv() + if not metadata: + return None + self._update_metadata(metadata) + return metadata + + def _update_metadata(self, metadata): + self._brokers.update({ + node_id: BrokerConnection(host, port) + for node_id, host, port in metadata.brokers + if node_id not in self._brokers + }) + + self._topics = { + topic: { + partition: leader + for _, partition, leader, _, _ in partitions + } + for _, topic, partitions in metadata.topics + } + + def _bootstrap(self, hosts, timeout=2): + for host, port in hosts: + conn = BrokerConnection(host, port, timeout) + if not conn.connect(): + continue + self._brokers['bootstrap'] = conn + if self.refresh_metadata(): + break + else: + raise ValueError("Could not bootstrap kafka cluster from %s" % hosts) + + if len(self._brokers) > 1: + self._brokers.pop('bootstrap') + conn.close() + + def __str__(self): + return 'Cluster(brokers: %d, topics: %d, groups: %d)' % \ + (len(self._brokers), len(self._topics), len(self._groups)) -- cgit v1.2.1 From 235f7ac855f937207c3d430ad0dc762ff0c21091 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 1 Dec 2015 11:11:51 -0800 Subject: Unfinished kafka.consumer.group commit --- kafka/consumer/group.py | 883 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 883 insertions(+) create mode 100644 kafka/consumer/group.py diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py new file mode 100644 index 0000000..4a630ed --- /dev/null +++ b/kafka/consumer/group.py @@ -0,0 +1,883 @@ +from __future__ import absolute_import + +from collections import namedtuple +from copy import deepcopy +import logging +import random +import sys +import time + +import six + +from kafka.cluster import Cluster +from kafka.common import ( + OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, + check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, + OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, + FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError +) +from kafka.util import kafka_bytestring + +logger = logging.getLogger(__name__) + +OffsetsStruct = namedtuple("OffsetsStruct", ["fetch", "highwater", "commit", "task_done"]) + +NEW_CONSUMER_CONFIGS = { + 'bootstrap_servers': None, + 'client_id': None, + 'group_id': None, + 'key_deserializer': None, + 'value_deserializer': None, + 'auto_commit_interval_ms': 5000, + 'auto_offset_reset': 'latest', + 'check_crcs': True, # "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; + 'connections_max_idle_ms': 9 * 60 * 1000, + 'enable_auto_commit': True, + 'fetch_max_wait_ms': 500, + 'fetch_min_bytes': 1024, + 'heartbeat_interval_ms': 3000, + 'max_partition_fetch_bytes': 1 * 1024 * 1024, + 'metadata_max_age_ms': 5 * 60 * 1000, # >0 + 'metric_reporters': None, + 'metrics_num_samples': 2, + 'metrics_sample_window_ms': 30000, + 'partition_assignment_strategy': None, # This should default to something like 'roundrobin' or 'range' + 'reconnect_backoff_ms': 50, + 'request_timeout_ms': 40 * 1000, + 'retry_backoff_ms': 100, + 'send_buffer_bytes': 128 * 1024, + 'receive_buffer_bytes': 32 * 1024, + 'session_timeout_ms': 30000, # "The timeout used to detect failures when using Kafka's group management facilities."; +} + +DEFAULT_CONSUMER_CONFIG = { + 'client_id': __name__, + 'group_id': None, + 'bootstrap_servers': [], + 'socket_timeout_ms': 30 * 1000, + 'fetch_message_max_bytes': 1024 * 1024, + 'auto_offset_reset': 'largest', + 'fetch_min_bytes': 1, + 'fetch_wait_max_ms': 100, + 'refresh_leader_backoff_ms': 200, + 'deserializer_class': lambda msg: msg, + 'auto_commit_enable': False, + 'auto_commit_interval_ms': 60 * 1000, + 'auto_commit_interval_messages': None, + 'consumer_timeout_ms': -1, + + # Currently unused + 'socket_receive_buffer_bytes': 64 * 1024, + 'num_consumer_fetchers': 1, + 'default_fetcher_backoff_ms': 1000, + 'queued_max_message_chunks': 10, + 'rebalance_max_retries': 4, + 'rebalance_backoff_ms': 2000, +} + +DEPRECATED_CONFIG_KEYS = { + 'metadata_broker_list': 'bootstrap_servers', +} + +class KafkaConsumer(object): + """A simpler kafka consumer""" + + def __init__(self, *topics, **configs): + self._config = deepcopy(DEFAULT_CONSUMER_CONFIG) + self._topics = topics + self._partitions = [] + self._offsets = OffsetsStruct(fetch=dict(), commit=dict(), highwater=dict(), task_done=dict()) + self._consumer_timeout = False + self._uncommitted_message_count = 0 + self._next_commit_time = None + self._msg_iter = None + + self._configure(**configs) + self._cluster = Cluster(**self._config) + + def assign(self, topic_partitions): + pass + + def assignment(self): + """Get the set of partitions currently assigned to this consumer.""" + pass + + def close(self): + """Close the consumer, waiting indefinitely for any needed cleanup.""" + pass + + def commitAsync(self, topic_partition_offsets_and_metadata=None, callback=None): + """ + Commit offsets the specified offsets, or those returned on the last poll(), + for all the subscribed list of topics and partition. Asynchronous. + """ + pass + + def commitSync(self, topic_partition_offsets_and_metadata=None): + """ + Commit offsets the specified offsets, or those returned on the last poll(), + for all the subscribed list of topics and partition. Synchronous. + Blocks until either the commit succeeds or an unrecoverable error is + encountered (in which case it is thrown to the caller). + """ + pass + + def committed(self, topic_partition): + """ + Get the last committed offset for the given partition (whether the + commit happened by this process or another). + Returns: offset_and_metadata + """ + pass + + def listTopics(self): + """ + Get metadata about partitions for all topics that the user is authorized + to view. + Returns: {topic: [partition_info]} + """ + pass + + def metrics(self): + """ + Get the metrics kept by the consumer. + Returns: {metric_name: metric} + """ + pass + + def partitionsFor(self, topic): + """ + Get metadata about the partitions for a given topic. + Returns: [partition_info] + """ + pass + + def pause(self, *topic_partitions): + """Suspend fetching from the requested partitions.""" + pass + + def poll(self, timeout): + """ + Fetch data for the topics or partitions specified using one of the + subscribe/assign APIs. + Returns: [consumer_records] + """ + pass + + def position(self, topic_partition): + """Get the offset of the next record that will be fetched (if a record + with that offset exists).""" + pass + + def resume(self, *topic_partitions): + """Resume specified partitions which have been paused""" + pass + + def seek(self, topic_partition, offset): + """Overrides the fetch offsets that the consumer will use on the next + poll(timeout).""" + pass + + def seekToBeginning(self, *topic_partitions): + """Seek to the first offset for each of the given partitions.""" + pass + + def seekToEnd(self, *topic_partitions): + """Seek to the last offset for each of the given partitions.""" + pass + + def subscribe(self, topics, callback=None): + """Subscribe to the given list of topics or those matching a regex to get dynamically assigned + partitions.""" + pass + + def subscription(self): + """ + Get the current subscription. + Returns: [topic] + """ + pass + + def unsubscribe(self): + """Unsubscribe from topics currently subscribed with subscribe(List).""" + pass + + def wakeup(self): + """Wakeup the consumer.""" + pass + + def _configure(self, **configs): + """Configure the consumer instance + + Configuration settings can be passed to constructor, + otherwise defaults will be used: + + Keyword Arguments: + bootstrap_servers (list): List of initial broker nodes the consumer + should contact to bootstrap initial cluster metadata. This does + not have to be the full node list. It just needs to have at + least one broker that will respond to a Metadata API Request. + client_id (str): a unique name for this client. Defaults to + 'kafka.consumer.kafka'. + group_id (str): the name of the consumer group to join, + Offsets are fetched / committed to this group name. + fetch_message_max_bytes (int, optional): Maximum bytes for each + topic/partition fetch request. Defaults to 1024*1024. + fetch_min_bytes (int, optional): Minimum amount of data the server + should return for a fetch request, otherwise wait up to + fetch_wait_max_ms for more data to accumulate. Defaults to 1. + fetch_wait_max_ms (int, optional): Maximum time for the server to + block waiting for fetch_min_bytes messages to accumulate. + Defaults to 100. + refresh_leader_backoff_ms (int, optional): Milliseconds to backoff + when refreshing metadata on errors (subject to random jitter). + Defaults to 200. + socket_timeout_ms (int, optional): TCP socket timeout in + milliseconds. Defaults to 30*1000. + auto_offset_reset (str, optional): A policy for resetting offsets on + OffsetOutOfRange errors. 'smallest' will move to the oldest + available message, 'largest' will move to the most recent. Any + ofther value will raise the exception. Defaults to 'largest'. + deserializer_class (callable, optional): Any callable that takes a + raw message value and returns a deserialized value. Defaults to + lambda msg: msg. + auto_commit_enable (bool, optional): Enabling auto-commit will cause + the KafkaConsumer to periodically commit offsets without an + explicit call to commit(). Defaults to False. + auto_commit_interval_ms (int, optional): If auto_commit_enabled, + the milliseconds between automatic offset commits. Defaults to + 60 * 1000. + auto_commit_interval_messages (int, optional): If + auto_commit_enabled, a number of messages consumed between + automatic offset commits. Defaults to None (disabled). + consumer_timeout_ms (int, optional): number of millisecond to throw + a timeout exception to the consumer if no message is available + for consumption. Defaults to -1 (dont throw exception). + + Configuration parameters are described in more detail at + http://kafka.apache.org/documentation.html#highlevelconsumerapi + """ + configs = self._deprecate_configs(**configs) + self._config.update(configs) + + if self._config['auto_commit_enable']: + logger.info('Configuring consumer to auto-commit offsets') + self._reset_auto_commit() + + def set_topic_partitions(self, *topics): + """ + Set the topic/partitions to consume + Optionally specify offsets to start from + + Accepts types: + + * str (utf-8): topic name (will consume all available partitions) + * tuple: (topic, partition) + * dict: + - { topic: partition } + - { topic: [partition list] } + - { topic: (partition tuple,) } + + Optionally, offsets can be specified directly: + + * tuple: (topic, partition, offset) + * dict: { (topic, partition): offset, ... } + + Example: + + .. code:: python + + kafka = KafkaConsumer() + + # Consume topic1-all; topic2-partition2; topic3-partition0 + kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) + + # Consume topic1-0 starting at offset 12, and topic2-1 at offset 45 + # using tuples -- + kafka.set_topic_partitions(("topic1", 0, 12), ("topic2", 1, 45)) + + # using dict -- + kafka.set_topic_partitions({ ("topic1", 0): 12, ("topic2", 1): 45 }) + + """ + self._cluster.refresh_metadata() + + # Handle different topic types + for arg in topics: + + # Topic name str -- all partitions + if isinstance(arg, (six.string_types, six.binary_type)): + topic = kafka_bytestring(arg) + for partition in self._cluster.partitions_for_topic(topic): + self._consume_topic_partition(topic, partition) + + # (topic, partition [, offset]) tuple + elif isinstance(arg, tuple): + topic = kafka_bytestring(arg[0]) + partition = arg[1] + self._consume_topic_partition(topic, partition) + if len(arg) == 3: + offset = arg[2] + self._offsets.fetch[(topic, partition)] = offset + + # { topic: partitions, ... } dict + elif isinstance(arg, dict): + for key, value in six.iteritems(arg): + + # key can be string (a topic) + if isinstance(key, (six.string_types, six.binary_type)): + topic = kafka_bytestring(key) + + # topic: partition + if isinstance(value, int): + self._consume_topic_partition(topic, value) + + # topic: [ partition1, partition2, ... ] + elif isinstance(value, (list, tuple)): + for partition in value: + self._consume_topic_partition(topic, partition) + else: + raise KafkaConfigurationError( + 'Unknown topic type ' + '(dict key must be int or list/tuple of ints)' + ) + + # (topic, partition): offset + elif isinstance(key, tuple): + topic = kafka_bytestring(key[0]) + partition = key[1] + self._consume_topic_partition(topic, partition) + self._offsets.fetch[(topic, partition)] = value + + else: + raise KafkaConfigurationError('Unknown topic type (%s)' % type(arg)) + + # If we have a consumer group, try to fetch stored offsets + if self._config['group_id']: + self._get_commit_offsets() + + # Update missing fetch/commit offsets + for topic_partition in self._topics: + + # Commit offsets default is None + if topic_partition not in self._offsets.commit: + self._offsets.commit[topic_partition] = None + + # Skip if we already have a fetch offset from user args + if topic_partition not in self._offsets.fetch: + + # Fetch offsets default is (1) commit + if self._offsets.commit[topic_partition] is not None: + self._offsets.fetch[topic_partition] = self._offsets.commit[topic_partition] + + # or (2) auto reset + else: + self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) + + # highwater marks (received from server on fetch response) + # and task_done (set locally by user) + # should always get initialized to None + self._reset_highwater_offsets() + self._reset_task_done_offsets() + + # Reset message iterator in case we were in the middle of one + self._reset_message_iterator() + + def next(self): + """Return the next available message + + Blocks indefinitely unless consumer_timeout_ms > 0 + + Returns: + a single KafkaMessage from the message iterator + + Raises: + ConsumerTimeout after consumer_timeout_ms and no message + + Note: + This is also the method called internally during iteration + + """ + self._set_consumer_timeout_start() + while True: + + try: + return six.next(self._get_message_iterator()) + + # Handle batch completion + except StopIteration: + self._reset_message_iterator() + + self._check_consumer_timeout() + + def fetch_messages(self): + """Sends FetchRequests for all topic/partitions set for consumption + + Returns: + Generator that yields KafkaMessage structs + after deserializing with the configured `deserializer_class` + + Note: + Refreshes metadata on errors, and resets fetch offset on + OffsetOutOfRange, per the configured `auto_offset_reset` policy + + See Also: + Key KafkaConsumer configuration parameters: + * `fetch_message_max_bytes` + * `fetch_max_wait_ms` + * `fetch_min_bytes` + * `deserializer_class` + * `auto_offset_reset` + + """ + + max_bytes = self._config['fetch_message_max_bytes'] + max_wait_time = self._config['fetch_wait_max_ms'] + min_bytes = self._config['fetch_min_bytes'] + + if not self._topics: + raise KafkaConfigurationError('No topics or partitions configured') + + if not self._offsets.fetch: + raise KafkaConfigurationError( + 'No fetch offsets found when calling fetch_messages' + ) + + fetches = [FetchRequest(topic, partition, + self._offsets.fetch[(topic, partition)], + max_bytes) + for (topic, partition) in self._topics] + + # send_fetch_request will batch topic/partition requests by leader + responses = self._client.send_fetch_request( + fetches, + max_wait_time=max_wait_time, + min_bytes=min_bytes, + fail_on_error=False + ) + + for resp in responses: + + if isinstance(resp, FailedPayloadsError): + logger.warning('FailedPayloadsError attempting to fetch data') + self._refresh_metadata_on_error() + continue + + topic = kafka_bytestring(resp.topic) + partition = resp.partition + try: + check_error(resp) + except OffsetOutOfRangeError: + logger.warning('OffsetOutOfRange: topic %s, partition %d, ' + 'offset %d (Highwatermark: %d)', + topic, partition, + self._offsets.fetch[(topic, partition)], + resp.highwaterMark) + # Reset offset + self._offsets.fetch[(topic, partition)] = ( + self._reset_partition_offset((topic, partition)) + ) + continue + + except NotLeaderForPartitionError: + logger.warning("NotLeaderForPartitionError for %s - %d. " + "Metadata may be out of date", + topic, partition) + self._refresh_metadata_on_error() + continue + + except RequestTimedOutError: + logger.warning("RequestTimedOutError for %s - %d", + topic, partition) + continue + + # Track server highwater mark + self._offsets.highwater[(topic, partition)] = resp.highwaterMark + + # Yield each message + # Kafka-python could raise an exception during iteration + # we are not catching -- user will need to address + for (offset, message) in resp.messages: + # deserializer_class could raise an exception here + val = self._config['deserializer_class'](message.value) + msg = KafkaMessage(topic, partition, offset, message.key, val) + + # in some cases the server will return earlier messages + # than we requested. skip them per kafka spec + if offset < self._offsets.fetch[(topic, partition)]: + logger.debug('message offset less than fetched offset ' + 'skipping: %s', msg) + continue + # Only increment fetch offset + # if we safely got the message and deserialized + self._offsets.fetch[(topic, partition)] = offset + 1 + + # Then yield to user + yield msg + + def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): + """Request available fetch offsets for a single topic/partition + + Keyword Arguments: + topic (str): topic for offset request + partition (int): partition for offset request + request_time_ms (int): Used to ask for all messages before a + certain time (ms). There are two special values. + Specify -1 to receive the latest offset (i.e. the offset of the + next coming message) and -2 to receive the earliest available + offset. Note that because offsets are pulled in descending + order, asking for the earliest offset will always return you a + single element. + max_num_offsets (int): Maximum offsets to include in the OffsetResponse + + Returns: + a list of offsets in the OffsetResponse submitted for the provided + topic / partition. See: + https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI + """ + reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] + + (resp,) = self._client.send_offset_request(reqs) + + check_error(resp) + + # Just for sanity.. + # probably unnecessary + assert resp.topic == topic + assert resp.partition == partition + + return resp.offsets + + def offsets(self, group=None): + """Get internal consumer offset values + + Keyword Arguments: + group: Either "fetch", "commit", "task_done", or "highwater". + If no group specified, returns all groups. + + Returns: + A copy of internal offsets struct + """ + if not group: + return { + 'fetch': self.offsets('fetch'), + 'commit': self.offsets('commit'), + 'task_done': self.offsets('task_done'), + 'highwater': self.offsets('highwater') + } + else: + return dict(deepcopy(getattr(self._offsets, group))) + + def task_done(self, message): + """Mark a fetched message as consumed. + + Offsets for messages marked as "task_done" will be stored back + to the kafka cluster for this consumer group on commit() + + Arguments: + message (KafkaMessage): the message to mark as complete + + Returns: + True, unless the topic-partition for this message has not + been configured for the consumer. In normal operation, this + should not happen. But see github issue 364. + """ + topic_partition = (message.topic, message.partition) + if topic_partition not in self._topics: + logger.warning('Unrecognized topic/partition in task_done message: ' + '{0}:{1}'.format(*topic_partition)) + return False + + offset = message.offset + + # Warn on non-contiguous offsets + prev_done = self._offsets.task_done[topic_partition] + if prev_done is not None and offset != (prev_done + 1): + logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', + offset, prev_done) + + # Warn on smaller offsets than previous commit + # "commit" offsets are actually the offset of the next message to fetch. + prev_commit = self._offsets.commit[topic_partition] + if prev_commit is not None and ((offset + 1) <= prev_commit): + logger.warning('Marking task_done on a previously committed offset?: %d (+1) <= %d', + offset, prev_commit) + + self._offsets.task_done[topic_partition] = offset + + # Check for auto-commit + if self._does_auto_commit_messages(): + self._incr_auto_commit_message_count() + + if self._should_auto_commit(): + self.commit() + + return True + + def commit(self): + """Store consumed message offsets (marked via task_done()) + to kafka cluster for this consumer_group. + + Returns: + True on success, or False if no offsets were found for commit + + Note: + this functionality requires server version >=0.8.1.1 + https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + """ + if not self._config['group_id']: + logger.warning('Cannot commit without a group_id!') + raise KafkaConfigurationError( + 'Attempted to commit offsets ' + 'without a configured consumer group (group_id)' + ) + + # API supports storing metadata with each commit + # but for now it is unused + metadata = b'' + + offsets = self._offsets.task_done + commits = [] + for topic_partition, task_done_offset in six.iteritems(offsets): + + # Skip if None + if task_done_offset is None: + continue + + # Commit offsets as the next offset to fetch + # which is consistent with the Java Client + # task_done is marked by messages consumed, + # so add one to mark the next message for fetching + commit_offset = (task_done_offset + 1) + + # Skip if no change from previous committed + if commit_offset == self._offsets.commit[topic_partition]: + continue + + commits.append( + OffsetCommitRequest(topic_partition[0], topic_partition[1], + commit_offset, metadata) + ) + + if commits: + logger.info('committing consumer offsets to group %s', self._config['group_id']) + resps = self._client.send_offset_commit_request( + kafka_bytestring(self._config['group_id']), commits, + fail_on_error=False + ) + + for r in resps: + check_error(r) + topic_partition = (r.topic, r.partition) + task_done = self._offsets.task_done[topic_partition] + self._offsets.commit[topic_partition] = (task_done + 1) + + if self._config['auto_commit_enable']: + self._reset_auto_commit() + + return True + + else: + logger.info('No new offsets found to commit in group %s', self._config['group_id']) + return False + + # + # Topic/partition management private methods + # + + def _consume_topic_partition(self, topic, partition): + if not isinstance(partition, int): + raise KafkaConfigurationError('Unknown partition type (%s) ' + '-- expected int' % type(partition)) + + if topic not in self._cluster.topics(): + raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) + if partition not in self._cluster.partitions_for_topic(topic): + raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " + "in broker metadata" % (partition, topic)) + logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) + self._topics.append((topic, partition)) + + def _refresh_metadata_on_error(self): + refresh_ms = self._config['refresh_leader_backoff_ms'] + jitter_pct = 0.20 + sleep_ms = random.randint( + int((1.0 - 0.5 * jitter_pct) * refresh_ms), + int((1.0 + 0.5 * jitter_pct) * refresh_ms) + ) + while True: + logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) + time.sleep(sleep_ms / 1000.0) + try: + self._client.load_metadata_for_topics() + except KafkaUnavailableError: + logger.warning("Unable to refresh topic metadata... cluster unavailable") + self._check_consumer_timeout() + else: + logger.info("Topic metadata refreshed") + return + + # + # Offset-managment private methods + # + + def _get_commit_offsets(self): + logger.info("Consumer fetching stored offsets") + for topic_partition in self._topics: + (resp,) = self._client.send_offset_fetch_request( + kafka_bytestring(self._config['group_id']), + [OffsetFetchRequest(topic_partition[0], topic_partition[1])], + fail_on_error=False) + try: + check_error(resp) + # API spec says server wont set an error here + # but 0.8.1.1 does actually... + except UnknownTopicOrPartitionError: + pass + + # -1 offset signals no commit is currently stored + if resp.offset == -1: + self._offsets.commit[topic_partition] = None + + # Otherwise we committed the stored offset + # and need to fetch the next one + else: + self._offsets.commit[topic_partition] = resp.offset + + def _reset_highwater_offsets(self): + for topic_partition in self._topics: + self._offsets.highwater[topic_partition] = None + + def _reset_task_done_offsets(self): + for topic_partition in self._topics: + self._offsets.task_done[topic_partition] = None + + def _reset_partition_offset(self, topic_partition): + (topic, partition) = topic_partition + LATEST = -1 + EARLIEST = -2 + + request_time_ms = None + if self._config['auto_offset_reset'] == 'largest': + request_time_ms = LATEST + elif self._config['auto_offset_reset'] == 'smallest': + request_time_ms = EARLIEST + else: + + # Let's raise an reasonable exception type if user calls + # outside of an exception context + if sys.exc_info() == (None, None, None): + raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' + 'valid auto_offset_reset setting ' + '(largest|smallest)') + + # Otherwise we should re-raise the upstream exception + # b/c it typically includes additional data about + # the request that triggered it, and we do not want to drop that + raise + + (offset, ) = self.get_partition_offsets(topic, partition, + request_time_ms, max_num_offsets=1) + return offset + + # + # Consumer Timeout private methods + # + + def _set_consumer_timeout_start(self): + self._consumer_timeout = False + if self._config['consumer_timeout_ms'] >= 0: + self._consumer_timeout = time.time() + (self._config['consumer_timeout_ms'] / 1000.0) + + def _check_consumer_timeout(self): + if self._consumer_timeout and time.time() > self._consumer_timeout: + raise ConsumerTimeout('Consumer timed out after %d ms' % + self._config['consumer_timeout_ms']) + + # + # Autocommit private methods + # + + def _should_auto_commit(self): + if self._does_auto_commit_ms(): + if time.time() >= self._next_commit_time: + return True + + if self._does_auto_commit_messages(): + if self._uncommitted_message_count >= self._config['auto_commit_interval_messages']: + return True + + return False + + def _reset_auto_commit(self): + if not self._config['group_id']: + raise KafkaConfigurationError('auto_commit requires group_id') + self._uncommitted_message_count = 0 + self._next_commit_time = None + if self._does_auto_commit_ms(): + self._next_commit_time = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) + + def _incr_auto_commit_message_count(self, n=1): + self._uncommitted_message_count += n + + def _does_auto_commit_ms(self): + if not self._config['auto_commit_enable']: + return False + + conf = self._config['auto_commit_interval_ms'] + if conf is not None and conf > 0: + return True + return False + + def _does_auto_commit_messages(self): + if not self._config['auto_commit_enable']: + return False + + conf = self._config['auto_commit_interval_messages'] + if conf is not None and conf > 0: + return True + return False + + # + # Message iterator private methods + # + + def __iter__(self): + return self + + def __next__(self): + return self.next() + + def _get_message_iterator(self): + # Fetch a new batch if needed + if self._msg_iter is None: + self._msg_iter = self.fetch_messages() + + return self._msg_iter + + def _reset_message_iterator(self): + self._msg_iter = None + + # + # python private methods + # + + def __repr__(self): + return '<{0} topics=({1})>'.format( + self.__class__.__name__, + '|'.join(["%s-%d" % topic_partition + for topic_partition in self._topics]) + ) + + # + # other private methods + # + + def _deprecate_configs(self, **configs): + for old, new in six.iteritems(DEPRECATED_CONFIG_KEYS): + if old in configs: + logger.warning('Deprecated Kafka Consumer configuration: %s. ' + 'Please use %s instead.', old, new) + old_value = configs.pop(old) + if new not in configs: + configs[new] = old_value + return configs -- cgit v1.2.1 From 254c17e39fb8790957da792acdd7e435551a9ac6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 1 Dec 2015 16:20:33 -0800 Subject: Add comment re bytes offset tracking in MessageSet decode() --- kafka/protocol/message.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 4f84c43..c3265f9 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -65,7 +65,7 @@ class MessageSet(AbstractType): while bytes_read < size: items.append(cls.ITEM.decode(data)) msg_size = items[-1][1] - bytes_read += (8 + 4 + msg_size) + bytes_read += (8 + 4 + msg_size) # item size = 8 byte offset, 4 byte message_size, plus message bytes return items @classmethod -- cgit v1.2.1 From 7a6c51bf2e0a926ffe2595f008c68c6b63db2ce7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 1 Dec 2015 16:21:08 -0800 Subject: Add size and correlation id decoding to try/except block in BrokerConnection --- kafka/conn.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index fee44c4..ab44073 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -88,12 +88,13 @@ class BrokerConnection(local): # instead we read directly from the socket fd buffer # alternatively, we could read size bytes into a separate buffer # and decode from that buffer (and verify buffer is empty afterwards) - size = Int32.decode(self._read_fd) - recv_correlation_id = Int32.decode(self._read_fd) - assert correlation_id == recv_correlation_id try: + size = Int32.decode(self._read_fd) + recv_correlation_id = Int32.decode(self._read_fd) + if correlation_id != recv_correlation_id: + raise RuntimeError('Correlation ids do not match!') response = response_type.decode(self._read_fd) - except socket.error as e: + except (RuntimeError, socket.error) as e: log.exception("Error in BrokerConnection.recv()") self.close() return None -- cgit v1.2.1 From 892f5dd9337fdf8aa06eccb37b4087432f7e0c14 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 3 Dec 2015 15:17:47 -0800 Subject: group membership api schemas --- kafka/protocol/group.py | 108 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 108 insertions(+) create mode 100644 kafka/protocol/group.py diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py new file mode 100644 index 0000000..3766e48 --- /dev/null +++ b/kafka/protocol/group.py @@ -0,0 +1,108 @@ +from .struct import Struct +from .types import Array, Bytes, Int16, Int32, Schema, String + + +class JoinGroupResponse(Struct): + SCHEMA = Schema( + ('error_code', Int16), + ('generation_id', Int32), + ('group_protocol', String('utf-8')), + ('leader_id', String('utf-8')), + ('member_id', String('utf-8')), + ('members', Array( + ('member_id', String('utf-8')), + ('member_metadata', Bytes))) + ) + + +class JoinGroupRequest(Struct): + API_KEY = 11 + API_VERSION = 0 + RESPONSE_TYPE = JoinGroupResponse + SCHEMA = Schema( + ('group', String('utf-8')), + ('session_timeout', Int32), + ('member_id', String('utf-8')), + ('protocol_type', String('utf-8')), + ('group_protocols', Array( + ('protocol_name', String('utf-8')), + ('protocol_metadata', Bytes))) + ) + + +class ProtocolName(Struct): + SCHEMA = Schema( + ('assignment_strategy', String('utf-8')) + ) + + +class ProtocolMetadata(Struct): + SCHEMA = Schema( + ('version', Int16), + ('subscription', Array(String('utf-8'))), # topics list + ('user_data', Bytes) + ) + + +class SyncGroupResponse(Struct): + SCHEMA = Schema( + ('error_code', Int16), + ('member_assignment', Bytes) + ) + + +class SyncGroupRequest(Struct): + API_KEY = 14 + API_VERSION = 0 + RESPONSE_TYPE = SyncGroupResponse + SCHEMA = Schema( + ('group', String('utf-8')), + ('generation_id', Int32), + ('member_id', String('utf-8')), + ('group_assignment', Array( + ('member_id', String('utf-8')), + ('member_metadata', Bytes))) + ) + + +class MemberAssignment(Struct): + SCHEMA = Schema( + ('version', Int16), + ('partition_assignment', Array( + ('topic', String('utf-8')), + ('partitions', Array(Int32)))), + ('user_data', Bytes) + ) + + +class HeartbeatResponse(Struct): + SCHEMA = Schema( + ('error_code', Int16) + ) + + +class HeartbeatRequest(Struct): + API_KEY = 12 + API_VERSION = 0 + RESPONSE_TYPE = HeartbeatResponse + SCHEMA = Schema( + ('group', String('utf-8')), + ('generation_id', Int32), + ('member_id', String('utf-8')) + ) + + +class LeaveGroupResponse(Struct): + SCHEMA = Schema( + ('error_code', Int16) + ) + + +class LeaveGroupRequest(Struct): + API_KEY = 13 + API_VERSION = 0 + RESPONSE_TYPE = LeaveGroupResponse + SCHEMA = Schema( + ('group', String('utf-8')), + ('member_id', String('utf-8')) + ) -- cgit v1.2.1 From 5d87a5edb411f9553b8a7b1f76130aadb37ac77d Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 3 Dec 2015 15:55:04 -0800 Subject: administration api schemas --- kafka/protocol/admin.py | 44 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 kafka/protocol/admin.py diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py new file mode 100644 index 0000000..56dd042 --- /dev/null +++ b/kafka/protocol/admin.py @@ -0,0 +1,44 @@ +from .struct import Struct +from .types import Array, Bytes, Int16, Schema, String + + +class ListGroupsResponse(Struct): + SCHEMA = Schema( + ('error_code', Int16), + ('groups', Array( + ('group', String('utf-8')), + ('protocol_type', String('utf-8')))) + ) + + +class ListGroupsRequest(Struct): + API_KEY = 16 + API_VERSION = 0 + RESPONSE_TYPE = ListGroupsResponse + SCHEMA = Schema() + + +class DescribeGroupsResponse(Struct): + SCHEMA = Schema( + ('groups', Array( + ('error_code', Int16), + ('group', String('utf-8')), + ('state', String('utf-8')), + ('protocol_type', String('utf-8')), + ('protocol', String('utf-8')), + ('members', Array( + ('member_id', String('utf-8')), + ('client_id', String('utf-8')), + ('client_host', String('utf-8')), + ('member_metadata', Bytes), + ('member_assignment', Bytes))))) + ) + + +class DescribeGroupsRequest(Struct): + API_KEY = 15 + API_VERSION = 0 + RESPONSE_TYPE = DescribeGroupsResponse + SCHEMA = Schema( + ('groups', Array(String('utf-8'))) + ) -- cgit v1.2.1 From efc3d4f466c0d6630c9fff09fb1b90035c5351d7 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Thu, 3 Dec 2015 17:46:08 -0800 Subject: few small cleanups --- kafka/cluster.py | 2 +- kafka/protocol/group.py | 6 ------ kafka/protocol/struct.py | 2 +- 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index 3cd0a3c..55765dc 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -10,7 +10,7 @@ logger = logging.getLogger(__name__) class Cluster(object): def __init__(self, **kwargs): if 'bootstrap_servers' not in kwargs: - kargs['bootstrap_servers'] = 'localhost' + kwargs['bootstrap_servers'] = 'localhost' self._brokers = {} self._topics = {} diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index 3766e48..63e4a11 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -30,12 +30,6 @@ class JoinGroupRequest(Struct): ) -class ProtocolName(Struct): - SCHEMA = Schema( - ('assignment_strategy', String('utf-8')) - ) - - class ProtocolMetadata(Struct): SCHEMA = Schema( ('version', Int16), diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index 30e233c..5b4c312 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -21,7 +21,7 @@ class Struct(AbstractType): self.encode = self._encode_self @classmethod - def encode(cls, item): + def encode(cls, item): # pylint: disable-msg=E0202 bits = [] for i, field in enumerate(cls.SCHEMA.fields): bits.append(field.encode(item[i])) -- cgit v1.2.1 From f719ffcc047d4c6e4ad79d83257c4d1b2b014314 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 10:13:19 -0800 Subject: Handle decoding partial messages in MessageSet - caused by FetchRequest max_bytes --- kafka/protocol/message.py | 30 ++++++++++++++++++++++++------ 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index c3265f9..8f32749 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -59,13 +59,31 @@ class MessageSet(AbstractType): @classmethod def decode(cls, data): - size = Int32.decode(data) - bytes_read = 0 + bytes_to_read = Int32.decode(data) items = [] - while bytes_read < size: - items.append(cls.ITEM.decode(data)) - msg_size = items[-1][1] - bytes_read += (8 + 4 + msg_size) # item size = 8 byte offset, 4 byte message_size, plus message bytes + + # We need at least 12 bytes to read offset + message size + while bytes_to_read >= 12: + offset = Int64.decode(data) + bytes_to_read -= 8 + + message_size = Int32.decode(data) + bytes_to_read -= 4 + + # if FetchRequest max_bytes is smaller than the available message set + # the server returns partial data for the final message + if message_size > bytes_to_read: + break + + message = Message.decode(data) + bytes_to_read -= message_size + + items.append((offset, message_size, message)) + + # If any bytes are left over, clear them from the buffer + if bytes_to_read: + data.read(bytes_to_read) + return items @classmethod -- cgit v1.2.1 From 3a0a8e1ee4c39655ba12900eb6bd6f7901262239 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 10:14:10 -0800 Subject: Handle special __init__ signature in Message decode() --- kafka/protocol/message.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 8f32749..501ce47 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -31,6 +31,14 @@ class Message(Struct): self.crc = crc32(message[4:]) return self.SCHEMA.fields[0].encode(self.crc) + message[4:] + @classmethod + def decode(cls, data): + if isinstance(data, bytes): + data = BytesIO(data) + fields = [field.decode(data) for field in cls.SCHEMA.fields] + return cls(fields[4], key=fields[3], + magic=fields[1], attributes=fields[2], crc=fields[0]) + class MessageSet(AbstractType): ITEM = Schema( -- cgit v1.2.1 From 58bdeb17d7e337c48ee2c14bf1f73b00eed0e727 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 13:56:49 -0800 Subject: Fix _mp_consume queue variable name conflict --- kafka/consumer/multiprocess.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index 18a5014..d0e2920 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -25,7 +25,7 @@ log = logging.getLogger(__name__) Events = namedtuple("Events", ["start", "pause", "exit"]) -def _mp_consume(client, group, topic, queue, size, events, **consumer_options): +def _mp_consume(client, group, topic, message_queue, size, events, **consumer_options): """ A child process worker which consumes messages based on the notifications given by the controller process @@ -69,7 +69,7 @@ def _mp_consume(client, group, topic, queue, size, events, **consumer_options): if message: while True: try: - queue.put(message, timeout=FULL_QUEUE_WAIT_TIME_SECONDS) + message_queue.put(message, timeout=FULL_QUEUE_WAIT_TIME_SECONDS) break except queue.Full: if events.exit.is_set(): break -- cgit v1.2.1 From c4f87bce204d27cb7897baccab8454f997ce6c49 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:13:54 -0800 Subject: Fix BytesIO import in kafka.protocol.message --- kafka/protocol/message.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 501ce47..3027ebd 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -1,3 +1,5 @@ +import io + from .struct import Struct from .types import ( Int8, Int32, Int64, Bytes, Schema, AbstractType @@ -34,7 +36,7 @@ class Message(Struct): @classmethod def decode(cls, data): if isinstance(data, bytes): - data = BytesIO(data) + data = io.BytesIO(data) fields = [field.decode(data) for field in cls.SCHEMA.fields] return cls(fields[4], key=fields[3], magic=fields[1], attributes=fields[2], crc=fields[0]) -- cgit v1.2.1 From 5aeba4a7dc68e76c96f743a8a9e3e6603875695e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:14:50 -0800 Subject: Fallback to simple repr() in Schema.repr() --- kafka/protocol/types.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 99d89a6..01799bb 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -105,13 +105,16 @@ class Schema(AbstractType): def repr(self, value): key_vals = [] - for i in range(len(self)): - try: - field_val = getattr(value, self.names[i]) - except AttributeError: - field_val = value[i] - key_vals.append('%s=%s' % (self.names[i], self.fields[i].repr(field_val))) - return '(' + ', '.join(key_vals) + ')' + try: + for i in range(len(self)): + try: + field_val = getattr(value, self.names[i]) + except AttributeError: + field_val = value[i] + key_vals.append('%s=%s' % (self.names[i], self.fields[i].repr(field_val))) + return '(' + ', '.join(key_vals) + ')' + except: + return repr(value) class Array(AbstractType): -- cgit v1.2.1 From 1636c96df41b61b37883a60238dfb42b353f36a2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:16:37 -0800 Subject: Return PartialMessage object in MessageSet.decode if message is truncated by max_bytes --- kafka/protocol/message.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 3027ebd..cd5d274 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -42,6 +42,11 @@ class Message(Struct): magic=fields[1], attributes=fields[2], crc=fields[0]) +class PartialMessage(bytes): + def __repr__(self): + return 'PartialMessage(%s)' % self + + class MessageSet(AbstractType): ITEM = Schema( ('offset', Int64), @@ -72,8 +77,9 @@ class MessageSet(AbstractType): bytes_to_read = Int32.decode(data) items = [] - # We need at least 12 bytes to read offset + message size - while bytes_to_read >= 12: + # We need at least 8 + 4 + 14 bytes to read offset + message size + message + # (14 bytes is a message w/ null key and null value) + while bytes_to_read >= 26: offset = Int64.decode(data) bytes_to_read -= 8 @@ -91,8 +97,9 @@ class MessageSet(AbstractType): items.append((offset, message_size, message)) # If any bytes are left over, clear them from the buffer + # and append a PartialMessage to signal that max_bytes may be too small if bytes_to_read: - data.read(bytes_to_read) + items.append((None, None, PartialMessage(data.read(bytes_to_read)))) return items -- cgit v1.2.1 From 9740b2b88b41726f143b3367285dbc118bfa0a8a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:17:52 -0800 Subject: Support pickling of Structs -- _encode_self instance method needs some magic --- kafka/protocol/message.py | 1 + kafka/protocol/pickle.py | 25 +++++++++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 kafka/protocol/pickle.py diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index cd5d274..a67d7f5 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -1,5 +1,6 @@ import io +from . import pickle from .struct import Struct from .types import ( Int8, Int32, Int64, Bytes, Schema, AbstractType diff --git a/kafka/protocol/pickle.py b/kafka/protocol/pickle.py new file mode 100644 index 0000000..af0d1ee --- /dev/null +++ b/kafka/protocol/pickle.py @@ -0,0 +1,25 @@ +from __future__ import absolute_import + +import copy_reg +import types + + +def _pickle_method(method): + func_name = method.im_func.__name__ + obj = method.im_self + cls = method.im_class + return _unpickle_method, (func_name, obj, cls) + + +def _unpickle_method(func_name, obj, cls): + for cls in cls.mro(): + try: + func = cls.__dict__[func_name] + except KeyError: + pass + else: + break + return func.__get__(obj, cls) + +# https://bytes.com/topic/python/answers/552476-why-cant-you-pickle-instancemethods +copy_reg.pickle(types.MethodType, _pickle_method, _unpickle_method) -- cgit v1.2.1 From d0de279459a92e787730f5c85a2cf6f2741cbd97 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:20:57 -0800 Subject: Change KafkaProtocol to encode/decode Structs, not bytes - add Payload to kafka.common Request/Responses namedtuples - OffsetFetch and OffsetCommit still need to be converted --- kafka/common.py | 12 +- kafka/protocol/legacy.py | 352 ++++++++++++++--------------------------------- 2 files changed, 112 insertions(+), 252 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index a7d8164..7ae3294 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -21,24 +21,24 @@ ConsumerMetadataResponse = namedtuple("ConsumerMetadataResponse", ["error", "nodeId", "host", "port"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ProduceAPI -ProduceRequest = namedtuple("ProduceRequest", +ProduceRequestPayload = namedtuple("ProduceRequestPayload", ["topic", "partition", "messages"]) -ProduceResponse = namedtuple("ProduceResponse", +ProduceResponsePayload = namedtuple("ProduceResponsePayload", ["topic", "partition", "error", "offset"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-FetchAPI -FetchRequest = namedtuple("FetchRequest", +FetchRequestPayload = namedtuple("FetchRequest", ["topic", "partition", "offset", "max_bytes"]) -FetchResponse = namedtuple("FetchResponse", +FetchResponsePayload = namedtuple("FetchResponse", ["topic", "partition", "error", "highwaterMark", "messages"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI -OffsetRequest = namedtuple("OffsetRequest", +OffsetRequestPayload = namedtuple("OffsetRequest", ["topic", "partition", "time", "max_offsets"]) -OffsetResponse = namedtuple("OffsetResponse", +OffsetResponsePayload = namedtuple("OffsetResponse", ["topic", "partition", "error", "offsets"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py index db9f3e0..c5babf7 100644 --- a/kafka/protocol/legacy.py +++ b/kafka/protocol/legacy.py @@ -7,16 +7,21 @@ import six from six.moves import xrange +import kafka.common +import kafka.protocol.commit +import kafka.protocol.fetch +import kafka.protocol.message +import kafka.protocol.metadata +import kafka.protocol.offset +import kafka.protocol.produce + from kafka.codec import ( gzip_encode, gzip_decode, snappy_encode, snappy_decode ) from kafka.common import ( - Message, OffsetAndMessage, TopicAndPartition, - BrokerMetadata, TopicMetadata, PartitionMetadata, - MetadataResponse, ProduceResponse, FetchResponse, - OffsetResponse, OffsetCommitResponse, OffsetFetchResponse, - ProtocolError, BufferUnderflowError, ChecksumError, - ConsumerFetchSizeTooSmall, UnsupportedCodecError, + OffsetCommitResponse, OffsetFetchResponse, + ProtocolError, ChecksumError, + UnsupportedCodecError, ConsumerMetadataResponse ) from kafka.util import ( @@ -114,41 +119,6 @@ class KafkaProtocol(object): raise ProtocolError("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 - read_message = False - 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): - read_message = True - yield OffsetAndMessage(offset, message) - except BufferUnderflowError: - # NOTE: Not sure this is correct error handling: - # Is it possible to get a BUE if the message set is somewhere - # in the middle of the fetch response? If so, we probably have - # an issue that's not fetch size too small. - # Aren't we ignoring errors if we fail to unpack data by - # raising StopIteration()? - # If _decode_message() raises a ChecksumError, couldn't that - # also be due to the fetch size being too small? - if read_message is False: - # If we get a partial read of a message, but haven't - # yielded anything there's a problem - raise ConsumerFetchSizeTooSmall() - else: - raise StopIteration() - @classmethod def _decode_message(cls, data, offset): """ @@ -169,7 +139,7 @@ class KafkaProtocol(object): codec = att & ATTRIBUTE_CODEC_MASK if codec == CODEC_NONE: - yield (offset, Message(magic, att, key, value)) + yield (offset, kafka.common.Message(magic, att, key, value)) elif codec == CODEC_GZIP: gz = gzip_decode(value) @@ -186,253 +156,143 @@ class KafkaProtocol(object): ################## @classmethod - def encode_produce_request(cls, client_id, correlation_id, - payloads=None, acks=1, timeout=1000): + def encode_produce_request(cls, payloads=(), acks=1, timeout=1000): """ - Encode some ProduceRequest structs + Encode a ProduceRequest struct Arguments: - client_id: string - correlation_id: int - payloads: list of ProduceRequest + payloads: list of ProduceRequestPayload 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 + 0: immediate response -1: waits for all replicas to be in sync - timeout: Maximum time the server will wait for acks from replicas. + timeout: Maximum time (in ms) the server will wait for replica acks. This is _not_ a socket timeout - """ - payloads = [] if payloads is None else payloads - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.PRODUCE_KEY)) - - message.append(struct.pack('>hii', acks, timeout, - len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(struct.pack('>h%dsi' % len(topic), len(topic), topic, - len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - msg_set = KafkaProtocol._encode_message_set(payload.messages) - message.append(struct.pack('>ii%ds' % len(msg_set), partition, - len(msg_set), msg_set)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) + Returns: ProduceRequest + """ + if acks not in (1, 0, -1): + raise ValueError('ProduceRequest acks (%s) must be 1, 0, -1' % acks) + + return kafka.protocol.produce.ProduceRequest( + required_acks=acks, + timeout=timeout, + topics=[( + topic, + [( + partition, + [(0, 0, kafka.protocol.message.Message(msg.value, key=msg.key, + magic=msg.magic, + attributes=msg.attributes)) + for msg in payload.messages]) + for partition, payload in topic_payloads.items()]) + for topic, topic_payloads in group_by_topic_and_partition(payloads).items()]) @classmethod - def decode_produce_response(cls, data): + def decode_produce_response(cls, response): """ - Decode bytes to a ProduceResponse + Decode ProduceResponse to ProduceResponsePayload Arguments: - data: bytes to decode + response: ProduceResponse + Return: list of ProduceResponsePayload """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - - for _ 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 _ in range(num_partitions): - ((partition, error, offset), cur) = relative_unpack('>ihq', - data, cur) - - yield ProduceResponse(topic, partition, error, offset) + return [ + kafka.common.ProduceResponsePayload(topic, partition, error, offset) + for topic, partitions in response.topics + for partition, error, offset in partitions + ] @classmethod - def encode_fetch_request(cls, client_id, correlation_id, payloads=None, - max_wait_time=100, min_bytes=4096): + def encode_fetch_request(cls, payloads=(), max_wait_time=100, min_bytes=4096): """ - Encodes some FetchRequest structs + Encodes a FetchRequest struct Arguments: - client_id: string - correlation_id: int - 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 - """ - - payloads = [] if payloads is None else payloads - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.FETCH_KEY)) - - # -1 is the replica id - message.append(struct.pack('>iiii', -1, max_wait_time, min_bytes, - len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>iqi', partition, payload.offset, - payload.max_bytes)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) + payloads: list of FetchRequestPayload + max_wait_time (int, optional): ms to block waiting for min_bytes + data. Defaults to 100. + min_bytes (int, optional): minimum bytes required to return before + max_wait_time. Defaults to 4096. + + Return: FetchRequest + """ + return kafka.protocol.fetch.FetchRequest( + replica_id=-1, + max_wait_time=max_wait_time, + min_bytes=min_bytes, + topics=[( + topic, + [( + partition, + payload.offset, + payload.max_bytes) + for partition, payload in topic_payloads.items()]) + for topic, topic_payloads in group_by_topic_and_partition(payloads).items()]) @classmethod - def decode_fetch_response(cls, data): + def decode_fetch_response(cls, response): """ - Decode bytes to a FetchResponse + Decode FetchResponse struct to FetchResponsePayloads Arguments: - data: bytes to decode + response: FetchResponse """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) - - for _ in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for j 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)) + return [ + kafka.common.FetchResponsePayload( + topic, partition, error, highwater_offset, [ + kafka.common.OffsetAndMessage(offset, message) + for offset, _, message in messages]) + for topic, partitions in response.topics + for partition, error, highwater_offset, messages in partitions + ] @classmethod - 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 = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_KEY)) - - # -1 is the replica id - message.append(struct.pack('>ii', -1, len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>iqi', partition, payload.time, - payload.max_offsets)) - - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) + def encode_offset_request(cls, payloads=()): + return kafka.protocol.offset.OffsetRequest( + replica_id=-1, + topics=[( + topic, + [( + partition, + payload.time, + payload.max_offsets) + for partition, payload in six.iteritems(topic_payloads)]) + for topic, topic_payloads in six.iteritems(group_by_topic_and_partition(payloads))]) @classmethod - def decode_offset_response(cls, data): + def decode_offset_response(cls, response): """ - Decode bytes to an OffsetResponse + Decode OffsetResponse into OffsetResponsePayloads Arguments: - data: bytes to decode - """ - ((correlation_id, num_topics), cur) = relative_unpack('>ii', data, 0) + response: OffsetResponse - for _ in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for _ in range(num_partitions): - ((partition, error, num_offsets,), cur) = \ - relative_unpack('>ihi', data, cur) - - offsets = [] - for k in range(num_offsets): - ((offset,), cur) = relative_unpack('>q', data, cur) - offsets.append(offset) - - yield OffsetResponse(topic, partition, error, tuple(offsets)) + Returns: list of OffsetResponsePayloads + """ + return [ + kafka.common.OffsetResponsePayload(topic, partition, error, tuple(offsets)) + for topic, partitions in response.topics + for partition, error, offsets in partitions + ] @classmethod - def encode_metadata_request(cls, client_id, correlation_id, topics=None, - payloads=None): + def encode_metadata_request(cls, topics=(), payloads=None): """ Encode a MetadataRequest Arguments: - client_id: string - correlation_id: int topics: list of strings """ - if payloads is None: - topics = [] if topics is None else topics - else: + if payloads is not None: topics = payloads - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.METADATA_KEY)) - - message.append(struct.pack('>i', len(topics))) - - for topic in topics: - message.append(struct.pack('>h%ds' % len(topic), len(topic), topic)) - - msg = b''.join(message) - return write_int_string(msg) + return kafka.protocol.metadata.MetadataRequest(topics) @classmethod - def decode_metadata_response(cls, data): - """ - Decode bytes to a MetadataResponse - - Arguments: - data: bytes to decode - """ - ((correlation_id, numbrokers), cur) = relative_unpack('>ii', data, 0) - - # Broker info - brokers = [] - for _ 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.append(BrokerMetadata(nodeId, host, port)) - - # Topic info - ((num_topics,), cur) = relative_unpack('>i', data, cur) - topic_metadata = [] - - for _ in range(num_topics): - ((topic_error,), cur) = relative_unpack('>h', data, cur) - (topic_name, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - partition_metadata = [] - - for _ in range(num_partitions): - ((partition_error_code, partition, leader, numReplicas), cur) = \ - relative_unpack('>hiii', data, cur) - - (replicas, cur) = relative_unpack( - '>%di' % numReplicas, data, cur) - - ((num_isr,), cur) = relative_unpack('>i', data, cur) - (isr, cur) = relative_unpack('>%di' % num_isr, data, cur) - - partition_metadata.append( - PartitionMetadata(topic_name, partition, leader, - replicas, isr, partition_error_code) - ) - - topic_metadata.append( - TopicMetadata(topic_name, topic_error, partition_metadata) - ) - - return MetadataResponse(brokers, topic_metadata) + def decode_metadata_response(cls, response): + return response @classmethod def encode_consumer_metadata_request(cls, client_id, correlation_id, payloads): @@ -587,7 +447,7 @@ def create_message(payload, key=None): key: bytes, a key used for partition routing (optional) """ - return Message(0, 0, key, payload) + return kafka.common.Message(0, 0, key, payload) def create_gzip_message(payloads, key=None, compresslevel=None): @@ -608,7 +468,7 @@ def create_gzip_message(payloads, key=None, compresslevel=None): gzipped = gzip_encode(message_set, compresslevel=compresslevel) codec = ATTRIBUTE_CODEC_MASK & CODEC_GZIP - return Message(0, 0x00 | codec, key, gzipped) + return kafka.common.Message(0, 0x00 | codec, key, gzipped) def create_snappy_message(payloads, key=None): @@ -629,7 +489,7 @@ def create_snappy_message(payloads, key=None): snapped = snappy_encode(message_set) codec = ATTRIBUTE_CODEC_MASK & CODEC_SNAPPY - return Message(0, 0x00 | codec, key, snapped) + return kafka.common.Message(0, 0x00 | codec, key, snapped) def create_message_set(messages, codec=CODEC_NONE, key=None, compresslevel=None): -- cgit v1.2.1 From 17d6a68f6ececea3b4d0290dc84c4c2fc9508e9a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:31:48 -0800 Subject: Add client_id and correlation_id to BrokerConnection constructor kwargs --- kafka/conn.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index ab44073..84a72aa 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -21,14 +21,16 @@ DEFAULT_KAFKA_PORT = 9092 class BrokerConnection(local): - def __init__(self, host, port, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS): + def __init__(self, host, port, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS, + client_id='kafka-python-0.10.0', correlation_id=0): super(BrokerConnection, self).__init__() self.host = host self.port = port self.timeout = timeout self._write_fd = None self._read_fd = None - self.correlation_id = 0 + self.correlation_id = correlation_id + self.client_id = client_id self.in_flight_requests = deque() def connect(self): @@ -63,7 +65,9 @@ class BrokerConnection(local): if not self.connected() and not self.connect(): return None self.correlation_id += 1 - header = RequestHeader(request, correlation_id=self.correlation_id) + header = RequestHeader(request, + correlation_id=self.correlation_id, + client_id=self.client_id) message = b''.join([header.encode(), request.encode()]) size = Int32.encode(len(message)) try: -- cgit v1.2.1 From 4be8a58592e63859964ca903fa09a7a31ba0c3a2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:32:41 -0800 Subject: Exception handling cleanup in BrokerConnection (also catch struct.errors in recv) --- kafka/conn.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 84a72aa..e9877f2 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -40,7 +40,7 @@ class BrokerConnection(local): sock = socket.create_connection((self.host, self.port), self.timeout) self._write_fd = sock.makefile('wb') self._read_fd = sock.makefile('rb') - except socket.error as e: + except socket.error: log.exception("Error in BrokerConnection.connect()") return None self.in_flight_requests.clear() @@ -54,7 +54,7 @@ class BrokerConnection(local): try: self._read_fd.close() self._write_fd.close() - except socket.error as e: + except socket.error: log.exception("Error in BrokerConnection.close()") pass self._read_fd = None @@ -74,7 +74,7 @@ class BrokerConnection(local): self._write_fd.write(size) self._write_fd.write(message) self._write_fd.flush() - except socket.error as e: + except socket.error: log.exception("Error in BrokerConnection.send()") self.close() return None @@ -98,7 +98,7 @@ class BrokerConnection(local): if correlation_id != recv_correlation_id: raise RuntimeError('Correlation ids do not match!') response = response_type.decode(self._read_fd) - except (RuntimeError, socket.error) as e: + except (RuntimeError, socket.error, struct.error): log.exception("Error in BrokerConnection.recv()") self.close() return None -- cgit v1.2.1 From ad030ccd4df57305bb624b03eddaa2641f956160 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:34:58 -0800 Subject: Refactor KafkaClient to use BrokerConnections and new Request/Response structs --- kafka/client.py | 279 ++++++++++++++++++++++++++------------------------------ 1 file changed, 131 insertions(+), 148 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 9018bb4..cb60d98 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -2,17 +2,20 @@ import collections import copy import functools import logging +import random import select import time +import six + import kafka.common -from kafka.common import (TopicAndPartition, BrokerMetadata, +from kafka.common import (TopicAndPartition, BrokerMetadata, UnknownError, ConnectionError, FailedPayloadsError, KafkaTimeoutError, KafkaUnavailableError, LeaderNotAvailableError, UnknownTopicOrPartitionError, NotLeaderForPartitionError, ReplicaNotAvailableError) -from kafka.conn import collect_hosts, KafkaConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS +from kafka.conn import collect_hosts, BrokerConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS from kafka.protocol import KafkaProtocol from kafka.util import kafka_bytestring @@ -31,13 +34,12 @@ class KafkaClient(object): timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS, correlation_id=0): # We need one connection to bootstrap - self.client_id = kafka_bytestring(client_id) + self.client_id = client_id self.timeout = timeout self.hosts = collect_hosts(hosts) self.correlation_id = correlation_id - # create connections only when we need them - self.conns = {} + self._conns = {} self.brokers = {} # broker_id -> BrokerMetadata self.topics_to_brokers = {} # TopicAndPartition -> BrokerMetadata self.topic_partitions = {} # topic -> partition -> PartitionMetadata @@ -52,14 +54,14 @@ class KafkaClient(object): def _get_conn(self, host, port): """Get or create a connection to a broker using host and port""" host_key = (host, port) - if host_key not in self.conns: - self.conns[host_key] = KafkaConnection( - host, - port, - timeout=self.timeout + if host_key not in self._conns: + self._conns[host_key] = BrokerConnection( + host, port, + timeout=self.timeout, + client_id=self.client_id ) - return self.conns[host_key] + return self._conns[host_key] def _get_leader_for_partition(self, topic, partition): """ @@ -91,12 +93,12 @@ class KafkaClient(object): raise UnknownTopicOrPartitionError(key) # If there's no leader for the partition, raise - meta = self.topic_partitions[topic][partition] - if meta.leader == -1: - raise LeaderNotAvailableError(meta) + leader = self.topic_partitions[topic][partition] + if leader == -1: + raise LeaderNotAvailableError((topic, partition)) # Otherwise return the BrokerMetadata - return self.brokers[meta.leader] + return self.brokers[leader] def _get_coordinator_for_group(self, group): """ @@ -129,27 +131,35 @@ class KafkaClient(object): Attempt to send a broker-agnostic request to one of the available brokers. Keep trying until you succeed. """ - for (host, port) in self.hosts: - requestId = self._next_id() - log.debug('Request %s: %s', requestId, payloads) - try: - conn = self._get_conn(host, port) - request = encoder_fn(client_id=self.client_id, - correlation_id=requestId, - payloads=payloads) - - conn.send(requestId, request) - response = conn.recv(requestId) + hosts = set([(broker.host, broker.port) for broker in self.brokers.values()]) + hosts.update(self.hosts) + hosts = list(hosts) + random.shuffle(hosts) + + for (host, port) in hosts: + conn = self._get_conn(host, port) + request = encoder_fn(payloads=payloads) + correlation_id = conn.send(request) + if correlation_id is None: + continue + response = conn.recv() + if response is not None: decoded = decoder_fn(response) - log.debug('Response %s: %s', requestId, decoded) + log.debug('Response %s: %s', correlation_id, decoded) return decoded - except Exception: - log.exception('Error sending request [%s] to server %s:%s, ' - 'trying next server', requestId, host, port) - raise KafkaUnavailableError('All servers failed to process request') + def _payloads_by_broker(self, payloads): + payloads_by_broker = collections.defaultdict(list) + for payload in payloads: + try: + leader = self._get_leader_for_partition(payload.topic, payload.partition) + except KafkaUnavailableError: + leader = None + payloads_by_broker[leader].append(payload) + return dict(payloads_by_broker) + def _send_broker_aware_request(self, payloads, encoder_fn, decoder_fn): """ Group a list of request payloads by topic+partition and send them to @@ -178,97 +188,76 @@ class KafkaClient(object): # so we need to keep this so we can rebuild order before returning original_ordering = [(p.topic, p.partition) for p in payloads] - # Group the requests by topic+partition - brokers_for_payloads = [] - payloads_by_broker = collections.defaultdict(list) - - responses = {} - for payload in payloads: - try: - leader = self._get_leader_for_partition(payload.topic, - payload.partition) - payloads_by_broker[leader].append(payload) - brokers_for_payloads.append(leader) - except KafkaUnavailableError as e: - log.warning('KafkaUnavailableError attempting to send request ' - 'on topic %s partition %d', payload.topic, payload.partition) - topic_partition = (payload.topic, payload.partition) - responses[topic_partition] = FailedPayloadsError(payload) + # Connection errors generally mean stale metadata + # although sometimes it means incorrect api request + # Unfortunately there is no good way to tell the difference + # so we'll just reset metadata on all errors to be safe + refresh_metadata = False # For each broker, send the list of request payloads # and collect the responses and errors - broker_failures = [] + payloads_by_broker = self._payloads_by_broker(payloads) + responses = {} - # For each KafkaConnection keep the real socket so that we can use + def failed_payloads(payloads): + for payload in payloads: + topic_partition = (str(payload.topic), payload.partition) + responses[(topic_partition)] = FailedPayloadsError(payload) + + # For each BrokerConnection keep the real socket so that we can use # a select to perform unblocking I/O connections_by_socket = {} - for broker, payloads in payloads_by_broker.items(): - requestId = self._next_id() - log.debug('Request %s to %s: %s', requestId, broker, payloads) - request = encoder_fn(client_id=self.client_id, - correlation_id=requestId, payloads=payloads) - - # Send the request, recv the response - try: - conn = self._get_conn(broker.host.decode('utf-8'), broker.port) - conn.send(requestId, request) - - except ConnectionError as e: - broker_failures.append(broker) - log.warning('ConnectionError attempting to send request %s ' - 'to server %s: %s', requestId, broker, e) + for broker, broker_payloads in six.iteritems(payloads_by_broker): + if broker is None: + failed_payloads(broker_payloads) + continue - for payload in payloads: - topic_partition = (payload.topic, payload.partition) - responses[topic_partition] = FailedPayloadsError(payload) + conn = self._get_conn(broker.host.decode('utf-8'), broker.port) + request = encoder_fn(payloads=broker_payloads) + # decoder_fn=None signal that the server is expected to not + # send a response. This probably only applies to + # ProduceRequest w/ acks = 0 + expect_response = (decoder_fn is not None) + correlation_id = conn.send(request, expect_response=expect_response) + + if correlation_id is None: + refresh_metadata = True + failed_payloads(broker_payloads) + log.warning('Error attempting to send request %s ' + 'to server %s', correlation_id, broker) + continue - # No exception, try to get response - else: + if not expect_response: + log.debug('Request %s does not expect a response ' + '(skipping conn.recv)', correlation_id) + for payload in broker_payloads: + topic_partition = (str(payload.topic), payload.partition) + responses[topic_partition] = None + continue - # decoder_fn=None signal that the server is expected to not - # send a response. This probably only applies to - # ProduceRequest w/ acks = 0 - if decoder_fn is None: - log.debug('Request %s does not expect a response ' - '(skipping conn.recv)', requestId) - for payload in payloads: - topic_partition = (payload.topic, payload.partition) - responses[topic_partition] = None - continue - else: - connections_by_socket[conn.get_connected_socket()] = (conn, broker, requestId) + connections_by_socket[conn._read_fd] = (conn, broker) conn = None while connections_by_socket: sockets = connections_by_socket.keys() rlist, _, _ = select.select(sockets, [], [], None) - conn, broker, requestId = connections_by_socket.pop(rlist[0]) - try: - response = conn.recv(requestId) - except ConnectionError as e: - broker_failures.append(broker) - log.warning('ConnectionError attempting to receive a ' - 'response to request %s from server %s: %s', - requestId, broker, e) + conn, broker = connections_by_socket.pop(rlist[0]) + correlation_id = conn.next_correlation_id_recv() + response = conn.recv() + if response is None: + refresh_metadata = True + failed_payloads(payloads_by_broker[broker]) + log.warning('Error receiving response to request %s ' + 'from server %s', correlation_id, broker) + continue - for payload in payloads_by_broker[broker]: - topic_partition = (payload.topic, payload.partition) - responses[topic_partition] = FailedPayloadsError(payload) + log.debug('Response %s: %s', correlation_id, response) + for payload_response in decoder_fn(response): + topic_partition = (str(payload_response.topic), + payload_response.partition) + responses[topic_partition] = payload_response - else: - _resps = [] - for payload_response in decoder_fn(response): - topic_partition = (payload_response.topic, - payload_response.partition) - responses[topic_partition] = payload_response - _resps.append(payload_response) - log.debug('Response %s: %s', requestId, _resps) - - # Connection errors generally mean stale metadata - # although sometimes it means incorrect api request - # Unfortunately there is no good way to tell the difference - # so we'll just reset metadata on all errors to be safe - if broker_failures: + if refresh_metadata: self.reset_all_metadata() # Return responses in the same order as provided @@ -387,7 +376,7 @@ class KafkaClient(object): # Public API # ################# def close(self): - for conn in self.conns.values(): + for conn in self._conns.values(): conn.close() def copy(self): @@ -398,13 +387,14 @@ class KafkaClient(object): Note that the copied connections are not initialized, so reinit() must be called on the returned copy. """ + _conns = self._conns + self._conns = {} c = copy.deepcopy(self) - for key in c.conns: - c.conns[key] = self.conns[key].copy() + self._conns = _conns return c def reinit(self): - for conn in self.conns.values(): + for conn in self._conns.values(): conn.reinit() def reset_topic_metadata(self, *topics): @@ -480,11 +470,8 @@ class KafkaClient(object): Partition-level errors will also not be raised here (a single partition w/o a leader, for example) """ - topics = [kafka_bytestring(t) for t in topics] - if topics: - for topic in topics: - self.reset_topic_metadata(topic) + self.reset_topic_metadata(*topics) else: self.reset_all_metadata() @@ -493,50 +480,46 @@ class KafkaClient(object): log.debug('Updating broker metadata: %s', resp.brokers) log.debug('Updating topic metadata: %s', resp.topics) - self.brokers = dict([(broker.nodeId, broker) - for broker in resp.brokers]) - - for topic_metadata in resp.topics: - topic = topic_metadata.topic - partitions = topic_metadata.partitions + self.brokers = dict([(nodeId, BrokerMetadata(nodeId, host, port)) + for nodeId, host, port in resp.brokers]) + for error, topic, partitions in resp.topics: # Errors expected for new topics - try: - kafka.common.check_error(topic_metadata) - except (UnknownTopicOrPartitionError, LeaderNotAvailableError) as e: - - # Raise if the topic was passed in explicitly - if topic in topics: - raise - - # Otherwise, just log a warning - log.error('Error loading topic metadata for %s: %s', topic, type(e)) - continue + if error: + error_type = kafka.common.kafka_errors.get(error, UnknownError) + if error_type in (UnknownTopicOrPartitionError, LeaderNotAvailableError): + log.error('Error loading topic metadata for %s: %s (%s)', + topic, error_type, error) + if topic not in topics: + continue + raise error_type(topic) self.topic_partitions[topic] = {} - for partition_metadata in partitions: - partition = partition_metadata.partition - leader = partition_metadata.leader + for error, partition, leader, _, _ in partitions: - self.topic_partitions[topic][partition] = partition_metadata + self.topic_partitions[topic][partition] = leader # Populate topics_to_brokers dict topic_part = TopicAndPartition(topic, partition) # Check for partition errors - try: - kafka.common.check_error(partition_metadata) - - # If No Leader, topics_to_brokers topic_partition -> None - except LeaderNotAvailableError: - log.error('No leader for topic %s partition %d', topic, partition) - self.topics_to_brokers[topic_part] = None - continue - # If one of the replicas is unavailable -- ignore - # this error code is provided for admin purposes only - # we never talk to replicas, only the leader - except ReplicaNotAvailableError: - log.debug('Some (non-leader) replicas not available for topic %s partition %d', topic, partition) + if error: + error_type = kafka.common.kafka_errors.get(error, UnknownError) + + # If No Leader, topics_to_brokers topic_partition -> None + if error_type is LeaderNotAvailableError: + log.error('No leader for topic %s partition %d', topic, partition) + self.topics_to_brokers[topic_part] = None + continue + + # If one of the replicas is unavailable -- ignore + # this error code is provided for admin purposes only + # we never talk to replicas, only the leader + elif error_type is ReplicaNotAvailableError: + log.debug('Some (non-leader) replicas not available for topic %s partition %d', topic, partition) + + else: + raise error_type(topic_part) # If Known Broker, topic_partition -> BrokerMetadata if leader in self.brokers: -- cgit v1.2.1 From a3ec9bd8e8c730c9f6715b536c0c590230fc2e28 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:37:17 -0800 Subject: Update references to kafka.common Request/Response (now Payload) --- kafka/consumer/base.py | 4 ++-- kafka/consumer/kafka.py | 11 ++++++----- kafka/consumer/simple.py | 26 ++++++++++---------------- kafka/producer/base.py | 17 +++++++++-------- test/test_client_integration.py | 14 +++++++------- test/test_consumer_integration.py | 4 ++-- test/test_producer.py | 10 +++++----- test/test_producer_integration.py | 6 +++--- test/testutil.py | 4 ++-- 9 files changed, 46 insertions(+), 50 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index c9f6e48..034d35c 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -7,7 +7,7 @@ from threading import Lock import kafka.common from kafka.common import ( - OffsetRequest, OffsetCommitRequest, OffsetFetchRequest, + OffsetRequestPayload, OffsetCommitRequest, OffsetFetchRequest, UnknownTopicOrPartitionError, check_error, KafkaError ) @@ -217,7 +217,7 @@ class Consumer(object): reqs = [] for partition in partitions: - reqs.append(OffsetRequest(self.topic, partition, -1, 1)) + reqs.append(OffsetRequestPayload(self.topic, partition, -1, 1)) resps = self.client.send_offset_request(reqs) for resp in resps: diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 3ef106c..1bd3def 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -11,7 +11,8 @@ import six from kafka.client import KafkaClient from kafka.common import ( - OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, + OffsetFetchRequest, OffsetCommitRequest, + OffsetRequestPayload, FetchRequestPayload, check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError @@ -333,9 +334,9 @@ class KafkaConsumer(object): 'No fetch offsets found when calling fetch_messages' ) - fetches = [FetchRequest(topic, partition, - self._offsets.fetch[(topic, partition)], - max_bytes) + fetches = [FetchRequestPayload(topic, partition, + self._offsets.fetch[(topic, partition)], + max_bytes) for (topic, partition) in self._topics] # send_fetch_request will batch topic/partition requests by leader @@ -425,7 +426,7 @@ class KafkaConsumer(object): topic / partition. See: https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI """ - reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] + reqs = [OffsetRequestPayload(topic, partition, request_time_ms, max_num_offsets)] (resp,) = self._client.send_offset_request(reqs) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 7c63246..1c2aee6 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -27,7 +27,7 @@ from .base import ( NO_MESSAGES_WAIT_TIME_SECONDS ) from ..common import ( - FetchRequest, KafkaError, OffsetRequest, + FetchRequestPayload, KafkaError, OffsetRequestPayload, ConsumerFetchSizeTooSmall, ConsumerNoMoreData, UnknownTopicOrPartitionError, NotLeaderForPartitionError, OffsetOutOfRangeError, FailedPayloadsError, check_error @@ -153,9 +153,9 @@ class SimpleConsumer(Consumer): LATEST = -1 EARLIEST = -2 if self.auto_offset_reset == 'largest': - reqs = [OffsetRequest(self.topic, partition, LATEST, 1)] + reqs = [OffsetRequestPayload(self.topic, partition, LATEST, 1)] elif self.auto_offset_reset == 'smallest': - reqs = [OffsetRequest(self.topic, partition, EARLIEST, 1)] + reqs = [OffsetRequestPayload(self.topic, partition, EARLIEST, 1)] else: # Let's raise an reasonable exception type if user calls # outside of an exception context @@ -224,23 +224,17 @@ class SimpleConsumer(Consumer): for tmp_partition in self.offsets.keys(): if whence == 0: - reqs.append(OffsetRequest(self.topic, - tmp_partition, - -2, - 1)) + reqs.append(OffsetRequestPayload(self.topic, tmp_partition, -2, 1)) elif whence == 2: - reqs.append(OffsetRequest(self.topic, - tmp_partition, - -1, - 1)) + reqs.append(OffsetRequestPayload(self.topic, tmp_partition, -1, 1)) else: pass else: deltas[partition] = offset if whence == 0: - reqs.append(OffsetRequest(self.topic, partition, -2, 1)) + reqs.append(OffsetRequestPayload(self.topic, partition, -2, 1)) elif whence == 2: - reqs.append(OffsetRequest(self.topic, partition, -1, 1)) + reqs.append(OffsetRequestPayload(self.topic, partition, -1, 1)) else: pass @@ -370,9 +364,9 @@ class SimpleConsumer(Consumer): while partitions: requests = [] for partition, buffer_size in six.iteritems(partitions): - requests.append(FetchRequest(self.topic, partition, - self.fetch_offsets[partition], - buffer_size)) + requests.append(FetchRequestPayload(self.topic, partition, + self.fetch_offsets[partition], + buffer_size)) # Send request responses = self.client.send_fetch_request( requests, diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 39b1f84..3f2bba6 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -15,7 +15,7 @@ from threading import Thread, Event import six from kafka.common import ( - ProduceRequest, ProduceResponse, TopicAndPartition, RetryOptions, + ProduceRequestPayload, ProduceResponsePayload, TopicAndPartition, RetryOptions, kafka_errors, UnsupportedCodecError, FailedPayloadsError, RequestTimedOutError, AsyncProducerQueueFull, UnknownError, RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES @@ -133,9 +133,10 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # Send collected requests upstream for topic_partition, msg in msgset.items(): messages = create_message_set(msg, codec, key, codec_compresslevel) - req = ProduceRequest(topic_partition.topic, - topic_partition.partition, - tuple(messages)) + req = ProduceRequestPayload( + topic_partition.topic, + topic_partition.partition, + tuple(messages)) request_tries[req] = 0 if not request_tries: @@ -169,13 +170,13 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, error_cls = response.__class__ orig_req = response.payload - elif isinstance(response, ProduceResponse) and response.error: + elif isinstance(response, ProduceResponsePayload) and response.error: error_cls = kafka_errors.get(response.error, UnknownError) orig_req = requests[i] if error_cls: _handle_error(error_cls, orig_req) - log.error('%s sending ProduceRequest (#%d of %d) ' + log.error('%s sending ProduceRequestPayload (#%d of %d) ' 'to %s:%d with msgs %s', error_cls.__name__, (i + 1), len(requests), orig_req.topic, orig_req.partition, @@ -210,7 +211,7 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, # Log messages we are going to retry for orig_req in request_tries.keys(): - log.info('Retrying ProduceRequest to %s:%d with msgs %s', + log.info('Retrying ProduceRequestPayload to %s:%d with msgs %s', orig_req.topic, orig_req.partition, orig_req.messages if log_messages_on_error else hash(orig_req.messages)) @@ -404,7 +405,7 @@ class Producer(object): resp = [] else: messages = create_message_set([(m, key) for m in msg], self.codec, key, self.codec_compresslevel) - req = ProduceRequest(topic, partition, messages) + req = ProduceRequestPayload(topic, partition, messages) try: resp = self.client.send_produce_request( [req], acks=self.req_acks, timeout=self.ack_timeout, diff --git a/test/test_client_integration.py b/test/test_client_integration.py index 8853350..70da4a3 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -1,8 +1,8 @@ import os from kafka.common import ( - FetchRequest, OffsetCommitRequest, OffsetFetchRequest, - KafkaTimeoutError, ProduceRequest + FetchRequestPayload, OffsetCommitRequest, OffsetFetchRequest, + KafkaTimeoutError, ProduceRequestPayload ) from kafka.protocol import create_message @@ -29,7 +29,7 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_consume_none(self): - fetch = FetchRequest(self.bytes_topic, 0, 0, 1024) + fetch = FetchRequestPayload(self.bytes_topic, 0, 0, 1024) fetch_resp, = self.client.send_fetch_request([fetch]) self.assertEqual(fetch_resp.error, 0) @@ -57,16 +57,16 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): self.client.ensure_topic_exists(b'bar') requests = [ - ProduceRequest( + ProduceRequestPayload( b'foo', 0, [create_message(b'a'), create_message(b'b')]), - ProduceRequest( + ProduceRequestPayload( b'bar', 1, [create_message(b'a'), create_message(b'b')]), - ProduceRequest( + ProduceRequestPayload( b'foo', 1, [create_message(b'a'), create_message(b'b')]), - ProduceRequest( + ProduceRequestPayload( b'bar', 0, [create_message(b'a'), create_message(b'b')]), ] diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index fee53f5..d536537 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -7,7 +7,7 @@ from kafka import ( KafkaConsumer, MultiProcessConsumer, SimpleConsumer, create_message ) from kafka.common import ( - ProduceRequest, ConsumerFetchSizeTooSmall, ConsumerTimeout, + ProduceRequestPayload, ConsumerFetchSizeTooSmall, ConsumerTimeout, OffsetOutOfRangeError ) from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES @@ -41,7 +41,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def send_messages(self, partition, messages): messages = [ create_message(self.msg(str(msg))) for msg in messages ] - produce = ProduceRequest(self.bytes_topic, partition, messages = messages) + produce = ProduceRequestPayload(self.bytes_topic, partition, messages = messages) resp, = self.client.send_produce_request([produce]) self.assertEqual(resp.error, 0) diff --git a/test/test_producer.py b/test/test_producer.py index 31282bf..cbc1773 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -10,7 +10,7 @@ from . import unittest from kafka import KafkaClient, SimpleProducer, KeyedProducer from kafka.common import ( AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError, - ProduceResponse, RetryOptions, TopicAndPartition + ProduceResponsePayload, RetryOptions, TopicAndPartition ) from kafka.producer.base import Producer, _send_upstream from kafka.protocol import CODEC_NONE @@ -186,7 +186,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): offset = offsets[req.topic][req.partition] offsets[req.topic][req.partition] += len(req.messages) responses.append( - ProduceResponse(req.topic, req.partition, 0, offset) + ProduceResponsePayload(req.topic, req.partition, 0, offset) ) return responses @@ -234,8 +234,8 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def send_side_effect(reqs, *args, **kwargs): if self.client.is_first_time: self.client.is_first_time = False - return [ProduceResponse(req.topic, req.partition, - NotLeaderForPartitionError.errno, -1) + return [ProduceResponsePayload(req.topic, req.partition, + NotLeaderForPartitionError.errno, -1) for req in reqs] responses = [] @@ -243,7 +243,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): offset = offsets[req.topic][req.partition] offsets[req.topic][req.partition] += len(req.messages) responses.append( - ProduceResponse(req.topic, req.partition, 0, offset) + ProduceResponsePayload(req.topic, req.partition, 0, offset) ) return responses diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index c99ed63..ee0b2fd 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -11,7 +11,7 @@ from kafka import ( ) from kafka.codec import has_snappy from kafka.common import ( - FetchRequest, ProduceRequest, + FetchRequestPayload, ProduceRequestPayload, UnknownTopicOrPartitionError, LeaderNotAvailableError ) from kafka.producer.base import Producer @@ -488,7 +488,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): def assert_produce_request(self, messages, initial_offset, message_ct, partition=0): - produce = ProduceRequest(self.bytes_topic, partition, messages=messages) + produce = ProduceRequestPayload(self.bytes_topic, partition, messages=messages) # There should only be one response message from the server. # This will throw an exception if there's more than one. @@ -506,7 +506,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # There should only be one response message from the server. # This will throw an exception if there's more than one. - resp, = self.client.send_fetch_request([ FetchRequest(self.bytes_topic, partition, start_offset, 1024) ]) + resp, = self.client.send_fetch_request([FetchRequestPayload(self.bytes_topic, partition, start_offset, 1024)]) self.assertEqual(resp.error, 0) self.assertEqual(resp.partition, partition) diff --git a/test/testutil.py b/test/testutil.py index 3a1d2ba..b5b2529 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -11,7 +11,7 @@ from six.moves import xrange from . import unittest from kafka import KafkaClient -from kafka.common import OffsetRequest +from kafka.common import OffsetRequestPayload from kafka.util import kafka_bytestring __all__ = [ @@ -81,7 +81,7 @@ class KafkaIntegrationTestCase(unittest.TestCase): def current_offset(self, topic, partition): try: - offsets, = self.client.send_offset_request([ OffsetRequest(kafka_bytestring(topic), partition, -1, 1) ]) + offsets, = self.client.send_offset_request([OffsetRequestPayload(kafka_bytestring(topic), partition, -1, 1)]) except: # XXX: We've seen some UnknownErrors here and cant debug w/o server logs self.zk.child.dump_logs() -- cgit v1.2.1 From 172a272c4258ddb76f8e8e246ade1618116610c7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:45:18 -0800 Subject: Handle PartialMessage / ConsumerFetchSizeTooSmall in SimpleConsumer --- kafka/consumer/simple.py | 45 ++++++++++++++++++++++++--------------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 1c2aee6..9e76730 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -32,6 +32,7 @@ from ..common import ( UnknownTopicOrPartitionError, NotLeaderForPartitionError, OffsetOutOfRangeError, FailedPayloadsError, check_error ) +from kafka.protocol.message import PartialMessage log = logging.getLogger(__name__) @@ -407,32 +408,34 @@ class SimpleConsumer(Consumer): partition = resp.partition buffer_size = partitions[partition] - try: - for message in resp.messages: - if message.offset < self.fetch_offsets[partition]: - log.debug('Skipping message %s because its offset is less than the consumer offset', - message) - continue - # Put the message in our queue - self.queue.put((partition, message)) - self.fetch_offsets[partition] = message.offset + 1 - except ConsumerFetchSizeTooSmall: + + # Check for partial message + if resp.messages and isinstance(resp.messages[-1].message, PartialMessage): + + # If buffer is at max and all we got was a partial message + # raise ConsumerFetchSizeTooSmall if (self.max_buffer_size is not None and - buffer_size == self.max_buffer_size): - log.error('Max fetch size %d too small', - self.max_buffer_size) - raise + buffer_size == self.max_buffer_size and + len(resp.messages) == 1): + + log.error('Max fetch size %d too small', self.max_buffer_size) + raise ConsumerFetchSizeTooSmall() + if self.max_buffer_size is None: buffer_size *= 2 else: - buffer_size = min(buffer_size * 2, - self.max_buffer_size) + buffer_size = min(buffer_size * 2, self.max_buffer_size) log.warning('Fetch size too small, increase to %d (2x) ' 'and retry', buffer_size) retry_partitions[partition] = buffer_size - except ConsumerNoMoreData as e: - log.debug('Iteration was ended by %r', e) - except StopIteration: - # Stop iterating through this partition - log.debug('Done iterating over partition %s', partition) + resp.messages.pop() + + for message in resp.messages: + if message.offset < self.fetch_offsets[partition]: + log.debug('Skipping message %s because its offset is less than the consumer offset', + message) + continue + # Put the message in our queue + self.queue.put((partition, message)) + self.fetch_offsets[partition] = message.offset + 1 partitions = retry_partitions -- cgit v1.2.1 From 1a50de4c3c3fb06c97bb83ffb8854fc71a96a7a9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:46:36 -0800 Subject: Update client tests for new protocol usage --- test/test_client.py | 135 ++++++++++++++++++++++++---------------------------- 1 file changed, 61 insertions(+), 74 deletions(-) diff --git a/test/test_client.py b/test/test_client.py index bab7916..a3e04f4 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -7,14 +7,15 @@ from . import unittest from kafka import KafkaClient from kafka.common import ( - ProduceRequest, MetadataResponse, - BrokerMetadata, TopicMetadata, PartitionMetadata, + ProduceRequestPayload, + BrokerMetadata, TopicAndPartition, KafkaUnavailableError, LeaderNotAvailableError, UnknownTopicOrPartitionError, KafkaTimeoutError, ConnectionError ) from kafka.conn import KafkaConnection from kafka.protocol import KafkaProtocol, create_message +from kafka.protocol.metadata import MetadataResponse from test.testutil import Timer @@ -48,16 +49,14 @@ class TestKafkaClient(unittest.TestCase): sorted(client.hosts)) def test_send_broker_unaware_request_fail(self): - 'Tests that call fails when all hosts are unavailable' - mocked_conns = { ('kafka01', 9092): MagicMock(), ('kafka02', 9092): MagicMock() } # inject KafkaConnection side effects - mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)") - mocked_conns[('kafka02', 9092)].send.side_effect = RuntimeError("Kafka02 went away (unittest)") + mocked_conns[('kafka01', 9092)].send.return_value = None + mocked_conns[('kafka02', 9092)].send.return_value = None def mock_get_conn(host, port): return mocked_conns[(host, port)] @@ -67,27 +66,25 @@ class TestKafkaClient(unittest.TestCase): with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) - req = KafkaProtocol.encode_metadata_request(b'client', 0) + req = KafkaProtocol.encode_metadata_request() with self.assertRaises(KafkaUnavailableError): client._send_broker_unaware_request(payloads=['fake request'], encoder_fn=MagicMock(return_value='fake encoded message'), decoder_fn=lambda x: x) for key, conn in six.iteritems(mocked_conns): - conn.send.assert_called_with(ANY, 'fake encoded message') + conn.send.assert_called_with('fake encoded message') def test_send_broker_unaware_request(self): - 'Tests that call works when at least one of the host is available' - mocked_conns = { ('kafka01', 9092): MagicMock(), ('kafka02', 9092): MagicMock(), ('kafka03', 9092): MagicMock() } # inject KafkaConnection side effects - mocked_conns[('kafka01', 9092)].send.side_effect = RuntimeError("kafka01 went away (unittest)") + mocked_conns[('kafka01', 9092)].send.return_value = None mocked_conns[('kafka02', 9092)].recv.return_value = 'valid response' - mocked_conns[('kafka03', 9092)].send.side_effect = RuntimeError("kafka03 went away (unittest)") + mocked_conns[('kafka03', 9092)].send.return_value = None def mock_get_conn(host, port): return mocked_conns[(host, port)] @@ -95,17 +92,16 @@ class TestKafkaClient(unittest.TestCase): # patch to avoid making requests before we want it with patch.object(KafkaClient, 'load_metadata_for_topics'): with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): - with patch.object(KafkaClient, '_next_id', return_value=1): - client = KafkaClient(hosts='kafka01:9092,kafka02:9092') - resp = client._send_broker_unaware_request(payloads=['fake request'], - encoder_fn=MagicMock(), - decoder_fn=lambda x: x) + client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + resp = client._send_broker_unaware_request(payloads=['fake request'], + encoder_fn=MagicMock(), + decoder_fn=lambda x: x) - self.assertEqual('valid response', resp) - mocked_conns[('kafka02', 9092)].recv.assert_called_with(1) + self.assertEqual('valid response', resp) + mocked_conns[('kafka02', 9092)].recv.assert_called_once_with() - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_load_metadata(self, protocol, conn): @@ -117,21 +113,19 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata(b'topic_1', NO_ERROR, [ - PartitionMetadata(b'topic_1', 0, 1, [1, 2], [1, 2], NO_ERROR) + (NO_ERROR, 'topic_1', [ + (NO_ERROR, 0, 1, [1, 2], [1, 2]) ]), - TopicMetadata(b'topic_noleader', NO_ERROR, [ - PartitionMetadata(b'topic_noleader', 0, -1, [], [], - NO_LEADER), - PartitionMetadata(b'topic_noleader', 1, -1, [], [], - NO_LEADER), + (NO_ERROR, 'topic_noleader', [ + (NO_LEADER, 0, -1, [], []), + (NO_LEADER, 1, -1, [], []), ]), - TopicMetadata(b'topic_no_partitions', NO_LEADER, []), - TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), - TopicMetadata(b'topic_3', NO_ERROR, [ - PartitionMetadata(b'topic_3', 0, 0, [0, 1], [0, 1], NO_ERROR), - PartitionMetadata(b'topic_3', 1, 1, [1, 0], [1, 0], NO_ERROR), - PartitionMetadata(b'topic_3', 2, 0, [0, 1], [0, 1], NO_ERROR) + (NO_LEADER, 'topic_no_partitions', []), + (UNKNOWN_TOPIC_OR_PARTITION, 'topic_unknown', []), + (NO_ERROR, 'topic_3', [ + (NO_ERROR, 0, 0, [0, 1], [0, 1]), + (NO_ERROR, 1, 1, [1, 0], [1, 0]), + (NO_ERROR, 2, 0, [0, 1], [0, 1]) ]) ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -158,7 +152,7 @@ class TestKafkaClient(unittest.TestCase): client.load_metadata_for_topics('topic_no_leader') client.load_metadata_for_topics(b'topic_no_leader') - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_has_metadata_for_topic(self, protocol, conn): @@ -170,11 +164,11 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata(b'topic_still_creating', NO_LEADER, []), - TopicMetadata(b'topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), - TopicMetadata(b'topic_noleaders', NO_ERROR, [ - PartitionMetadata(b'topic_noleaders', 0, -1, [], [], NO_LEADER), - PartitionMetadata(b'topic_noleaders', 1, -1, [], [], NO_LEADER), + (NO_LEADER, 'topic_still_creating', []), + (UNKNOWN_TOPIC_OR_PARTITION, 'topic_doesnt_exist', []), + (NO_ERROR, 'topic_noleaders', [ + (NO_LEADER, 0, -1, [], []), + (NO_LEADER, 1, -1, [], []), ]), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -188,7 +182,7 @@ class TestKafkaClient(unittest.TestCase): # Topic with partition metadata, but no leaders return True self.assertTrue(client.has_metadata_for_topic('topic_noleaders')) - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol.decode_metadata_response') def test_ensure_topic_exists(self, decode_metadata_response, conn): @@ -200,11 +194,11 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata(b'topic_still_creating', NO_LEADER, []), - TopicMetadata(b'topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), - TopicMetadata(b'topic_noleaders', NO_ERROR, [ - PartitionMetadata(b'topic_noleaders', 0, -1, [], [], NO_LEADER), - PartitionMetadata(b'topic_noleaders', 1, -1, [], [], NO_LEADER), + (NO_LEADER, 'topic_still_creating', []), + (UNKNOWN_TOPIC_OR_PARTITION, 'topic_doesnt_exist', []), + (NO_ERROR, 'topic_noleaders', [ + (NO_LEADER, 0, -1, [], []), + (NO_LEADER, 1, -1, [], []), ]), ] decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -219,9 +213,8 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.ensure_topic_exists('topic_noleaders', timeout=1) - client.ensure_topic_exists(b'topic_noleaders', timeout=1) - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn): "Get leader for partitions reload metadata if it is not available" @@ -234,7 +227,7 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_no_partitions', NO_LEADER, []) + (NO_LEADER, 'topic_no_partitions', []) ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -244,8 +237,8 @@ class TestKafkaClient(unittest.TestCase): self.assertDictEqual({}, client.topics_to_brokers) topics = [ - TopicMetadata('topic_one_partition', NO_ERROR, [ - PartitionMetadata('topic_no_partition', 0, 0, [0, 1], [0, 1], NO_ERROR) + (NO_ERROR, 'topic_one_partition', [ + (NO_ERROR, 0, 0, [0, 1], [0, 1]) ]) ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -259,7 +252,7 @@ class TestKafkaClient(unittest.TestCase): TopicAndPartition('topic_one_partition', 0): brokers[0]}, client.topics_to_brokers) - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_get_leader_for_unassigned_partitions(self, protocol, conn): @@ -271,8 +264,8 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata(b'topic_no_partitions', NO_LEADER, []), - TopicMetadata(b'topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), + (NO_LEADER, 'topic_no_partitions', []), + (UNKNOWN_TOPIC_OR_PARTITION, 'topic_unknown', []), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -286,7 +279,7 @@ class TestKafkaClient(unittest.TestCase): with self.assertRaises(UnknownTopicOrPartitionError): client._get_leader_for_partition(b'topic_unknown', 0) - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_get_leader_exceptions_when_noleader(self, protocol, conn): @@ -298,11 +291,9 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_noleader', NO_ERROR, [ - PartitionMetadata('topic_noleader', 0, -1, [], [], - NO_LEADER), - PartitionMetadata('topic_noleader', 1, -1, [], [], - NO_LEADER), + (NO_ERROR, 'topic_noleader', [ + (NO_LEADER, 0, -1, [], []), + (NO_LEADER, 1, -1, [], []), ]), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) @@ -326,20 +317,18 @@ class TestKafkaClient(unittest.TestCase): self.assertIsNone(client._get_leader_for_partition('topic_noleader', 2)) topics = [ - TopicMetadata('topic_noleader', NO_ERROR, [ - PartitionMetadata('topic_noleader', 0, 0, [0, 1], [0, 1], NO_ERROR), - PartitionMetadata('topic_noleader', 1, 1, [1, 0], [1, 0], NO_ERROR) + (NO_ERROR, 'topic_noleader', [ + (NO_ERROR, 0, 0, [0, 1], [0, 1]), + (NO_ERROR, 1, 1, [1, 0], [1, 0]) ]), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) self.assertEqual(brokers[0], client._get_leader_for_partition('topic_noleader', 0)) self.assertEqual(brokers[1], client._get_leader_for_partition('topic_noleader', 1)) - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_noleader(self, protocol, conn): - "Send producer request raises LeaderNotAvailableError if leader is not available" - conn.recv.return_value = 'response' # anything but None brokers = [ @@ -348,25 +337,23 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_noleader', NO_ERROR, [ - PartitionMetadata('topic_noleader', 0, -1, [], [], - NO_LEADER), - PartitionMetadata('topic_noleader', 1, -1, [], [], - NO_LEADER), + (NO_ERROR, 'topic_noleader', [ + (NO_LEADER, 0, -1, [], []), + (NO_LEADER, 1, -1, [], []), ]), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) client = KafkaClient(hosts=['broker_1:4567']) - requests = [ProduceRequest( + requests = [ProduceRequestPayload( "topic_noleader", 0, [create_message("a"), create_message("b")])] with self.assertRaises(LeaderNotAvailableError): client.send_produce_request(requests) - @patch('kafka.client.KafkaConnection') + @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_topic_unknown(self, protocol, conn): @@ -378,13 +365,13 @@ class TestKafkaClient(unittest.TestCase): ] topics = [ - TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), + (UNKNOWN_TOPIC_OR_PARTITION, 'topic_doesnt_exist', []), ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) client = KafkaClient(hosts=['broker_1:4567']) - requests = [ProduceRequest( + requests = [ProduceRequestPayload( "topic_doesnt_exist", 0, [create_message("a"), create_message("b")])] -- cgit v1.2.1 From 99514c3337e507710a071ed19561502de5968a35 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:48:00 -0800 Subject: Skip consumer tests that cause iterator errors -- needs investigation --- test/test_consumer.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test/test_consumer.py b/test/test_consumer.py index df15115..97110e2 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -30,6 +30,7 @@ class TestMultiProcessConsumer(unittest.TestCase): self.assertEqual(client.get_partition_ids_for_topic.call_count, 0) # pylint: disable=no-member class TestSimpleConsumer(unittest.TestCase): + @unittest.skip def test_simple_consumer_failed_payloads(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, @@ -44,6 +45,7 @@ class TestSimpleConsumer(unittest.TestCase): # This should not raise an exception consumer.get_messages(5) + @unittest.skip def test_simple_consumer_leader_change(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, @@ -64,6 +66,7 @@ class TestSimpleConsumer(unittest.TestCase): self.assertGreaterEqual(client.reset_topic_metadata.call_count, 1) self.assertGreaterEqual(client.load_metadata_for_topics.call_count, 1) + @unittest.skip def test_simple_consumer_unknown_topic_partition(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, -- cgit v1.2.1 From 5c0bf5f377f15e301ad8ac011854db54ab0f6f92 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:48:30 -0800 Subject: Update to Payload namedtuples in consumer tests --- test/test_consumer.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index 97110e2..0cab116 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -4,7 +4,7 @@ from . import unittest from kafka import SimpleConsumer, KafkaConsumer, MultiProcessConsumer from kafka.common import ( - KafkaConfigurationError, FetchResponse, OffsetFetchResponse, + KafkaConfigurationError, FetchResponsePayload, OffsetFetchResponse, FailedPayloadsError, OffsetAndMessage, NotLeaderForPartitionError, UnknownTopicOrPartitionError ) @@ -54,7 +54,7 @@ class TestSimpleConsumer(unittest.TestCase): # Mock so that only the first request gets a valid response def not_leader(request): - return FetchResponse(request.topic, request.partition, + return FetchResponsePayload(request.topic, request.partition, NotLeaderForPartitionError.errno, -1, ()) client.send_fetch_request.side_effect = self.fail_requests_factory(not_leader) @@ -75,7 +75,7 @@ class TestSimpleConsumer(unittest.TestCase): # Mock so that only the first request gets a valid response def unknown_topic_partition(request): - return FetchResponse(request.topic, request.partition, + return FetchResponsePayload(request.topic, request.partition, UnknownTopicOrPartitionError.errno, -1, ()) client.send_fetch_request.side_effect = self.fail_requests_factory(unknown_topic_partition) @@ -128,7 +128,7 @@ class TestSimpleConsumer(unittest.TestCase): # Mock so that only the first request gets a valid response def fail_requests(payloads, **kwargs): responses = [ - FetchResponse(payloads[0].topic, payloads[0].partition, 0, 0, + FetchResponsePayload(payloads[0].topic, payloads[0].partition, 0, 0, (OffsetAndMessage( payloads[0].offset + i, "msg %d" % (payloads[0].offset + i)) -- cgit v1.2.1 From f304fb6c4d37a54d142a83e9f654d098347a0750 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:50:33 -0800 Subject: Skip protocol tests for now -- need updating --- test/test_protocol.py | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/test/test_protocol.py b/test/test_protocol.py index 368c2d0..9653ee3 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -193,6 +193,7 @@ class TestProtocol(unittest.TestCase): with self.assertRaises(ProtocolError): KafkaProtocol._encode_message(Message(1, 0, "key", "test")) + @unittest.skip('needs updating for new protocol classes') def test_encode_message_set(self): message_set = [ create_message(b"v1", b"k1"), @@ -222,6 +223,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(encoded, expect) + @unittest.skip('needs updating for new protocol classes') def test_decode_message_set(self): encoded = b"".join([ struct.pack(">q", 0), # MsgSet Offset @@ -256,6 +258,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(returned_offset2, 1) self.assertEqual(decoded_message2, create_message(b"v2", b"k2")) + @unittest.skip('needs updating for new protocol classes') def test_decode_message_gzip(self): gzip_encoded = (b'\xc0\x11\xb2\xf0\x00\x01\xff\xff\xff\xff\x00\x00\x000' b'\x1f\x8b\x08\x00\xa1\xc1\xc5R\x02\xffc`\x80\x03\x01' @@ -276,6 +279,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(returned_offset2, 0) self.assertEqual(decoded_message2, create_message(b"v2")) + @unittest.skip('needs updating for new protocol classes') @unittest.skipUnless(has_snappy(), "Snappy not available") def test_decode_message_snappy(self): snappy_encoded = (b'\xec\x80\xa1\x95\x00\x02\xff\xff\xff\xff\x00\x00' @@ -303,10 +307,12 @@ class TestProtocol(unittest.TestCase): # NOTE: The error handling in _decode_message_set_iter() is questionable. # If it's modified, the next two tests might need to be fixed. + @unittest.skip('needs updating for new protocol classes') def test_decode_message_set_fetch_size_too_small(self): with self.assertRaises(ConsumerFetchSizeTooSmall): list(KafkaProtocol._decode_message_set_iter('a')) + @unittest.skip('needs updating for new protocol classes') def test_decode_message_set_stop_iteration(self): encoded = b"".join([ struct.pack(">q", 0), # MsgSet Offset @@ -342,6 +348,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(returned_offset2, 1) self.assertEqual(decoded_message2, create_message(b"v2", b"k2")) + @unittest.skip('needs updating for new protocol classes') def test_encode_produce_request(self): requests = [ ProduceRequest(b"topic1", 0, [ @@ -398,6 +405,7 @@ class TestProtocol(unittest.TestCase): encoded = KafkaProtocol.encode_produce_request(b"client1", 2, requests, 2, 100) self.assertIn(encoded, [ expected1, expected2 ]) + @unittest.skip('needs updating for new protocol classes') def test_decode_produce_response(self): t1 = b"topic1" t2 = b"topic2" @@ -413,6 +421,7 @@ class TestProtocol(unittest.TestCase): ProduceResponse(t1, 1, 1, _long(20)), ProduceResponse(t2, 0, 0, _long(30))]) + @unittest.skip('needs updating for new protocol classes') def test_encode_fetch_request(self): requests = [ FetchRequest(b"topic1", 0, 10, 1024), @@ -453,6 +462,7 @@ class TestProtocol(unittest.TestCase): encoded = KafkaProtocol.encode_fetch_request(b"client1", 3, requests, 2, 100) self.assertIn(encoded, [ expected1, expected2 ]) + @unittest.skip('needs updating for new protocol classes') def test_decode_fetch_response(self): t1 = b"topic1" t2 = b"topic2" @@ -482,6 +492,7 @@ class TestProtocol(unittest.TestCase): OffsetAndMessage(0, msgs[4])])] self.assertEqual(expanded_responses, expect) + @unittest.skip('needs updating for new protocol classes') def test_encode_metadata_request_no_topics(self): expected = b"".join([ struct.pack(">i", 17), # Total length of the request @@ -496,6 +507,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(encoded, expected) + @unittest.skip('needs updating for new protocol classes') def test_encode_metadata_request_with_topics(self): expected = b"".join([ struct.pack(">i", 25), # Total length of the request @@ -539,6 +551,7 @@ class TestProtocol(unittest.TestCase): *metadata.isr)) return b''.join(encoded) + @unittest.skip('needs updating for new protocol classes') def test_decode_metadata_response(self): node_brokers = [ BrokerMetadata(0, b"brokers1.kafka.rdio.com", 1000), @@ -588,6 +601,7 @@ class TestProtocol(unittest.TestCase): ConsumerMetadataResponse(error = 0, nodeId = 1, host = b'brokers1.kafka.rdio.com', port = 1000) ) + @unittest.skip('needs updating for new protocol classes') def test_encode_offset_request(self): expected = b"".join([ struct.pack(">i", 21), # Total length of the request @@ -603,6 +617,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(encoded, expected) + @unittest.skip('needs updating for new protocol classes') def test_encode_offset_request__no_payload(self): expected = b"".join([ struct.pack(">i", 65), # Total length of the request @@ -632,6 +647,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(encoded, expected) + @unittest.skip('needs updating for new protocol classes') def test_decode_offset_response(self): encoded = b"".join([ struct.pack(">i", 42), # Correlation ID @@ -656,6 +672,7 @@ class TestProtocol(unittest.TestCase): OffsetResponse(topic = b'topic1', partition = 4, error = 0, offsets=(8,)), ])) + @unittest.skip('needs updating for new protocol classes') def test_encode_offset_commit_request(self): header = b"".join([ struct.pack('>i', 99), # Total message length @@ -698,6 +715,7 @@ class TestProtocol(unittest.TestCase): self.assertIn(encoded, [ expected1, expected2 ]) + @unittest.skip('needs updating for new protocol classes') def test_decode_offset_commit_response(self): encoded = b"".join([ struct.pack(">i", 42), # Correlation ID @@ -718,6 +736,7 @@ class TestProtocol(unittest.TestCase): OffsetCommitResponse(topic = b'topic1', partition = 4, error = 0), ])) + @unittest.skip('needs updating for new protocol classes') def test_encode_offset_fetch_request(self): header = b"".join([ struct.pack('>i', 69), # Total message length @@ -753,6 +772,7 @@ class TestProtocol(unittest.TestCase): self.assertIn(encoded, [ expected1, expected2 ]) + @unittest.skip('needs updating for new protocol classes') def test_decode_offset_fetch_response(self): encoded = b"".join([ struct.pack(">i", 42), # Correlation ID -- cgit v1.2.1 From 98f393ba6f8dda107c464069fa846e390aee9b42 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:51:11 -0800 Subject: Default consumer integration tests should not use offset commits --- test/test_consumer_integration.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index d536537..17c5844 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -60,10 +60,11 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): kwargs['group'] = None kwargs['auto_commit'] = False else: - kwargs.setdefault('auto_commit', True) + kwargs.setdefault('group', None) + kwargs.setdefault('auto_commit', False) consumer_class = kwargs.pop('consumer', SimpleConsumer) - group = kwargs.pop('group', self.id().encode('utf-8')) + group = kwargs.pop('group', None) topic = kwargs.pop('topic', self.topic) if consumer_class in [SimpleConsumer, MultiProcessConsumer]: -- cgit v1.2.1 From 06cc91c64cb35dd1c02597f091f8bf10d63daf3d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:53:07 -0800 Subject: Support requests that do not expect a response in BrokerConnection --- kafka/conn.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index e9877f2..4516ddc 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -61,7 +61,7 @@ class BrokerConnection(local): self._write_fd = None self.in_flight_requests.clear() - def send(self, request): + def send(self, request, expect_response=True): if not self.connected() and not self.connect(): return None self.correlation_id += 1 @@ -78,7 +78,8 @@ class BrokerConnection(local): log.exception("Error in BrokerConnection.send()") self.close() return None - self.in_flight_requests.append((self.correlation_id, request.RESPONSE_TYPE)) + if expect_response: + self.in_flight_requests.append((self.correlation_id, request.RESPONSE_TYPE)) return self.correlation_id def recv(self, timeout=None): -- cgit v1.2.1 From e37049fb691cdab1d18becf044aaeaf58d46b5d2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 15:58:18 -0800 Subject: Add next_correlation_id_recv + send helper methods to BrokerConnection --- kafka/conn.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/kafka/conn.py b/kafka/conn.py index 4516ddc..9907cb1 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -105,6 +105,14 @@ class BrokerConnection(local): return None return response + def next_correlation_id_recv(self): + if len(self.in_flight_requests) == 0: + return None + return self.in_flight_requests[0][0] + + def next_correlation_id_send(self): + return self.correlation_id + 1 + def __getnewargs__(self): return (self.host, self.port, self.timeout) -- cgit v1.2.1 From eea162eb0366ec15782568ae29e482814b06cc0e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 9 Dec 2015 16:03:33 -0800 Subject: Update kafka.common imports to Payloads namedtuples in test_protocol --- test/test_protocol.py | 46 ++++++++++++++++++++++++---------------------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/test/test_protocol.py b/test/test_protocol.py index 9653ee3..c5086b1 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -7,10 +7,10 @@ from . import unittest from kafka.codec import has_snappy, gzip_decode, snappy_decode from kafka.common import ( - OffsetRequest, OffsetCommitRequest, OffsetFetchRequest, - OffsetResponse, OffsetCommitResponse, OffsetFetchResponse, - ProduceRequest, FetchRequest, Message, ChecksumError, - ProduceResponse, FetchResponse, OffsetAndMessage, + OffsetRequestPayload, OffsetCommitRequest, OffsetFetchRequest, + OffsetResponsePayload, OffsetCommitResponse, OffsetFetchResponse, + ProduceRequestPayload, FetchRequestPayload, Message, ChecksumError, + ProduceResponsePayload, FetchResponsePayload, OffsetAndMessage, BrokerMetadata, TopicMetadata, PartitionMetadata, TopicAndPartition, KafkaUnavailableError, UnsupportedCodecError, ConsumerFetchSizeTooSmall, ProtocolError, ConsumerMetadataResponse @@ -335,28 +335,30 @@ class TestProtocol(unittest.TestCase): b"@1$%(Y!", # Random padding ]) - msgs = list(KafkaProtocol._decode_message_set_iter(encoded)) + msgs = MessageSet.decode(io.BytesIO(encoded)) self.assertEqual(len(msgs), 2) msg1, msg2 = msgs - returned_offset1, decoded_message1 = msg1 - returned_offset2, decoded_message2 = msg2 + returned_offset1, msg_size1, decoded_message1 = msg1 + returned_offset2, msg_size2, decoded_message2 = msg2 self.assertEqual(returned_offset1, 0) - self.assertEqual(decoded_message1, create_message(b"v1", b"k1")) + self.assertEqual(decoded_message1.value, b"v1") + self.assertEqual(decoded_message1.key, b"k1") self.assertEqual(returned_offset2, 1) - self.assertEqual(decoded_message2, create_message(b"v2", b"k2")) + self.assertEqual(decoded_message2.value, b"v2") + self.assertEqual(decoded_message2.key, b"k2") @unittest.skip('needs updating for new protocol classes') def test_encode_produce_request(self): requests = [ - ProduceRequest(b"topic1", 0, [ - create_message(b"a"), - create_message(b"b") + ProduceRequestPayload("topic1", 0, [ + kafka.protocol.message.Message(b"a"), + kafka.protocol.message.Message(b"b") ]), - ProduceRequest(b"topic2", 1, [ - create_message(b"c") + ProduceRequestPayload("topic2", 1, [ + kafka.protocol.message.Message(b"c") ]) ] @@ -480,16 +482,16 @@ class TestProtocol(unittest.TestCase): responses = list(KafkaProtocol.decode_fetch_response(encoded)) def expand_messages(response): - return FetchResponse(response.topic, response.partition, - response.error, response.highwaterMark, - list(response.messages)) + return FetchResponsePayload(response.topic, response.partition, + response.error, response.highwaterMark, + list(response.messages)) expanded_responses = list(map(expand_messages, responses)) - expect = [FetchResponse(t1, 0, 0, 10, [OffsetAndMessage(0, msgs[0]), - OffsetAndMessage(0, msgs[1])]), - FetchResponse(t1, 1, 1, 20, [OffsetAndMessage(0, msgs[2])]), - FetchResponse(t2, 0, 0, 30, [OffsetAndMessage(0, msgs[3]), - OffsetAndMessage(0, msgs[4])])] + expect = [FetchResponsePayload(t1, 0, 0, 10, [OffsetAndMessage(0, msgs[0]), + OffsetAndMessage(0, msgs[1])]), + FetchResponsePayload(t1, 1, 1, 20, [OffsetAndMessage(0, msgs[2])]), + FetchResponsePayload(t2, 0, 0, 30, [OffsetAndMessage(0, msgs[3]), + OffsetAndMessage(0, msgs[4])])] self.assertEqual(expanded_responses, expect) @unittest.skip('needs updating for new protocol classes') -- cgit v1.2.1 From c3d2fda3c368771cb93a09bb2f1edaa7a3cf9c2b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 09:57:39 -0800 Subject: Fix test_consumer generator/list mock, reenable tests --- test/test_consumer.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index 0cab116..31b7e72 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -30,7 +30,6 @@ class TestMultiProcessConsumer(unittest.TestCase): self.assertEqual(client.get_partition_ids_for_topic.call_count, 0) # pylint: disable=no-member class TestSimpleConsumer(unittest.TestCase): - @unittest.skip def test_simple_consumer_failed_payloads(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, @@ -45,7 +44,6 @@ class TestSimpleConsumer(unittest.TestCase): # This should not raise an exception consumer.get_messages(5) - @unittest.skip def test_simple_consumer_leader_change(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, @@ -66,7 +64,6 @@ class TestSimpleConsumer(unittest.TestCase): self.assertGreaterEqual(client.reset_topic_metadata.call_count, 1) self.assertGreaterEqual(client.load_metadata_for_topics.call_count, 1) - @unittest.skip def test_simple_consumer_unknown_topic_partition(self): client = MagicMock() consumer = SimpleConsumer(client, group=None, @@ -129,10 +126,10 @@ class TestSimpleConsumer(unittest.TestCase): def fail_requests(payloads, **kwargs): responses = [ FetchResponsePayload(payloads[0].topic, payloads[0].partition, 0, 0, - (OffsetAndMessage( + [OffsetAndMessage( payloads[0].offset + i, "msg %d" % (payloads[0].offset + i)) - for i in range(10))), + for i in range(10)]), ] for failure in payloads[1:]: responses.append(error_factory(failure)) -- cgit v1.2.1 From 7470cade6bb8629d17541e136527369f9d2ec387 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 10:57:27 -0800 Subject: Convert OffsetCommit and OffsetFetch protocol encode/decode --- kafka/common.py | 16 ++--- kafka/consumer/base.py | 6 +- kafka/consumer/kafka.py | 6 +- kafka/context.py | 4 +- kafka/protocol/legacy.py | 128 +++++++++++++++------------------------- test/test_client_integration.py | 6 +- test/test_consumer.py | 4 +- test/test_protocol.py | 4 +- 8 files changed, 69 insertions(+), 105 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 7ae3294..4fc1e19 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -28,30 +28,30 @@ ProduceResponsePayload = namedtuple("ProduceResponsePayload", ["topic", "partition", "error", "offset"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-FetchAPI -FetchRequestPayload = namedtuple("FetchRequest", +FetchRequestPayload = namedtuple("FetchRequestPayload", ["topic", "partition", "offset", "max_bytes"]) -FetchResponsePayload = namedtuple("FetchResponse", +FetchResponsePayload = namedtuple("FetchResponsePayload", ["topic", "partition", "error", "highwaterMark", "messages"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI -OffsetRequestPayload = namedtuple("OffsetRequest", +OffsetRequestPayload = namedtuple("OffsetRequestPayload", ["topic", "partition", "time", "max_offsets"]) -OffsetResponsePayload = namedtuple("OffsetResponse", +OffsetResponsePayload = namedtuple("OffsetResponsePayload", ["topic", "partition", "error", "offsets"]) # https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI -OffsetCommitRequest = namedtuple("OffsetCommitRequest", +OffsetCommitRequestPayload = namedtuple("OffsetCommitRequestPayload", ["topic", "partition", "offset", "metadata"]) -OffsetCommitResponse = namedtuple("OffsetCommitResponse", +OffsetCommitResponsePayload = namedtuple("OffsetCommitResponsePayload", ["topic", "partition", "error"]) -OffsetFetchRequest = namedtuple("OffsetFetchRequest", +OffsetFetchRequestPayload = namedtuple("OffsetFetchRequestPayload", ["topic", "partition"]) -OffsetFetchResponse = namedtuple("OffsetFetchResponse", +OffsetFetchResponsePayload = namedtuple("OffsetFetchResponsePayload", ["topic", "partition", "offset", "metadata", "error"]) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 034d35c..5859d36 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -7,7 +7,7 @@ from threading import Lock import kafka.common from kafka.common import ( - OffsetRequestPayload, OffsetCommitRequest, OffsetFetchRequest, + OffsetRequestPayload, OffsetCommitRequestPayload, OffsetFetchRequestPayload, UnknownTopicOrPartitionError, check_error, KafkaError ) @@ -101,7 +101,7 @@ class Consumer(object): responses = self.client.send_offset_fetch_request( self.group, - [OffsetFetchRequest(self.topic, p) for p in partitions], + [OffsetFetchRequestPayload(self.topic, p) for p in partitions], fail_on_error=False ) @@ -155,7 +155,7 @@ class Consumer(object): 'group=%s, topic=%s, partition=%s', offset, self.group, self.topic, partition) - reqs.append(OffsetCommitRequest(self.topic, partition, + reqs.append(OffsetCommitRequestPayload(self.topic, partition, offset, None)) try: diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 1bd3def..fa70124 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -11,7 +11,7 @@ import six from kafka.client import KafkaClient from kafka.common import ( - OffsetFetchRequest, OffsetCommitRequest, + OffsetFetchRequestPayload, OffsetCommitRequestPayload, OffsetRequestPayload, FetchRequestPayload, check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, @@ -546,7 +546,7 @@ class KafkaConsumer(object): continue commits.append( - OffsetCommitRequest(topic_partition[0], topic_partition[1], + OffsetCommitRequestPayload(topic_partition[0], topic_partition[1], commit_offset, metadata) ) @@ -618,7 +618,7 @@ class KafkaConsumer(object): for topic_partition in self._topics: (resp,) = self._client.send_offset_fetch_request( kafka_bytestring(self._config['group_id']), - [OffsetFetchRequest(topic_partition[0], topic_partition[1])], + [OffsetFetchRequestPayload(topic_partition[0], topic_partition[1])], fail_on_error=False) try: check_error(resp) diff --git a/kafka/context.py b/kafka/context.py index ade4db8..376fad1 100644 --- a/kafka/context.py +++ b/kafka/context.py @@ -3,7 +3,7 @@ Context manager to commit/rollback consumer offsets. """ from logging import getLogger -from kafka.common import check_error, OffsetCommitRequest, OffsetOutOfRangeError +from kafka.common import check_error, OffsetCommitRequestPayload, OffsetOutOfRangeError class OffsetCommitContext(object): @@ -139,7 +139,7 @@ class OffsetCommitContext(object): self.logger.debug("Committing partition offsets: %s", partition_offsets) commit_requests = [ - OffsetCommitRequest(self.consumer.topic, partition, offset, None) + OffsetCommitRequestPayload(self.consumer.topic, partition, offset, None) for partition, offset in partition_offsets.items() ] commit_responses = self.consumer.client.send_offset_commit_request( diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py index c5babf7..feabed3 100644 --- a/kafka/protocol/legacy.py +++ b/kafka/protocol/legacy.py @@ -19,7 +19,6 @@ from kafka.codec import ( gzip_encode, gzip_decode, snappy_encode, snappy_decode ) from kafka.common import ( - OffsetCommitResponse, OffsetFetchResponse, ProtocolError, ChecksumError, UnsupportedCodecError, ConsumerMetadataResponse @@ -258,8 +257,8 @@ class KafkaProtocol(object): partition, payload.time, payload.max_offsets) - for partition, payload in six.iteritems(topic_payloads)]) - for topic, topic_payloads in six.iteritems(group_by_topic_and_partition(payloads))]) + for partition, payload in six.iteritems(topic_payloads)]) + for topic, topic_payloads in six.iteritems(group_by_topic_and_partition(payloads))]) @classmethod def decode_offset_response(cls, response): @@ -327,115 +326,80 @@ class KafkaProtocol(object): return ConsumerMetadataResponse(error, nodeId, host, port) @classmethod - def encode_offset_commit_request(cls, client_id, correlation_id, - group, payloads): + def encode_offset_commit_request(cls, group, payloads): """ - Encode some OffsetCommitRequest structs + Encode an OffsetCommitRequest struct Arguments: - client_id: string - correlation_id: int group: string, the consumer group you are committing offsets for - payloads: list of OffsetCommitRequest + payloads: list of OffsetCommitRequestPayload """ - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_COMMIT_KEY)) - message.append(write_short_string(group)) - message.append(struct.pack('>i', len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>iq', partition, payload.offset)) - message.append(write_short_string(payload.metadata)) + return kafka.protocol.commit.OffsetCommitRequest_v0( + consumer_group=group, + topics=[( + topic, + [( + partition, + payload.offset, + payload.metadata) + for partition, payload in six.iteritems(topic_payloads)]) + for topic, topic_payloads in six.iteritems(group_by_topic_and_partition(payloads))]) - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) @classmethod - def decode_offset_commit_response(cls, data): + def decode_offset_commit_response(cls, response): """ - Decode bytes to an OffsetCommitResponse + Decode OffsetCommitResponse to an OffsetCommitResponsePayload Arguments: - data: bytes to decode + response: OffsetCommitResponse """ - ((correlation_id,), cur) = relative_unpack('>i', data, 0) - ((num_topics,), cur) = relative_unpack('>i', data, cur) - - for _ in xrange(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for _ in xrange(num_partitions): - ((partition, error), cur) = relative_unpack('>ih', data, cur) - yield OffsetCommitResponse(topic, partition, error) + return [ + kafka.common.OffsetCommitResponsePayload(topic, partition, error) + for topic, partitions in response.topics + for partition, error in partitions + ] @classmethod - def encode_offset_fetch_request(cls, client_id, correlation_id, - group, payloads, from_kafka=False): + def encode_offset_fetch_request(cls, group, payloads, from_kafka=False): """ - Encode some OffsetFetchRequest structs. The request is encoded using + Encode an OffsetFetchRequest struct. The request is encoded using version 0 if from_kafka is false, indicating a request for Zookeeper offsets. It is encoded using version 1 otherwise, indicating a request for Kafka offsets. Arguments: - client_id: string - correlation_id: int group: string, the consumer group you are fetching offsets for - payloads: list of OffsetFetchRequest + payloads: list of OffsetFetchRequestPayload from_kafka: bool, default False, set True for Kafka-committed offsets """ - grouped_payloads = group_by_topic_and_partition(payloads) - - message = [] - reqver = 1 if from_kafka else 0 - message.append(cls._encode_message_header(client_id, correlation_id, - KafkaProtocol.OFFSET_FETCH_KEY, - version=reqver)) - - message.append(write_short_string(group)) - message.append(struct.pack('>i', len(grouped_payloads))) - - for topic, topic_payloads in grouped_payloads.items(): - message.append(write_short_string(topic)) - message.append(struct.pack('>i', len(topic_payloads))) - - for partition, payload in topic_payloads.items(): - message.append(struct.pack('>i', partition)) + if from_kafka: + request_class = kafka.protocol.commit.OffsetFetchRequest_v1 + else: + request_class = kafka.protocol.commit.OffsetFetchRequest_v0 - msg = b''.join(message) - return struct.pack('>i%ds' % len(msg), len(msg), msg) + return request_class( + consumer_group=group, + topics=[( + topic, + list(topic_payloads.keys())) + for topic, topic_payloads in six.iteritems(group_by_topic_and_partition(payloads))]) @classmethod - def decode_offset_fetch_response(cls, data): + def decode_offset_fetch_response(cls, response): """ - Decode bytes to an OffsetFetchResponse + Decode OffsetFetchResponse to OffsetFetchResponsePayloads Arguments: - data: bytes to decode + response: OffsetFetchResponse """ - - ((correlation_id,), cur) = relative_unpack('>i', data, 0) - ((num_topics,), cur) = relative_unpack('>i', data, cur) - - for _ in range(num_topics): - (topic, cur) = read_short_string(data, cur) - ((num_partitions,), cur) = relative_unpack('>i', data, cur) - - for _ 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) + return [ + kafka.common.OffsetFetchResponsePayload( + topic, partition, offset, metadata, error + ) + for topic, partitions in response.topics + for partition, offset, metadata, error in partitions + ] def create_message(payload, key=None): diff --git a/test/test_client_integration.py b/test/test_client_integration.py index 70da4a3..edd62da 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -1,7 +1,7 @@ import os from kafka.common import ( - FetchRequestPayload, OffsetCommitRequest, OffsetFetchRequest, + FetchRequestPayload, OffsetCommitRequestPayload, OffsetFetchRequestPayload, KafkaTimeoutError, ProduceRequestPayload ) from kafka.protocol import create_message @@ -85,11 +85,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_commit_fetch_offsets(self): - req = OffsetCommitRequest(self.bytes_topic, 0, 42, b"metadata") + req = OffsetCommitRequestPayload(self.bytes_topic, 0, 42, b"metadata") (resp,) = self.client.send_offset_commit_request(b"group", [req]) self.assertEqual(resp.error, 0) - req = OffsetFetchRequest(self.bytes_topic, 0) + req = OffsetFetchRequestPayload(self.bytes_topic, 0) (resp,) = self.client.send_offset_fetch_request(b"group", [req]) self.assertEqual(resp.error, 0) self.assertEqual(resp.offset, 42) diff --git a/test/test_consumer.py b/test/test_consumer.py index 31b7e72..ffce578 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -4,7 +4,7 @@ from . import unittest from kafka import SimpleConsumer, KafkaConsumer, MultiProcessConsumer from kafka.common import ( - KafkaConfigurationError, FetchResponsePayload, OffsetFetchResponse, + KafkaConfigurationError, FetchResponsePayload, OffsetFetchResponsePayload, FailedPayloadsError, OffsetAndMessage, NotLeaderForPartitionError, UnknownTopicOrPartitionError ) @@ -86,7 +86,7 @@ class TestSimpleConsumer(unittest.TestCase): client.get_partition_ids_for_topic.return_value = [0, 1] def mock_offset_fetch_request(group, payloads, **kwargs): - return [OffsetFetchResponse(p.topic, p.partition, 0, b'', 0) for p in payloads] + return [OffsetFetchResponsePayload(p.topic, p.partition, 0, b'', 0) for p in payloads] client.send_offset_fetch_request.side_effect = mock_offset_fetch_request diff --git a/test/test_protocol.py b/test/test_protocol.py index c5086b1..8cd4fee 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -7,8 +7,8 @@ from . import unittest from kafka.codec import has_snappy, gzip_decode, snappy_decode from kafka.common import ( - OffsetRequestPayload, OffsetCommitRequest, OffsetFetchRequest, - OffsetResponsePayload, OffsetCommitResponse, OffsetFetchResponse, + OffsetRequestPayload, OffsetCommitRequestPayload, OffsetFetchRequestPayload, + OffsetResponsePayload, OffsetCommitResponsePayload, OffsetFetchResponsePayload, ProduceRequestPayload, FetchRequestPayload, Message, ChecksumError, ProduceResponsePayload, FetchResponsePayload, OffsetAndMessage, BrokerMetadata, TopicMetadata, PartitionMetadata, TopicAndPartition, -- cgit v1.2.1 From 7a804224949315251b9183fbfa56282ced881244 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 10:59:40 -0800 Subject: Configure consumer group in consumer_integration tests --- test/test_consumer_integration.py | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 17c5844..4cebed8 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -139,7 +139,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(1, range(100, 200)) # Create 1st consumer and change offsets - consumer = self.consumer() + consumer = self.consumer(group='test_simple_consumer_load_initial_offsets') self.assertEqual(consumer.offsets, {0: 0, 1: 0}) consumer.offsets.update({0:51, 1:101}) # Update counter after manual offsets update @@ -147,7 +147,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): consumer.commit() # Create 2nd consumer and check initial offsets - consumer = self.consumer(auto_commit=False) + consumer = self.consumer(group='test_simple_consumer_load_initial_offsets', + auto_commit=False) self.assertEqual(consumer.offsets, {0: 51, 1: 101}) @kafka_versions("all") @@ -315,7 +316,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(1, range(10, 20)) # Create 1st consumer and change offsets - consumer = self.consumer() + consumer = self.consumer(group='test_multi_process_consumer_load_initial_offsets') self.assertEqual(consumer.offsets, {0: 0, 1: 0}) consumer.offsets.update({0:5, 1:15}) # Update counter after manual offsets update @@ -324,6 +325,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # Create 2nd consumer and check initial offsets consumer = self.consumer(consumer = MultiProcessConsumer, + group='test_multi_process_consumer_load_initial_offsets', auto_commit=False) self.assertEqual(consumer.offsets, {0: 5, 1: 15}) @@ -382,6 +384,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # Start a consumer consumer1 = self.consumer( + group='test_offset_behavior__resuming_behavior', + auto_commit=True, auto_commit_every_t = None, auto_commit_every_n = 20, ) @@ -392,6 +396,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # The total offset across both partitions should be at 180 consumer2 = self.consumer( + group='test_offset_behavior__resuming_behavior', + auto_commit=True, auto_commit_every_t = None, auto_commit_every_n = 20, ) @@ -410,6 +416,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # Start a consumer consumer1 = self.consumer( consumer=MultiProcessConsumer, + group='test_multi_process_offset_behavior__resuming_behavior', + auto_commit=True, auto_commit_every_t = None, auto_commit_every_n = 20, ) @@ -427,6 +435,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # The total offset across both partitions should be at 180 consumer2 = self.consumer( consumer=MultiProcessConsumer, + group='test_multi_process_offset_behavior__resuming_behavior', + auto_commit=True, auto_commit_every_t = None, auto_commit_every_n = 20, ) -- cgit v1.2.1 From d54980a2cd918f243e30ecc23a588fb597957e41 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 16:24:32 -0800 Subject: Drop kafka_bytestring --- kafka/client.py | 7 ++----- kafka/consumer/base.py | 6 +++--- kafka/consumer/group.py | 15 +++++++-------- kafka/consumer/kafka.py | 16 +++++++--------- kafka/producer/base.py | 6 ------ kafka/producer/keyed.py | 2 -- kafka/producer/simple.py | 3 --- kafka/util.py | 12 ------------ test/test_client.py | 17 ++++++++--------- test/test_client_integration.py | 28 ++++++++++++++-------------- test/test_consumer_integration.py | 2 +- test/test_failover_integration.py | 5 ++--- test/test_producer_integration.py | 4 ++-- test/testutil.py | 5 +---- 14 files changed, 47 insertions(+), 81 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index cb60d98..ca737c4 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -17,7 +17,6 @@ from kafka.common import (TopicAndPartition, BrokerMetadata, UnknownError, from kafka.conn import collect_hosts, BrokerConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS from kafka.protocol import KafkaProtocol -from kafka.util import kafka_bytestring log = logging.getLogger(__name__) @@ -212,7 +211,7 @@ class KafkaClient(object): failed_payloads(broker_payloads) continue - conn = self._get_conn(broker.host.decode('utf-8'), broker.port) + conn = self._get_conn(broker.host, broker.port) request = encoder_fn(payloads=broker_payloads) # decoder_fn=None signal that the server is expected to not # send a response. This probably only applies to @@ -305,7 +304,7 @@ class KafkaClient(object): # Send the request, recv the response try: - conn = self._get_conn(broker.host.decode('utf-8'), broker.port) + conn = self._get_conn(broker.host, broker.port) conn.send(requestId, request) except ConnectionError as e: @@ -410,14 +409,12 @@ class KafkaClient(object): self.topic_partitions.clear() def has_metadata_for_topic(self, topic): - topic = kafka_bytestring(topic) return ( topic in self.topic_partitions and len(self.topic_partitions[topic]) > 0 ) def get_partition_ids_for_topic(self, topic): - topic = kafka_bytestring(topic) if topic not in self.topic_partitions: return [] diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 5859d36..4ac8c66 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -11,7 +11,7 @@ from kafka.common import ( UnknownTopicOrPartitionError, check_error, KafkaError ) -from kafka.util import kafka_bytestring, ReentrantTimer +from kafka.util import ReentrantTimer log = logging.getLogger('kafka.consumer') @@ -47,8 +47,8 @@ class Consumer(object): auto_commit_every_t=AUTO_COMMIT_INTERVAL): self.client = client - self.topic = kafka_bytestring(topic) - self.group = None if group is None else kafka_bytestring(group) + self.topic = topic + self.group = group self.client.load_metadata_for_topics(topic) self.offsets = {} diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 4a630ed..5d91469 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -16,7 +16,6 @@ from kafka.common import ( OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError ) -from kafka.util import kafka_bytestring logger = logging.getLogger(__name__) @@ -307,13 +306,13 @@ class KafkaConsumer(object): # Topic name str -- all partitions if isinstance(arg, (six.string_types, six.binary_type)): - topic = kafka_bytestring(arg) + topic = arg for partition in self._cluster.partitions_for_topic(topic): self._consume_topic_partition(topic, partition) # (topic, partition [, offset]) tuple elif isinstance(arg, tuple): - topic = kafka_bytestring(arg[0]) + topic = arg[0] partition = arg[1] self._consume_topic_partition(topic, partition) if len(arg) == 3: @@ -326,7 +325,7 @@ class KafkaConsumer(object): # key can be string (a topic) if isinstance(key, (six.string_types, six.binary_type)): - topic = kafka_bytestring(key) + topic = key # topic: partition if isinstance(value, int): @@ -344,7 +343,7 @@ class KafkaConsumer(object): # (topic, partition): offset elif isinstance(key, tuple): - topic = kafka_bytestring(key[0]) + topic = key[0] partition = key[1] self._consume_topic_partition(topic, partition) self._offsets.fetch[(topic, partition)] = value @@ -463,7 +462,7 @@ class KafkaConsumer(object): self._refresh_metadata_on_error() continue - topic = kafka_bytestring(resp.topic) + topic = resp.topic partition = resp.partition try: check_error(resp) @@ -662,7 +661,7 @@ class KafkaConsumer(object): if commits: logger.info('committing consumer offsets to group %s', self._config['group_id']) resps = self._client.send_offset_commit_request( - kafka_bytestring(self._config['group_id']), commits, + self._config['group_id'], commits, fail_on_error=False ) @@ -725,7 +724,7 @@ class KafkaConsumer(object): logger.info("Consumer fetching stored offsets") for topic_partition in self._topics: (resp,) = self._client.send_offset_fetch_request( - kafka_bytestring(self._config['group_id']), + self._config['group_id'], [OffsetFetchRequest(topic_partition[0], topic_partition[1])], fail_on_error=False) try: diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index fa70124..cdf8760 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -17,7 +17,6 @@ from kafka.common import ( OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError ) -from kafka.util import kafka_bytestring logger = logging.getLogger(__name__) @@ -193,14 +192,14 @@ class KafkaConsumer(object): # Topic name str -- all partitions if isinstance(arg, (six.string_types, six.binary_type)): - topic = kafka_bytestring(arg) + topic = arg for partition in self._client.get_partition_ids_for_topic(topic): self._consume_topic_partition(topic, partition) # (topic, partition [, offset]) tuple elif isinstance(arg, tuple): - topic = kafka_bytestring(arg[0]) + topic = arg[0] partition = arg[1] self._consume_topic_partition(topic, partition) if len(arg) == 3: @@ -213,7 +212,7 @@ class KafkaConsumer(object): # key can be string (a topic) if isinstance(key, (six.string_types, six.binary_type)): - topic = kafka_bytestring(key) + topic = key # topic: partition if isinstance(value, int): @@ -231,7 +230,7 @@ class KafkaConsumer(object): # (topic, partition): offset elif isinstance(key, tuple): - topic = kafka_bytestring(key[0]) + topic = key[0] partition = key[1] self._consume_topic_partition(topic, partition) self._offsets.fetch[(topic, partition)] = value @@ -354,7 +353,7 @@ class KafkaConsumer(object): self._refresh_metadata_on_error() continue - topic = kafka_bytestring(resp.topic) + topic = resp.topic partition = resp.partition try: check_error(resp) @@ -553,7 +552,7 @@ class KafkaConsumer(object): if commits: logger.info('committing consumer offsets to group %s', self._config['group_id']) resps = self._client.send_offset_commit_request( - kafka_bytestring(self._config['group_id']), commits, + self._config['group_id'], commits, fail_on_error=False ) @@ -577,7 +576,6 @@ class KafkaConsumer(object): # def _consume_topic_partition(self, topic, partition): - topic = kafka_bytestring(topic) if not isinstance(partition, int): raise KafkaConfigurationError('Unknown partition type (%s) ' '-- expected int' % type(partition)) @@ -617,7 +615,7 @@ class KafkaConsumer(object): logger.info("Consumer fetching stored offsets") for topic_partition in self._topics: (resp,) = self._client.send_offset_fetch_request( - kafka_bytestring(self._config['group_id']), + self._config['group_id'], [OffsetFetchRequestPayload(topic_partition[0], topic_partition[1])], fail_on_error=False) try: diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 3f2bba6..595ac37 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -22,7 +22,6 @@ from kafka.common import ( ) from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set -from kafka.util import kafka_bytestring log = logging.getLogger('kafka.producer') @@ -361,7 +360,6 @@ class Producer(object): All messages produced via this method will set the message 'key' to Null """ - topic = kafka_bytestring(topic) return self._send_messages(topic, partition, *msg) def _send_messages(self, topic, partition, *msg, **kwargs): @@ -381,10 +379,6 @@ class Producer(object): elif not isinstance(m, six.binary_type): raise TypeError("all produce message payloads must be null or type bytes") - # Raise TypeError if topic is not encoded as bytes - if not isinstance(topic, six.binary_type): - raise TypeError("the topic must be type bytes") - # Raise TypeError if the key is not encoded as bytes if key is not None and not isinstance(key, six.binary_type): raise TypeError("the key must be type bytes") diff --git a/kafka/producer/keyed.py b/kafka/producer/keyed.py index a5a26c9..f35aef0 100644 --- a/kafka/producer/keyed.py +++ b/kafka/producer/keyed.py @@ -5,7 +5,6 @@ import warnings from .base import Producer from ..partitioner import HashedPartitioner -from ..util import kafka_bytestring log = logging.getLogger(__name__) @@ -38,7 +37,6 @@ class KeyedProducer(Producer): return partitioner.partition(key) def send_messages(self, topic, key, *msg): - topic = kafka_bytestring(topic) partition = self._next_partition(topic, key) return self._send_messages(topic, partition, *msg, key=key) diff --git a/kafka/producer/simple.py b/kafka/producer/simple.py index 13e60d9..d8647b6 100644 --- a/kafka/producer/simple.py +++ b/kafka/producer/simple.py @@ -46,9 +46,6 @@ class SimpleProducer(Producer): return next(self.partition_cycles[topic]) def send_messages(self, topic, *msg): - if not isinstance(topic, six.binary_type): - topic = topic.encode('utf-8') - partition = self._next_partition(topic) return super(SimpleProducer, self).send_messages( topic, partition, *msg diff --git a/kafka/util.py b/kafka/util.py index e95d51d..c6e77fa 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -95,18 +95,6 @@ def group_by_topic_and_partition(tuples): return out -def kafka_bytestring(s): - """ - Takes a string or bytes instance - Returns bytes, encoding strings in utf-8 as necessary - """ - if isinstance(s, six.binary_type): - return s - if isinstance(s, six.string_types): - return s.encode('utf-8') - raise TypeError(s) - - class ReentrantTimer(object): """ A timer that can be restarted, unlike threading.Timer diff --git a/test/test_client.py b/test/test_client.py index a3e04f4..dd8948f 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -133,12 +133,12 @@ class TestKafkaClient(unittest.TestCase): # client loads metadata at init client = KafkaClient(hosts=['broker_1:4567']) self.assertDictEqual({ - TopicAndPartition(b'topic_1', 0): brokers[1], - TopicAndPartition(b'topic_noleader', 0): None, - TopicAndPartition(b'topic_noleader', 1): None, - TopicAndPartition(b'topic_3', 0): brokers[0], - TopicAndPartition(b'topic_3', 1): brokers[1], - TopicAndPartition(b'topic_3', 2): brokers[0]}, + TopicAndPartition('topic_1', 0): brokers[1], + TopicAndPartition('topic_noleader', 0): None, + TopicAndPartition('topic_noleader', 1): None, + TopicAndPartition('topic_3', 0): brokers[0], + TopicAndPartition('topic_3', 1): brokers[1], + TopicAndPartition('topic_3', 2): brokers[0]}, client.topics_to_brokers) # if we ask for metadata explicitly, it should raise errors @@ -150,7 +150,6 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.load_metadata_for_topics('topic_no_leader') - client.load_metadata_for_topics(b'topic_no_leader') @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') @@ -274,10 +273,10 @@ class TestKafkaClient(unittest.TestCase): self.assertDictEqual({}, client.topics_to_brokers) with self.assertRaises(LeaderNotAvailableError): - client._get_leader_for_partition(b'topic_no_partitions', 0) + client._get_leader_for_partition('topic_no_partitions', 0) with self.assertRaises(UnknownTopicOrPartitionError): - client._get_leader_for_partition(b'topic_unknown', 0) + client._get_leader_for_partition('topic_unknown', 0) @patch('kafka.client.BrokerConnection') @patch('kafka.client.KafkaProtocol') diff --git a/test/test_client_integration.py b/test/test_client_integration.py index edd62da..1599006 100644 --- a/test/test_client_integration.py +++ b/test/test_client_integration.py @@ -29,11 +29,11 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_consume_none(self): - fetch = FetchRequestPayload(self.bytes_topic, 0, 0, 1024) + fetch = FetchRequestPayload(self.topic, 0, 0, 1024) fetch_resp, = self.client.send_fetch_request([fetch]) self.assertEqual(fetch_resp.error, 0) - self.assertEqual(fetch_resp.topic, self.bytes_topic) + self.assertEqual(fetch_resp.topic, self.topic) self.assertEqual(fetch_resp.partition, 0) messages = list(fetch_resp.messages) @@ -48,26 +48,26 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): # ensure_topic_exists should fail with KafkaTimeoutError with self.assertRaises(KafkaTimeoutError): - self.client.ensure_topic_exists(b"this_topic_doesnt_exist", timeout=0) + self.client.ensure_topic_exists('this_topic_doesnt_exist', timeout=0) @kafka_versions('all') def test_send_produce_request_maintains_request_response_order(self): - self.client.ensure_topic_exists(b'foo') - self.client.ensure_topic_exists(b'bar') + self.client.ensure_topic_exists('foo') + self.client.ensure_topic_exists('bar') requests = [ ProduceRequestPayload( - b'foo', 0, + 'foo', 0, [create_message(b'a'), create_message(b'b')]), ProduceRequestPayload( - b'bar', 1, + 'bar', 1, [create_message(b'a'), create_message(b'b')]), ProduceRequestPayload( - b'foo', 1, + 'foo', 1, [create_message(b'a'), create_message(b'b')]), ProduceRequestPayload( - b'bar', 0, + 'bar', 0, [create_message(b'a'), create_message(b'b')]), ] @@ -85,12 +85,12 @@ class TestKafkaClientIntegration(KafkaIntegrationTestCase): @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_commit_fetch_offsets(self): - req = OffsetCommitRequestPayload(self.bytes_topic, 0, 42, b"metadata") - (resp,) = self.client.send_offset_commit_request(b"group", [req]) + req = OffsetCommitRequestPayload(self.topic, 0, 42, 'metadata') + (resp,) = self.client.send_offset_commit_request('group', [req]) self.assertEqual(resp.error, 0) - req = OffsetFetchRequestPayload(self.bytes_topic, 0) - (resp,) = self.client.send_offset_fetch_request(b"group", [req]) + req = OffsetFetchRequestPayload(self.topic, 0) + (resp,) = self.client.send_offset_fetch_request('group', [req]) self.assertEqual(resp.error, 0) self.assertEqual(resp.offset, 42) - self.assertEqual(resp.metadata, b"") # Metadata isn't stored for now + self.assertEqual(resp.metadata, '') # Metadata isn't stored for now diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 4cebed8..43370e2 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -41,7 +41,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): def send_messages(self, partition, messages): messages = [ create_message(self.msg(str(msg))) for msg in messages ] - produce = ProduceRequestPayload(self.bytes_topic, partition, messages = messages) + produce = ProduceRequestPayload(self.topic, partition, messages = messages) resp, = self.client.send_produce_request([produce]) self.assertEqual(resp.error, 0) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 91779d7..8c5efe2 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -5,7 +5,6 @@ import time from kafka import KafkaClient, SimpleConsumer, KeyedProducer from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer -from kafka.util import kafka_bytestring from test.fixtures import ZookeeperFixture, KafkaFixture from test.testutil import ( @@ -165,7 +164,7 @@ class TestFailover(KafkaIntegrationTestCase): key = random_string(3).encode('utf-8') msg = random_string(10).encode('utf-8') producer.send_messages(topic, key, msg) - if producer.partitioners[kafka_bytestring(topic)].partition(key) == 0: + if producer.partitioners[topic].partition(key) == 0: recovered = True except (FailedPayloadsError, ConnectionError): log.debug("caught exception sending message -- will retry") @@ -203,7 +202,7 @@ class TestFailover(KafkaIntegrationTestCase): break def _kill_leader(self, topic, partition): - leader = self.client.topics_to_brokers[TopicAndPartition(kafka_bytestring(topic), partition)] + leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] broker = self.brokers[leader.nodeId] broker.close() return broker diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index ee0b2fd..d09c1af 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -488,7 +488,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): def assert_produce_request(self, messages, initial_offset, message_ct, partition=0): - produce = ProduceRequestPayload(self.bytes_topic, partition, messages=messages) + produce = ProduceRequestPayload(self.topic, partition, messages=messages) # There should only be one response message from the server. # This will throw an exception if there's more than one. @@ -506,7 +506,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): # There should only be one response message from the server. # This will throw an exception if there's more than one. - resp, = self.client.send_fetch_request([FetchRequestPayload(self.bytes_topic, partition, start_offset, 1024)]) + resp, = self.client.send_fetch_request([FetchRequestPayload(self.topic, partition, start_offset, 1024)]) self.assertEqual(resp.error, 0) self.assertEqual(resp.partition, partition) diff --git a/test/testutil.py b/test/testutil.py index b5b2529..5c6ea1b 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -12,7 +12,6 @@ from . import unittest from kafka import KafkaClient from kafka.common import OffsetRequestPayload -from kafka.util import kafka_bytestring __all__ = [ 'random_string', @@ -50,7 +49,6 @@ def get_open_port(): class KafkaIntegrationTestCase(unittest.TestCase): create_client = True topic = None - bytes_topic = None zk = None server = None @@ -62,7 +60,6 @@ class KafkaIntegrationTestCase(unittest.TestCase): if not self.topic: topic = "%s-%s" % (self.id()[self.id().rindex(".") + 1:], random_string(10)) self.topic = topic - self.bytes_topic = topic.encode('utf-8') if self.create_client: self.client = KafkaClient('%s:%d' % (self.server.host, self.server.port)) @@ -81,7 +78,7 @@ class KafkaIntegrationTestCase(unittest.TestCase): def current_offset(self, topic, partition): try: - offsets, = self.client.send_offset_request([OffsetRequestPayload(kafka_bytestring(topic), partition, -1, 1)]) + offsets, = self.client.send_offset_request([OffsetRequestPayload(topic, partition, -1, 1)]) except: # XXX: We've seen some UnknownErrors here and cant debug w/o server logs self.zk.child.dump_logs() -- cgit v1.2.1 From b3bd4cf09baf7d46a3ef21b19d91b9e37679b26c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 16:26:14 -0800 Subject: Fix zk_chroot in consumer integration kafka fixtures --- test/test_consumer_integration.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 43370e2..a5878d1 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -25,8 +25,9 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): return cls.zk = ZookeeperFixture.instance() - cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) - cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port) + chroot = random_string(10) + cls.server1 = KafkaFixture.instance(0, cls.zk.host, cls.zk.port, chroot) + cls.server2 = KafkaFixture.instance(1, cls.zk.host, cls.zk.port, chroot) cls.server = cls.server1 # Bootstrapping server -- cgit v1.2.1 From 4d9f8971e114f38999592ccd149d1a59fcbb3883 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 16:26:45 -0800 Subject: Handle python3 in kafka.protocol.pickle --- kafka/protocol/pickle.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka/protocol/pickle.py b/kafka/protocol/pickle.py index af0d1ee..2265efd 100644 --- a/kafka/protocol/pickle.py +++ b/kafka/protocol/pickle.py @@ -1,6 +1,10 @@ from __future__ import absolute_import -import copy_reg +try: + import copyreg +except ImportError: + import copy_reg as copyreg # python2 + import types @@ -22,4 +26,4 @@ def _unpickle_method(func_name, obj, cls): return func.__get__(obj, cls) # https://bytes.com/topic/python/answers/552476-why-cant-you-pickle-instancemethods -copy_reg.pickle(types.MethodType, _pickle_method, _unpickle_method) +copyreg.pickle(types.MethodType, _pickle_method, _unpickle_method) -- cgit v1.2.1 From 8411c169ec4d24007e313bb71aa51049a5a42102 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 16:27:42 -0800 Subject: Add pylint hints to AbstractType because we cant mark a classmethod as abstract --- kafka/protocol/abstract.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/protocol/abstract.py b/kafka/protocol/abstract.py index 532d15e..160678f 100644 --- a/kafka/protocol/abstract.py +++ b/kafka/protocol/abstract.py @@ -5,11 +5,11 @@ class AbstractType(object): __metaclass__ = abc.ABCMeta @abc.abstractmethod - def encode(cls, value): + def encode(cls, value): # pylint: disable=no-self-argument pass @abc.abstractmethod - def decode(cls, data): + def decode(cls, data): # pylint: disable=no-self-argument pass @classmethod -- cgit v1.2.1 From 3662cb6c50f1a0f893cf23e18f4d14509b25dfa3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 17:24:39 -0800 Subject: Fix consumer integration test that assumed a single broker --- test/test_consumer_integration.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index a5878d1..eb87b00 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -193,13 +193,14 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.assert_message_count(messages, 0) self.assertGreaterEqual(t.interval, 1) - self.send_messages(0, range(0, 10)) + self.send_messages(0, range(0, 5)) + self.send_messages(1, range(5, 10)) # Ask for 5 messages, 10 in queue. Get 5 back, no blocking with Timer() as t: - messages = consumer.get_messages(count=5, block=True, timeout=5) + messages = consumer.get_messages(count=5, block=True, timeout=3) self.assert_message_count(messages, 5) - self.assertLessEqual(t.interval, 1) + self.assertLess(t.interval, 3) # Ask for 10 messages, get 5 back, block 1 second with Timer() as t: @@ -209,7 +210,8 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # Ask for 10 messages, 5 in queue, ask to block for 1 message or 1 # second, get 5 back, no blocking - self.send_messages(0, range(0, 5)) + self.send_messages(0, range(0, 3)) + self.send_messages(1, range(3, 5)) with Timer() as t: messages = consumer.get_messages(count=10, block=1, timeout=1) self.assert_message_count(messages, 5) -- cgit v1.2.1 From 5cf60f4482852220efa5f727b45617ee3b7ad773 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 17:48:05 -0800 Subject: Remove KafkaProtocol._decode_message --- kafka/protocol/legacy.py | 32 -------------------------------- test/test_protocol.py | 2 ++ 2 files changed, 2 insertions(+), 32 deletions(-) diff --git a/kafka/protocol/legacy.py b/kafka/protocol/legacy.py index feabed3..1835521 100644 --- a/kafka/protocol/legacy.py +++ b/kafka/protocol/legacy.py @@ -118,38 +118,6 @@ class KafkaProtocol(object): raise ProtocolError("Unexpected magic number: %d" % message.magic) return msg - @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 != crc32(data[4:]): - raise ChecksumError("Message checksum failed") - - (key, cur) = read_int_string(data, cur) - (value, cur) = read_int_string(data, cur) - - codec = att & ATTRIBUTE_CODEC_MASK - - if codec == CODEC_NONE: - yield (offset, kafka.common.Message(magic, att, key, value)) - - elif codec == CODEC_GZIP: - gz = gzip_decode(value) - for (offset, msg) in KafkaProtocol._decode_message_set_iter(gz): - yield (offset, msg) - - elif codec == CODEC_SNAPPY: - snp = snappy_decode(value) - for (offset, msg) in KafkaProtocol._decode_message_set_iter(snp): - yield (offset, msg) - ################## # Public API # ################## diff --git a/test/test_protocol.py b/test/test_protocol.py index 8cd4fee..7dfd44e 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -173,6 +173,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(encoded, expect) + @unittest.skip('needs updating for new protocol classes') def test_decode_message(self): encoded = b"".join([ struct.pack(">i", -1427009701), # CRC @@ -300,6 +301,7 @@ class TestProtocol(unittest.TestCase): self.assertEqual(returned_offset2, 0) self.assertEqual(decoded_message2, create_message(b"v2")) + @unittest.skip('needs updating for new protocol classes') def test_decode_message_checksum_error(self): invalid_encoded_message = b"This is not a valid encoded message" iter = KafkaProtocol._decode_message(invalid_encoded_message, 0) -- cgit v1.2.1 From 1cb732aba10e82232197e19fd84a79cbab6214c6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 17:48:12 -0800 Subject: pylint fixes - temporarily ignore kafka/consumer/group.py - temporarily ignore test/test_protocol.py - disable-msg deprecated; use disable= instead --- kafka/consumer/group.py | 1 + kafka/consumer/kafka.py | 2 +- kafka/consumer/simple.py | 4 ++-- kafka/protocol/struct.py | 2 +- test/fixtures.py | 2 +- test/test_protocol.py | 1 + 6 files changed, 7 insertions(+), 5 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 5d91469..dba5f60 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -1,3 +1,4 @@ +#pylint: skip-file from __future__ import absolute_import from collections import namedtuple diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index cdf8760..3f14444 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -664,7 +664,7 @@ class KafkaConsumer(object): # Otherwise we should re-raise the upstream exception # b/c it typically includes additional data about # the request that triggered it, and we do not want to drop that - raise # pylint: disable-msg=E0704 + raise # pylint: disable=E0704 (offset, ) = self.get_partition_offsets(topic, partition, request_time_ms, max_num_offsets=1) diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 9e76730..9c2812b 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -1,7 +1,7 @@ from __future__ import absolute_import try: - from itertools import zip_longest as izip_longest, repeat # pylint: disable-msg=E0611 + from itertools import zip_longest as izip_longest, repeat # pylint: disable=E0611 except ImportError: from itertools import izip_longest as izip_longest, repeat # python 2 import logging @@ -167,7 +167,7 @@ class SimpleConsumer(Consumer): # Otherwise we should re-raise the upstream exception # b/c it typically includes additional data about # the request that triggered it, and we do not want to drop that - raise # pylint: disable-msg=E0704 + raise # pylint: disable=E0704 # send_offset_request log.info('Resetting topic-partition offset to %s for %s:%d', diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index 5b4c312..d340abf 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -21,7 +21,7 @@ class Struct(AbstractType): self.encode = self._encode_self @classmethod - def encode(cls, item): # pylint: disable-msg=E0202 + def encode(cls, item): # pylint: disable=E0202 bits = [] for i, field in enumerate(cls.SCHEMA.fields): bits.append(field.encode(item[i])) diff --git a/test/fixtures.py b/test/fixtures.py index 0ae1c1e..91a67c1 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -8,7 +8,7 @@ import time from six.moves import urllib import uuid -from six.moves.urllib.parse import urlparse # pylint: disable-msg=E0611,F0401 +from six.moves.urllib.parse import urlparse # pylint: disable=E0611,F0401 from test.service import ExternalService, SpawnedService from test.testutil import get_open_port diff --git a/test/test_protocol.py b/test/test_protocol.py index 7dfd44e..6c79829 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -1,3 +1,4 @@ +#pylint: skip-file from contextlib import contextmanager import struct -- cgit v1.2.1 From 8fff81468df640c0c1fc5daeb8fd8dd980c15c0c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 18:38:34 -0800 Subject: Move Request / Response logging from KafkaClient to BrokerConnection and reenable kafka.conn debug logging in tests --- kafka/client.py | 2 -- kafka/conn.py | 6 ++++-- test/test_conn.py | 18 ------------------ test/testutil.py | 3 --- 4 files changed, 4 insertions(+), 25 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index ca737c4..e66190d 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -144,7 +144,6 @@ class KafkaClient(object): response = conn.recv() if response is not None: decoded = decoder_fn(response) - log.debug('Response %s: %s', correlation_id, decoded) return decoded raise KafkaUnavailableError('All servers failed to process request') @@ -250,7 +249,6 @@ class KafkaClient(object): 'from server %s', correlation_id, broker) continue - log.debug('Response %s: %s', correlation_id, response) for payload_response in decoder_fn(response): topic_partition = (str(payload_response.topic), payload_response.partition) diff --git a/kafka/conn.py b/kafka/conn.py index 9907cb1..bd399a9 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -75,11 +75,12 @@ class BrokerConnection(local): self._write_fd.write(message) self._write_fd.flush() except socket.error: - log.exception("Error in BrokerConnection.send()") + log.exception("Error in BrokerConnection.send(): %s", request) self.close() return None if expect_response: self.in_flight_requests.append((self.correlation_id, request.RESPONSE_TYPE)) + log.debug('Request %d: %s', self.correlation_id, request) return self.correlation_id def recv(self, timeout=None): @@ -100,9 +101,10 @@ class BrokerConnection(local): raise RuntimeError('Correlation ids do not match!') response = response_type.decode(self._read_fd) except (RuntimeError, socket.error, struct.error): - log.exception("Error in BrokerConnection.recv()") + log.exception("Error in BrokerConnection.recv() for request %d", correlation_id) self.close() return None + log.debug('Response %d: %s', correlation_id, response) return response def next_correlation_id_recv(self): diff --git a/test/test_conn.py b/test/test_conn.py index 1bdfc1e..684ffe5 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -1,4 +1,3 @@ -import logging import socket import struct from threading import Thread @@ -12,9 +11,6 @@ from kafka.conn import KafkaConnection, collect_hosts, DEFAULT_SOCKET_TIMEOUT_SE class ConnTest(unittest.TestCase): def setUp(self): - # kafka.conn debug logging is verbose, so only enable in conn tests - logging.getLogger('kafka.conn').setLevel(logging.DEBUG) - self.config = { 'host': 'localhost', 'port': 9090, @@ -50,11 +46,6 @@ class ConnTest(unittest.TestCase): # Reset any mock counts caused by __init__ self.MockCreateConn.reset_mock() - def tearDown(self): - # Return connection logging to INFO - logging.getLogger('kafka.conn').setLevel(logging.INFO) - - def test_collect_hosts__happy_path(self): hosts = "localhost:1234,localhost" results = collect_hosts(hosts) @@ -193,15 +184,6 @@ class ConnTest(unittest.TestCase): class TestKafkaConnection(unittest.TestCase): - - def setUp(self): - # kafka.conn debug logging is verbose, so only enable in conn tests - logging.getLogger('kafka.conn').setLevel(logging.DEBUG) - - def tearDown(self): - # Return connection logging to INFO - logging.getLogger('kafka.conn').setLevel(logging.INFO) - @mock.patch('socket.create_connection') def test_copy(self, socket): """KafkaConnection copies work as expected""" diff --git a/test/testutil.py b/test/testutil.py index 5c6ea1b..98fe805 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -112,6 +112,3 @@ class Timer(object): logging.basicConfig(level=logging.DEBUG) logging.getLogger('test.fixtures').setLevel(logging.ERROR) logging.getLogger('test.service').setLevel(logging.ERROR) - -# kafka.conn debug logging is verbose, disable in tests by default -logging.getLogger('kafka.conn').setLevel(logging.INFO) -- cgit v1.2.1 From 06e1b0329d0304b0fab3eaad9799f9f9967271f5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 18:39:23 -0800 Subject: Only log topic names (and broker metadata) in KafkaClient load_metadata debug logging --- kafka/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client.py b/kafka/client.py index e66190d..02f438a 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -473,7 +473,7 @@ class KafkaClient(object): resp = self.send_metadata_request(topics) log.debug('Updating broker metadata: %s', resp.brokers) - log.debug('Updating topic metadata: %s', resp.topics) + log.debug('Updating topic metadata: %s', [topic for _, topic, _ in resp.topics]) self.brokers = dict([(nodeId, BrokerMetadata(nodeId, host, port)) for nodeId, host, port in resp.brokers]) -- cgit v1.2.1 From f86068a51a50dcd472d2c5ddf62fc61545e7f173 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 18:40:11 -0800 Subject: Check for no in-flight-requests in BrokerConnection.recv --- kafka/conn.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/conn.py b/kafka/conn.py index bd399a9..d45b824 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -89,6 +89,9 @@ class BrokerConnection(local): readable, _, _ = select([self._read_fd], [], [], timeout) if not readable: return None + if not self.in_flight_requests: + log.warning('No in-flight-requests to recv') + return None correlation_id, response_type = self.in_flight_requests.popleft() # Current implementation does not use size # instead we read directly from the socket fd buffer -- cgit v1.2.1 From b528410818db4882755a14d89da16fc957c3b268 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 18:52:18 -0800 Subject: Remove bytes encoding for topic name in producer integration test --- test/test_producer_integration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_producer_integration.py b/test/test_producer_integration.py index d09c1af..26300db 100644 --- a/test/test_producer_integration.py +++ b/test/test_producer_integration.py @@ -166,7 +166,7 @@ class TestKafkaProducerIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_produce__new_topic_fails_with_reasonable_error(self): - new_topic = 'new_topic_{guid}'.format(guid = str(uuid.uuid4())).encode('utf-8') + new_topic = 'new_topic_{guid}'.format(guid = str(uuid.uuid4())) producer = SimpleProducer(self.client, random_start=False) # At first it doesn't exist -- cgit v1.2.1 From 9da5e49331bee811a858373888f8cbcabc3db634 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 10 Dec 2015 18:52:44 -0800 Subject: Drop bytes encoding of consumer group in consumer integration test --- test/test_consumer_integration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index eb87b00..cd5af5e 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -525,7 +525,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): @kafka_versions("0.8.1", "0.8.1.1", "0.8.2.1") def test_kafka_consumer__offset_commit_resume(self): - GROUP_ID = random_string(10).encode('utf-8') + GROUP_ID = random_string(10) self.send_messages(0, range(0, 100)) self.send_messages(1, range(100, 200)) -- cgit v1.2.1 From 799824535ceeb698152a3078f64ecbf6baca9b39 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Tue, 15 Dec 2015 16:29:53 -0800 Subject: new 0.9 error codes plus descriptions. reference: https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes --- kafka/client.py | 4 +- kafka/common.py | 178 +++++++++++++++++++++++++++++++++++++++++++++++++++++--- 2 files changed, 172 insertions(+), 10 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 02f438a..b09927d 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -103,10 +103,10 @@ class KafkaClient(object): """ Returns the coordinator broker for a consumer group. - ConsumerCoordinatorNotAvailableCode will be raised if the coordinator + GroupCoordinatorNotAvailableError will be raised if the coordinator does not currently exist for the group. - OffsetsLoadInProgressCode is raised if the coordinator is available + GroupLoadInProgressError is raised if the coordinator is available but is still loading offsets from the internal topic """ diff --git a/kafka/common.py b/kafka/common.py index 4fc1e19..253137c 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -94,92 +94,254 @@ class KafkaError(RuntimeError): class BrokerResponseError(KafkaError): - pass + errno = None + message = None + description = None + + def __str__(self): + return '%s - %s - %s' % (self.__class__.__name__, self.errno, self.description) + + +class NoError(BrokerResponseError): + errno = 0 + message = 'NO_ERROR' + description = 'No error--it worked!' class UnknownError(BrokerResponseError): errno = -1 message = 'UNKNOWN' + description = 'An unexpected server error.' class OffsetOutOfRangeError(BrokerResponseError): errno = 1 message = 'OFFSET_OUT_OF_RANGE' + description = ('The requested offset is outside the range of offsets' + ' maintained by the server for the given topic/partition.') class InvalidMessageError(BrokerResponseError): errno = 2 message = 'INVALID_MESSAGE' + description = ('This indicates that a message contents does not match its' + ' CRC.') class UnknownTopicOrPartitionError(BrokerResponseError): errno = 3 message = 'UNKNOWN_TOPIC_OR_PARTITON' + description = ('This request is for a topic or partition that does not' + ' exist on this broker.') class InvalidFetchRequestError(BrokerResponseError): errno = 4 message = 'INVALID_FETCH_SIZE' + description = 'The message has a negative size.' class LeaderNotAvailableError(BrokerResponseError): errno = 5 message = 'LEADER_NOT_AVAILABLE' + description = ('This error is thrown if we are in the middle of a' + 'leadership election and there is currently no leader for' + 'this partition and hence it is unavailable for writes.') class NotLeaderForPartitionError(BrokerResponseError): errno = 6 message = 'NOT_LEADER_FOR_PARTITION' + description = ('This error is thrown if the client attempts to send' + ' messages to a replica that is not the leader for some' + ' partition. It indicates that the clients metadata is out' + ' of date.') class RequestTimedOutError(BrokerResponseError): errno = 7 message = 'REQUEST_TIMED_OUT' + description = ('This error is thrown if the request exceeds the' + ' user-specified time limit in the request.') class BrokerNotAvailableError(BrokerResponseError): errno = 8 message = 'BROKER_NOT_AVAILABLE' - + description = ('This is not a client facing error and is used mostly by' + ' tools when a broker is not alive.') class ReplicaNotAvailableError(BrokerResponseError): errno = 9 message = 'REPLICA_NOT_AVAILABLE' + description = ('If replica is expected on a broker, but is not (this can be' + ' safely ignored).') class MessageSizeTooLargeError(BrokerResponseError): errno = 10 message = 'MESSAGE_SIZE_TOO_LARGE' + description = ('The server has a configurable maximum message size to avoid' + ' unbounded memory allocation. This error is thrown if the' + ' client attempt to produce a message larger than this' + 'maximum.') class StaleControllerEpochError(BrokerResponseError): errno = 11 message = 'STALE_CONTROLLER_EPOCH' + description = 'Internal error code for broker-to-broker communication.' class OffsetMetadataTooLargeError(BrokerResponseError): errno = 12 message = 'OFFSET_METADATA_TOO_LARGE' + description = ('If you specify a string larger than configured maximum for' + ' offset metadata.') +# TODO is this deprecated? https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes class StaleLeaderEpochCodeError(BrokerResponseError): errno = 13 message = 'STALE_LEADER_EPOCH_CODE' -class OffsetsLoadInProgressCode(BrokerResponseError): +class GroupLoadInProgressError(BrokerResponseError): errno = 14 - message = 'OFFSETS_LOAD_IN_PROGRESS_CODE' + message = 'OFFSETS_LOAD_IN_PROGRESS' + description = ('The broker returns this error code for an offset fetch' + ' request if it is still loading offsets (after a leader' + ' change for that offsets topic partition), or in response' + ' to group membership requests (such as heartbeats) when' + ' group metadata is being loaded by the coordinator.') -class ConsumerCoordinatorNotAvailableCode(BrokerResponseError): +class GroupCoordinatorNotAvailableError(BrokerResponseError): errno = 15 - message = 'CONSUMER_COORDINATOR_NOT_AVAILABLE_CODE' + message = 'CONSUMER_COORDINATOR_NOT_AVAILABLE' + description = ('The broker returns this error code for group coordinator' + ' requests, offset commits, and most group management' + ' requests if the offsets topic has not yet been created, or' + ' if the group coordinator is not active.') -class NotCoordinatorForConsumerCode(BrokerResponseError): +class NotCoordinatorForGroupError(BrokerResponseError): errno = 16 - message = 'NOT_COORDINATOR_FOR_CONSUMER_CODE' + message = 'NOT_COORDINATOR_FOR_CONSUMER' + description = ('The broker returns this error code if it receives an offset' + ' fetch or commit request for a group that it is not a' + ' coordinator for.') + + +class InvalidTopicError(BrokerResponseError): + errno = 17 + message = 'INVALID_TOPIC' + description = ('For a request which attempts to access an invalid topic' + ' (e.g. one which has an illegal name), or if an attempt' + ' is made to write to an internal topic (such as the' + ' consumer offsets topic).') + + +class RecordListTooLargeError(BrokerResponseError): + errno = 18 + message = 'RECORD_LIST_TOO_LARGE' + description = ('If a message batch in a produce request exceeds the maximum' + ' configured segment size.') + + +class NotEnoughReplicasError(BrokerResponseError): + errno = 19 + message = 'NOT_ENOUGH_REPLICAS' + description = ('Returned from a produce request when the number of in-sync' + ' replicas is lower than the configured minimum and' + ' requiredAcks is -1.') + + +class NotEnoughReplicasAfterAppendError(BrokerResponseError): + errno = 20 + message = 'NOT_ENOUGH_REPLICAS_AFTER_APPEND' + description = ('Returned from a produce request when the message was' + ' written to the log, but with fewer in-sync replicas than' + ' required.') + + +class InvalidRequiredAcksError(BrokerResponseError): + errno = 21 + message = 'INVALID_REQUIRED_ACKS' + description = ('Returned from a produce request if the requested' + ' requiredAcks is invalid (anything other than -1, 1, or 0).') + + +class IllegalGenerationError(BrokerResponseError): + errno = 22 + message = 'ILLEGAL_GENERATION' + description = ('Returned from group membership requests (such as heartbeats)' + ' when the generation id provided in the request is not the' + ' current generation.') + + +class InconsistentGroupProtocolError(BrokerResponseError): + errno = 23 + message = 'INCONSISTENT_GROUP_PROTOCOL' + description = ('Returned in join group when the member provides a protocol' + ' type or set of protocols which is not compatible with the current group.') + + +class InvalidGroupIdError(BrokerResponseError): + errno = 24 + message = 'INVALID_GROUP_ID' + description = 'Returned in join group when the groupId is empty or null.' + + +class UnknownMemberIdError(BrokerResponseError): + errno = 25 + message = 'UNKNOWN_MEMBER_ID' + description = ('Returned from group requests (offset commits/fetches,' + ' heartbeats, etc) when the memberId is not in the current' + ' generation.') + + +class InvalidSessionTimeoutError(BrokerResponseError): + errno = 26 + message = 'INVALID_SESSION_TIMEOUT' + description = ('Return in join group when the requested session timeout is' + ' outside of the allowed range on the broker') + + +class RebalanceInProgressError(BrokerResponseError): + errno = 27 + message = 'REBALANCE_IN_PROGRESS' + description = ('Returned in heartbeat requests when the coordinator has' + ' begun rebalancing the group. This indicates to the client' + ' that it should rejoin the group.') + + +class InvalidCommitOffsetSizeError(BrokerResponseError): + errno = 28 + message = 'INVALID_COMMIT_OFFSET_SIZE' + description = ('This error indicates that an offset commit was rejected' + ' because of oversize metadata.') + + +class TopicAuthorizationFailedError(BrokerResponseError): + errno = 29 + message = 'TOPIC_AUTHORIZATION_FAILED' + description = ('Returned by the broker when the client is not authorized to' + ' access the requested topic.') + + +class GroupAuthorizationFailedError(BrokerResponseError): + errno = 30 + message = 'GROUP_AUTHORIZATION_FAILED' + description = ('Returned by the broker when the client is not authorized to' + ' access a particular groupId.') + + +class ClusterAuthorizationFailedError(BrokerResponseError): + errno = 31 + message = 'CLUSTER_AUTHORIZATION_FAILED' + description = ('Returned by the broker when the client is not authorized to' + ' use an inter-broker or administrative API.') class KafkaUnavailableError(KafkaError): -- cgit v1.2.1 From f1ad0247df5bf6e0315ffbb1633d5979da828de0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Dec 2015 17:29:54 -0800 Subject: Switch BrokerConnection to (mostly) non-blocking IO. - return kafka.Future on send() - recv is now non-blocking call that completes futures when possible - update KafkaClient to block on future completion --- kafka/client.py | 90 +++++++++------- kafka/cluster.py | 2 +- kafka/common.py | 16 +++ kafka/conn.py | 299 ++++++++++++++++++++++++++++++++++++++-------------- kafka/future.py | 51 +++++++++ test/test_client.py | 82 ++++++++------ 6 files changed, 386 insertions(+), 154 deletions(-) create mode 100644 kafka/future.py diff --git a/kafka/client.py b/kafka/client.py index b09927d..7f9969e 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -3,7 +3,6 @@ import copy import functools import logging import random -import select import time import six @@ -15,7 +14,9 @@ from kafka.common import (TopicAndPartition, BrokerMetadata, UnknownError, LeaderNotAvailableError, UnknownTopicOrPartitionError, NotLeaderForPartitionError, ReplicaNotAvailableError) -from kafka.conn import collect_hosts, BrokerConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS +from kafka.conn import ( + collect_hosts, BrokerConnection, DEFAULT_SOCKET_TIMEOUT_SECONDS, + ConnectionStates) from kafka.protocol import KafkaProtocol @@ -45,7 +46,6 @@ class KafkaClient(object): self.load_metadata_for_topics() # bootstrap with all metadata - ################## # Private API # ################## @@ -56,11 +56,14 @@ class KafkaClient(object): if host_key not in self._conns: self._conns[host_key] = BrokerConnection( host, port, - timeout=self.timeout, + request_timeout_ms=self.timeout * 1000, client_id=self.client_id ) - return self._conns[host_key] + conn = self._conns[host_key] + while conn.connect() == ConnectionStates.CONNECTING: + pass + return conn def _get_leader_for_partition(self, topic, partition): """ @@ -137,16 +140,23 @@ class KafkaClient(object): for (host, port) in hosts: conn = self._get_conn(host, port) + if not conn.connected(): + log.warning("Skipping unconnected connection: %s", conn) + continue request = encoder_fn(payloads=payloads) - correlation_id = conn.send(request) - if correlation_id is None: + future = conn.send(request) + + # Block + while not future.is_done: + conn.recv() + + if future.failed(): + log.error("Request failed: %s", future.exception) continue - response = conn.recv() - if response is not None: - decoded = decoder_fn(response) - return decoded - raise KafkaUnavailableError('All servers failed to process request') + return decoder_fn(future.value) + + raise KafkaUnavailableError('All servers failed to process request: %s' % hosts) def _payloads_by_broker(self, payloads): payloads_by_broker = collections.defaultdict(list) @@ -204,55 +214,59 @@ class KafkaClient(object): # For each BrokerConnection keep the real socket so that we can use # a select to perform unblocking I/O - connections_by_socket = {} + connections_by_future = {} for broker, broker_payloads in six.iteritems(payloads_by_broker): if broker is None: failed_payloads(broker_payloads) continue conn = self._get_conn(broker.host, broker.port) + conn.connect() + if not conn.connected(): + refresh_metadata = True + failed_payloads(broker_payloads) + continue + request = encoder_fn(payloads=broker_payloads) # decoder_fn=None signal that the server is expected to not # send a response. This probably only applies to # ProduceRequest w/ acks = 0 expect_response = (decoder_fn is not None) - correlation_id = conn.send(request, expect_response=expect_response) + future = conn.send(request, expect_response=expect_response) - if correlation_id is None: + if future.failed(): refresh_metadata = True failed_payloads(broker_payloads) - log.warning('Error attempting to send request %s ' - 'to server %s', correlation_id, broker) continue if not expect_response: - log.debug('Request %s does not expect a response ' - '(skipping conn.recv)', correlation_id) for payload in broker_payloads: topic_partition = (str(payload.topic), payload.partition) responses[topic_partition] = None continue - connections_by_socket[conn._read_fd] = (conn, broker) + connections_by_future[future] = (conn, broker) conn = None - while connections_by_socket: - sockets = connections_by_socket.keys() - rlist, _, _ = select.select(sockets, [], [], None) - conn, broker = connections_by_socket.pop(rlist[0]) - correlation_id = conn.next_correlation_id_recv() - response = conn.recv() - if response is None: - refresh_metadata = True - failed_payloads(payloads_by_broker[broker]) - log.warning('Error receiving response to request %s ' - 'from server %s', correlation_id, broker) - continue + while connections_by_future: + futures = list(connections_by_future.keys()) + for future in futures: + + if not future.is_done: + conn, _ = connections_by_future[future] + conn.recv() + continue - for payload_response in decoder_fn(response): - topic_partition = (str(payload_response.topic), - payload_response.partition) - responses[topic_partition] = payload_response + _, broker = connections_by_future.pop(future) + if future.failed(): + refresh_metadata = True + failed_payloads(payloads_by_broker[broker]) + + else: + for payload_response in decoder_fn(future.value): + topic_partition = (str(payload_response.topic), + payload_response.partition) + responses[topic_partition] = payload_response if refresh_metadata: self.reset_all_metadata() @@ -392,7 +406,9 @@ class KafkaClient(object): def reinit(self): for conn in self._conns.values(): - conn.reinit() + conn.close() + while conn.connect() == ConnectionStates.CONNECTING: + pass def reset_topic_metadata(self, *topics): for topic in topics: diff --git a/kafka/cluster.py b/kafka/cluster.py index 55765dc..15921dc 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -73,7 +73,7 @@ class Cluster(object): def _bootstrap(self, hosts, timeout=2): for host, port in hosts: - conn = BrokerConnection(host, port, timeout) + conn = BrokerConnection(host, port) if not conn.connect(): continue self._brokers['bootstrap'] = conn diff --git a/kafka/common.py b/kafka/common.py index 253137c..173fc82 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -93,6 +93,22 @@ class KafkaError(RuntimeError): pass +class IllegalStateError(KafkaError): + pass + + +class RetriableError(KafkaError): + pass + + +class DisconnectError(KafkaError): + pass + + +class CorrelationIdError(KafkaError): + pass + + class BrokerResponseError(KafkaError): errno = None message = None diff --git a/kafka/conn.py b/kafka/conn.py index d45b824..c2b8fb0 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -1,15 +1,20 @@ -from collections import deque +import collections import copy +import errno import logging +import io from random import shuffle from select import select import socket import struct from threading import local +import time import six +import kafka.common as Errors from kafka.common import ConnectionError +from kafka.future import Future from kafka.protocol.api import RequestHeader from kafka.protocol.types import Int32 @@ -20,106 +25,244 @@ DEFAULT_SOCKET_TIMEOUT_SECONDS = 120 DEFAULT_KAFKA_PORT = 9092 -class BrokerConnection(local): - def __init__(self, host, port, timeout=DEFAULT_SOCKET_TIMEOUT_SECONDS, - client_id='kafka-python-0.10.0', correlation_id=0): - super(BrokerConnection, self).__init__() +class ConnectionStates(object): + DISCONNECTED = 1 + CONNECTING = 2 + CONNECTED = 3 + + +InFlightRequest = collections.namedtuple('InFlightRequest', + ['request', 'response_type', 'correlation_id', 'future', 'timestamp']) + + +class BrokerConnection(object): + _receive_buffer_bytes = 32768 + _send_buffer_bytes = 32768 + _client_id = 'kafka-python-0.10.0' + _correlation_id = 0 + _request_timeout_ms = 40000 + + def __init__(self, host, port, **kwargs): self.host = host self.port = port - self.timeout = timeout - self._write_fd = None - self._read_fd = None - self.correlation_id = correlation_id - self.client_id = client_id - self.in_flight_requests = deque() + self.in_flight_requests = collections.deque() + + for config in ('receive_buffer_bytes', 'send_buffer_bytes', + 'client_id', 'correlation_id', 'request_timeout_ms'): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) + + self.state = ConnectionStates.DISCONNECTED + self._sock = None + self._rbuffer = io.BytesIO() + self._receiving = False + self._next_payload_bytes = 0 + self._last_connection_attempt = None + self._last_connection_failure = None def connect(self): - if self.connected(): + """Attempt to connect and return ConnectionState""" + if self.state is ConnectionStates.DISCONNECTED: self.close() - try: - sock = socket.create_connection((self.host, self.port), self.timeout) - self._write_fd = sock.makefile('wb') - self._read_fd = sock.makefile('rb') - except socket.error: - log.exception("Error in BrokerConnection.connect()") - return None - self.in_flight_requests.clear() - return True + self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_RCVBUF, self._receive_buffer_bytes) + self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_SNDBUF, self._send_buffer_bytes) + self._sock.setblocking(False) + ret = self._sock.connect_ex((self.host, self.port)) + self._last_connection_attempt = time.time() + + if not ret or ret is errno.EISCONN: + self.state = ConnectionStates.CONNECTED + elif ret in (errno.EINPROGRESS, errno.EALREADY): + self.state = ConnectionStates.CONNECTING + else: + log.error('Connect attempt returned error %s. Disconnecting.', ret) + self.close() + self._last_connection_failure = time.time() + + if self.state is ConnectionStates.CONNECTING: + # in non-blocking mode, use repeated calls to socket.connect_ex + # to check connection status + if time.time() > (self._request_timeout_ms / 1000.0) + self._last_connection_attempt: + log.error('Connection attempt timed out') + self.close() # error=TimeoutError ? + self._last_connection_failure = time.time() + + ret = self._sock.connect_ex((self.host, self.port)) + if not ret or ret is errno.EISCONN: + self.state = ConnectionStates.CONNECTED + elif ret is not errno.EALREADY: + log.error('Connect attempt returned error %s. Disconnecting.', ret) + self.close() + self._last_connection_failure = time.time() + return self.state def connected(self): - return (self._read_fd is not None and self._write_fd is not None) + return self.state is ConnectionStates.CONNECTED - def close(self): - if self.connected(): - try: - self._read_fd.close() - self._write_fd.close() - except socket.error: - log.exception("Error in BrokerConnection.close()") - pass - self._read_fd = None - self._write_fd = None + def close(self, error=None): + if self._sock: + self._sock.close() + self._sock = None + self.state = ConnectionStates.DISCONNECTED + + if error is None: + error = Errors.DisconnectError() + while self.in_flight_requests: + ifr = self.in_flight_requests.popleft() + ifr.future.failure(error) self.in_flight_requests.clear() + self._receiving = False + self._next_payload_bytes = 0 + self._rbuffer.seek(0) + self._rbuffer.truncate() def send(self, request, expect_response=True): - if not self.connected() and not self.connect(): - return None - self.correlation_id += 1 + """send request, return Future() + + Can block on network if request is larger than send_buffer_bytes + """ + future = Future() + if not self.connected(): + return future.failure(Errors.DisconnectError()) + self._correlation_id += 1 header = RequestHeader(request, - correlation_id=self.correlation_id, - client_id=self.client_id) + correlation_id=self._correlation_id, + client_id=self._client_id) message = b''.join([header.encode(), request.encode()]) size = Int32.encode(len(message)) try: - self._write_fd.write(size) - self._write_fd.write(message) - self._write_fd.flush() - except socket.error: - log.exception("Error in BrokerConnection.send(): %s", request) - self.close() - return None + # In the future we might manage an internal write buffer + # and send bytes asynchronously. For now, just block + # sending each request payload + self._sock.setblocking(True) + sent_bytes = self._sock.send(size) + assert sent_bytes == len(size) + sent_bytes = self._sock.send(message) + assert sent_bytes == len(message) + self._sock.setblocking(False) + except (AssertionError, socket.error) as e: + log.debug("Error in BrokerConnection.send(): %s", request) + self.close(error=e) + return future.failure(e) + log.debug('Request %d: %s', self._correlation_id, request) + if expect_response: - self.in_flight_requests.append((self.correlation_id, request.RESPONSE_TYPE)) - log.debug('Request %d: %s', self.correlation_id, request) - return self.correlation_id + ifr = InFlightRequest(request=request, + correlation_id=self._correlation_id, + response_type=request.RESPONSE_TYPE, + future=future, + timestamp=time.time()) + self.in_flight_requests.append(ifr) + else: + future.success(None) + + return future + + def recv(self, timeout=0): + """Non-blocking network receive - def recv(self, timeout=None): + Return response if available + """ if not self.connected(): + log.warning('Cannot recv: socket not connected') + # If requests are pending, we should close the socket and + # fail all the pending request futures + if self.in_flight_requests: + self.close() return None - readable, _, _ = select([self._read_fd], [], [], timeout) - if not readable: - return None + if not self.in_flight_requests: log.warning('No in-flight-requests to recv') return None - correlation_id, response_type = self.in_flight_requests.popleft() - # Current implementation does not use size - # instead we read directly from the socket fd buffer - # alternatively, we could read size bytes into a separate buffer - # and decode from that buffer (and verify buffer is empty afterwards) - try: - size = Int32.decode(self._read_fd) - recv_correlation_id = Int32.decode(self._read_fd) - if correlation_id != recv_correlation_id: - raise RuntimeError('Correlation ids do not match!') - response = response_type.decode(self._read_fd) - except (RuntimeError, socket.error, struct.error): - log.exception("Error in BrokerConnection.recv() for request %d", correlation_id) - self.close() + + self._fail_timed_out_requests() + + readable, _, _ = select([self._sock], [], [], timeout) + if not readable: return None - log.debug('Response %d: %s', correlation_id, response) - return response - def next_correlation_id_recv(self): - if len(self.in_flight_requests) == 0: + # Not receiving is the state of reading the payload header + if not self._receiving: + try: + # An extremely small, but non-zero, probability that there are + # more than 0 but not yet 4 bytes available to read + self._rbuffer.write(self._sock.recv(4 - self._rbuffer.tell())) + except socket.error as e: + if e.errno == errno.EWOULDBLOCK: + # This shouldn't happen after selecting above + # but just in case + return None + log.exception("Error receiving 4-byte payload header - closing socket") + self.close(error=e) + return None + + if self._rbuffer.tell() == 4: + self._rbuffer.seek(0) + self._next_payload_bytes = Int32.decode(self._rbuffer) + # reset buffer and switch state to receiving payload bytes + self._rbuffer.seek(0) + self._rbuffer.truncate() + self._receiving = True + elif self._rbuffer.tell() > 4: + raise Errors.KafkaError('this should not happen - are you threading?') + + if self._receiving: + staged_bytes = self._rbuffer.tell() + try: + self._rbuffer.write(self._sock.recv(self._next_payload_bytes - staged_bytes)) + except socket.error as e: + # Extremely small chance that we have exactly 4 bytes for a + # header, but nothing to read in the body yet + if e.errno == errno.EWOULDBLOCK: + return None + log.exception() + self.close(error=e) + return None + + staged_bytes = self._rbuffer.tell() + if staged_bytes > self._next_payload_bytes: + self.close(error=Errors.KafkaError('Receive buffer has more bytes than expected?')) + + if staged_bytes != self._next_payload_bytes: + return None + + self._receiving = False + self._next_payload_bytes = 0 + self._rbuffer.seek(0) + response = self._process_response(self._rbuffer) + self._rbuffer.seek(0) + self._rbuffer.truncate() + return response + + def _process_response(self, read_buffer): + ifr = self.in_flight_requests.popleft() + + # verify send/recv correlation ids match + recv_correlation_id = Int32.decode(read_buffer) + if ifr.correlation_id != recv_correlation_id: + error = Errors.CorrelationIdError( + 'Correlation ids do not match: sent %d, recv %d' + % (ifr.correlation_id, recv_correlation_id)) + ifr.future.fail(error) + self.close() return None - return self.in_flight_requests[0][0] - def next_correlation_id_send(self): - return self.correlation_id + 1 + # decode response + response = ifr.response_type.decode(read_buffer) + ifr.future.success(response) + log.debug('Response %d: %s', ifr.correlation_id, response) + return response - def __getnewargs__(self): - return (self.host, self.port, self.timeout) + def _fail_timed_out_requests(self): + now = time.time() + while self.in_flight_requests: + next_timeout = self.in_flight_requests[0].timestamp + (self._request_timeout_ms / 1000.0) + if now < next_timeout: + break + timed_out = self.in_flight_requests.popleft() + error = Errors.RequestTimedOutError('Request timed out after %s ms' % self._request_timeout_ms) + timed_out.future.failure(error) def __repr__(self): return "" % (self.host, self.port) @@ -149,13 +292,7 @@ def collect_hosts(hosts, randomize=True): class KafkaConnection(local): - """ - 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. + """A socket connection to a single Kafka broker Arguments: host: the host name or IP address of a kafka broker diff --git a/kafka/future.py b/kafka/future.py new file mode 100644 index 0000000..24173bb --- /dev/null +++ b/kafka/future.py @@ -0,0 +1,51 @@ +from kafka.common import RetriableError, IllegalStateError + + +class Future(object): + def __init__(self): + self.is_done = False + self.value = None + self.exception = None + self._callbacks = [] + self._errbacks = [] + + def succeeded(self): + return self.is_done and not self.exception + + def failed(self): + return self.is_done and self.exception + + def retriable(self): + return isinstance(self.exception, RetriableError) + + def success(self, value): + if self.is_done: + raise IllegalStateError('Invalid attempt to complete a request future which is already complete') + self.value = value + self.is_done = True + for f in self._callbacks: + f(value) + return self + + def failure(self, e): + if self.is_done: + raise IllegalStateError('Invalid attempt to complete a request future which is already complete') + self.exception = e + self.is_done = True + for f in self._errbacks: + f(e) + return self + + def add_callback(self, f): + if self.is_done and not self.exception: + f(self.value) + else: + self._callbacks.append(f) + return self + + def add_errback(self, f): + if self.is_done and self.exception: + f(self.exception) + else: + self._errbacks.append(f) + return self diff --git a/test/test_client.py b/test/test_client.py index dd8948f..00e888c 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -14,6 +14,7 @@ from kafka.common import ( KafkaTimeoutError, ConnectionError ) from kafka.conn import KafkaConnection +from kafka.future import Future from kafka.protocol import KafkaProtocol, create_message from kafka.protocol.metadata import MetadataResponse @@ -23,6 +24,17 @@ NO_ERROR = 0 UNKNOWN_TOPIC_OR_PARTITION = 3 NO_LEADER = 5 + +def mock_conn(conn, success=True): + mocked = MagicMock() + mocked.connected.return_value = True + if success: + mocked.send.return_value = Future().success(True) + else: + mocked.send.return_value = Future().failure(Exception()) + conn.return_value = mocked + + class TestKafkaClient(unittest.TestCase): def test_init_with_list(self): with patch.object(KafkaClient, 'load_metadata_for_topics'): @@ -48,32 +60,30 @@ class TestKafkaClient(unittest.TestCase): sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]), sorted(client.hosts)) - def test_send_broker_unaware_request_fail(self): + @patch.object(KafkaClient, '_get_conn') + @patch.object(KafkaClient, 'load_metadata_for_topics') + def test_send_broker_unaware_request_fail(self, load_metadata, conn): mocked_conns = { ('kafka01', 9092): MagicMock(), ('kafka02', 9092): MagicMock() } - - # inject KafkaConnection side effects - mocked_conns[('kafka01', 9092)].send.return_value = None - mocked_conns[('kafka02', 9092)].send.return_value = None + for val in mocked_conns.values(): + mock_conn(val, success=False) def mock_get_conn(host, port): return mocked_conns[(host, port)] + conn.side_effect = mock_get_conn - # patch to avoid making requests before we want it - with patch.object(KafkaClient, 'load_metadata_for_topics'): - with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): - client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) + client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) - req = KafkaProtocol.encode_metadata_request() - with self.assertRaises(KafkaUnavailableError): - client._send_broker_unaware_request(payloads=['fake request'], - encoder_fn=MagicMock(return_value='fake encoded message'), - decoder_fn=lambda x: x) + req = KafkaProtocol.encode_metadata_request() + with self.assertRaises(KafkaUnavailableError): + client._send_broker_unaware_request(payloads=['fake request'], + encoder_fn=MagicMock(return_value='fake encoded message'), + decoder_fn=lambda x: x) - for key, conn in six.iteritems(mocked_conns): - conn.send.assert_called_with('fake encoded message') + for key, conn in six.iteritems(mocked_conns): + conn.send.assert_called_with('fake encoded message') def test_send_broker_unaware_request(self): mocked_conns = { @@ -82,9 +92,11 @@ class TestKafkaClient(unittest.TestCase): ('kafka03', 9092): MagicMock() } # inject KafkaConnection side effects - mocked_conns[('kafka01', 9092)].send.return_value = None - mocked_conns[('kafka02', 9092)].recv.return_value = 'valid response' - mocked_conns[('kafka03', 9092)].send.return_value = None + mock_conn(mocked_conns[('kafka01', 9092)], success=False) + mock_conn(mocked_conns[('kafka03', 9092)], success=False) + future = Future() + mocked_conns[('kafka02', 9092)].send.return_value = future + mocked_conns[('kafka02', 9092)].recv.side_effect = lambda: future.success('valid response') def mock_get_conn(host, port): return mocked_conns[(host, port)] @@ -101,11 +113,11 @@ class TestKafkaClient(unittest.TestCase): self.assertEqual('valid response', resp) mocked_conns[('kafka02', 9092)].recv.assert_called_once_with() - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_load_metadata(self, protocol, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -151,11 +163,11 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.load_metadata_for_topics('topic_no_leader') - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_has_metadata_for_topic(self, protocol, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -181,11 +193,11 @@ class TestKafkaClient(unittest.TestCase): # Topic with partition metadata, but no leaders return True self.assertTrue(client.has_metadata_for_topic('topic_noleaders')) - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol.decode_metadata_response') def test_ensure_topic_exists(self, decode_metadata_response, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -213,12 +225,12 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.ensure_topic_exists('topic_noleaders', timeout=1) - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn): "Get leader for partitions reload metadata if it is not available" - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -251,11 +263,11 @@ class TestKafkaClient(unittest.TestCase): TopicAndPartition('topic_one_partition', 0): brokers[0]}, client.topics_to_brokers) - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_get_leader_for_unassigned_partitions(self, protocol, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -278,11 +290,11 @@ class TestKafkaClient(unittest.TestCase): with self.assertRaises(UnknownTopicOrPartitionError): client._get_leader_for_partition('topic_unknown', 0) - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_get_leader_exceptions_when_noleader(self, protocol, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -325,10 +337,10 @@ class TestKafkaClient(unittest.TestCase): self.assertEqual(brokers[0], client._get_leader_for_partition('topic_noleader', 0)) self.assertEqual(brokers[1], client._get_leader_for_partition('topic_noleader', 1)) - @patch('kafka.client.BrokerConnection') + @patch.object(KafkaClient, '_get_conn') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_noleader(self, protocol, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), @@ -352,11 +364,11 @@ class TestKafkaClient(unittest.TestCase): with self.assertRaises(LeaderNotAvailableError): client.send_produce_request(requests) - @patch('kafka.client.BrokerConnection') + @patch('kafka.client.KafkaClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_topic_unknown(self, protocol, conn): - conn.recv.return_value = 'response' # anything but None + mock_conn(conn) brokers = [ BrokerMetadata(0, 'broker_1', 4567), -- cgit v1.2.1 From a6aa6d53fa794956a2b3504f4e08c0966df8c423 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 18 Dec 2015 09:49:58 -0800 Subject: Only download/install server artifacts if bin/kafka-run-class.sh not found --- build_integration.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/build_integration.sh b/build_integration.sh index aa9c399..ef4c869 100755 --- a/build_integration.sh +++ b/build_integration.sh @@ -51,14 +51,15 @@ pushd servers else KAFKA_ARTIFACT="kafka_${SCALA_VERSION}-${kafka}" fi - wget -N https://archive.apache.org/dist/kafka/$kafka/${KAFKA_ARTIFACT}.tgz || wget -N https://archive.apache.org/dist/kafka/$kafka/${KAFKA_ARTIFACT}.tar.gz - echo - if [ ! -d "../$kafka/kafka-bin" ]; then - echo "Extracting kafka binaries for ${kafka}" + if [ ! -f "../$kafka/kafka-bin/bin/kafka-run-class.sh" ]; then + echo "Downloading kafka ${kafka} tarball" + wget -N https://archive.apache.org/dist/kafka/$kafka/${KAFKA_ARTIFACT}.tgz || wget -N https://archive.apache.org/dist/kafka/$kafka/${KAFKA_ARTIFACT}.tar.gz + echo + echo "Extracting kafka ${kafka} binaries" tar xzvf ${KAFKA_ARTIFACT}.t* -C ../$kafka/ mv ../$kafka/${KAFKA_ARTIFACT} ../$kafka/kafka-bin else - echo "$kafka/kafka-bin directory already exists -- skipping tgz extraction" + echo "$kafka is already installed in servers/$kafka/ -- skipping" fi fi echo -- cgit v1.2.1 From d203b900eafbe8153e0cb3411ea25adebb827bf7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 18 Dec 2015 11:17:27 -0800 Subject: Remove old kafka-bin directory and verify kafka-run-class.sh exists when installing server artifacts --- build_integration.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/build_integration.sh b/build_integration.sh index ef4c869..613c291 100755 --- a/build_integration.sh +++ b/build_integration.sh @@ -40,6 +40,7 @@ pushd servers # Not sure how to construct the .tgz name accurately, so use a wildcard (ugh) tar xzvf $kafka/core/build/distributions/kafka_*.tgz -C ../$kafka/ rm $kafka/core/build/distributions/kafka_*.tgz + rm -rf ../$kafka/kafka-bin mv ../$kafka/kafka_* ../$kafka/kafka-bin else echo "-------------------------------------" @@ -57,7 +58,12 @@ pushd servers echo echo "Extracting kafka ${kafka} binaries" tar xzvf ${KAFKA_ARTIFACT}.t* -C ../$kafka/ + rm -rf ../$kafka/kafka-bin mv ../$kafka/${KAFKA_ARTIFACT} ../$kafka/kafka-bin + if [ ! -f "../$kafka/kafka-bin/bin/kafka-run-class.sh" ]; then + echo "Extraction Failed ($kafka/kafka-bin/bin/kafka-run-class.sh does not exist)!" + exit 1 + fi else echo "$kafka is already installed in servers/$kafka/ -- skipping" fi -- cgit v1.2.1 From cda2e17cd115f76f4992a34bab2b684ed08d4fef Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 20 Dec 2015 10:19:11 -0800 Subject: Rename Cluster -> ClusterMetadata; align with upstream Metadata class --- kafka/cluster.py | 187 +++++++++++++++++++++++++++++++++++-------------------- kafka/common.py | 4 ++ 2 files changed, 125 insertions(+), 66 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index 15921dc..2e9e117 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -1,91 +1,146 @@ +from __future__ import absolute_import + import logging import random +import time -from .conn import BrokerConnection, collect_hosts -from .protocol.metadata import MetadataRequest +import kafka.common as Errors +from kafka.common import BrokerMetadata +from .future import Future -logger = logging.getLogger(__name__) +log = logging.getLogger(__name__) -class Cluster(object): - def __init__(self, **kwargs): - if 'bootstrap_servers' not in kwargs: - kwargs['bootstrap_servers'] = 'localhost' +class ClusterMetadata(object): + _retry_backoff_ms = 100 + _metadata_max_age_ms = 300000 + def __init__(self, **kwargs): self._brokers = {} - self._topics = {} + self._partitions = {} self._groups = {} + self._version = 0 + self._last_refresh_ms = 0 + self._last_successful_refresh_ms = 0 + self._need_update = False + self._future = None + self._listeners = set() - self._bootstrap(collect_hosts(kwargs['bootstrap_servers']), - timeout=kwargs.get('bootstrap_timeout', 2)) + for config in ('retry_backoff_ms', 'metadata_max_age_ms'): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) def brokers(self): - brokers = list(self._brokers.values()) - return random.sample(brokers, len(brokers)) + return set(self._brokers.values()) - def random_broker(self): - for broker in self.brokers(): - if broker.connected() or broker.connect(): - return broker - return None - - def broker_by_id(self, broker_id): + def broker_metadata(self, broker_id): return self._brokers.get(broker_id) - def topics(self): - return list(self._topics.keys()) - def partitions_for_topic(self, topic): - if topic not in self._topics: + if topic not in self._partitions: return None - return list(self._topics[topic].keys()) + return set(self._partitions[topic].keys()) - def broker_for_partition(self, topic, partition): - if topic not in self._topics or partition not in self._topics[topic]: + def leader_for_partition(self, partition): + if partition.topic not in self._partitions: return None - broker_id = self._topics[topic][partition] - return self.broker_by_id(broker_id) + return self._partitions[partition.topic].get(partition.partition) - def refresh_metadata(self): - broker = self.random_broker() - if not broker.send(MetadataRequest([])): - return None - metadata = broker.recv() - if not metadata: - return None - self._update_metadata(metadata) - return metadata - - def _update_metadata(self, metadata): - self._brokers.update({ - node_id: BrokerConnection(host, port) - for node_id, host, port in metadata.brokers - if node_id not in self._brokers - }) - - self._topics = { - topic: { - partition: leader - for _, partition, leader, _, _ in partitions - } - for _, topic, partitions in metadata.topics - } - - def _bootstrap(self, hosts, timeout=2): - for host, port in hosts: - conn = BrokerConnection(host, port) - if not conn.connect(): - continue - self._brokers['bootstrap'] = conn - if self.refresh_metadata(): - break + def coordinator_for_group(self, group): + return self._groups.get(group) + + def ttl(self): + """Milliseconds until metadata should be refreshed""" + now = time.time() * 1000 + if self._need_update: + ttl = 0 else: - raise ValueError("Could not bootstrap kafka cluster from %s" % hosts) + ttl = self._last_successful_refresh_ms + self._metadata_max_age_ms - now + retry = self._last_refresh_ms + self._retry_backoff_ms - now + return max(ttl, retry, 0) + + def request_update(self): + """ + Flags metadata for update, return Future() + + Actual update must be handled separately. This method will only + change the reported ttl() + """ + self._need_update = True + if not self._future or self._future.is_done: + self._future = Future() + return self._future - if len(self._brokers) > 1: - self._brokers.pop('bootstrap') - conn.close() + def topics(self): + return set(self._partitions.keys()) + + def failed_update(self, exception): + if self._future: + self._future.failure(exception) + self._future = None + self._last_refresh_ms = time.time() * 1000 + + def update_metadata(self, metadata): + # In the common case where we ask for a single topic and get back an + # error, we should fail the future + if len(metadata.topics) == 1 and metadata.topics[0][0] != 0: + error_code, topic, _ = metadata.topics[0] + error = Errors.for_code(error_code)(topic) + return self.failed_update(error) + + if not metadata.brokers: + log.warning("No broker metadata found in MetadataResponse") + + for node_id, host, port in metadata.brokers: + self._brokers.update({ + node_id: BrokerMetadata(node_id, host, port) + }) + + # Drop any UnknownTopic, InvalidTopic, and TopicAuthorizationFailed + # but retain LeaderNotAvailable because it means topic is initializing + self._partitions = {} + + for error_code, topic, partitions in metadata.topics: + error_type = Errors.for_code(error_code) + if error_type is Errors.NoError: + self._partitions[topic] = {} + for _, partition, leader, _, _ in partitions: + self._partitions[topic][partition] = leader + elif error_type is Errors.LeaderNotAvailableError: + log.error("Topic %s is not available during auto-create" + " initialization", topic) + elif error_type is Errors.UnknownTopicOrPartitionError: + log.error("Topic %s not found in cluster metadata", topic) + elif error_type is Errors.TopicAuthorizationFailedError: + log.error("Topic %s is not authorized for this client", topic) + elif error_type is Errors.InvalidTopicError: + log.error("'%s' is not a valid topic name", topic) + else: + log.error("Error fetching metadata for topic %s: %s", + topic, error_type) + + if self._future: + self._future.success(self) + self._future = None + self._need_update = False + self._version += 1 + now = time.time() * 1000 + self._last_refresh_ms = now + self._last_successful_refresh_ms = now + log.debug("Updated cluster metadata version %d to %s", + self._version, self) + + for listener in self._listeners: + listener(self) + + def add_listener(self, listener): + """Add a callback function to be called on each metadata update""" + self._listeners.add(listener) + + def remove_listener(self, listener): + """Remove a previously added listener callback""" + self._listeners.remove(listener) def __str__(self): return 'Cluster(brokers: %d, topics: %d, groups: %d)' % \ - (len(self._brokers), len(self._topics), len(self._groups)) + (len(self._brokers), len(self._partitions), len(self._groups)) diff --git a/kafka/common.py b/kafka/common.py index 173fc82..9615981 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -425,6 +425,10 @@ def _iter_broker_errors(): kafka_errors = dict([(x.errno, x) for x in _iter_broker_errors()]) +def for_code(error_code): + return kafka_errors.get(error_code, UnknownError) + + def check_error(response): if isinstance(response, Exception): raise response -- cgit v1.2.1 From 7c76138640a9ab1390211abc0fb0d5d604c15621 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 13:08:23 -0800 Subject: Add ClusterMetadata.add_group_coordinator() --- kafka/cluster.py | 39 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/kafka/cluster.py b/kafka/cluster.py index 2e9e117..5b5fd8e 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -141,6 +141,45 @@ class ClusterMetadata(object): """Remove a previously added listener callback""" self._listeners.remove(listener) + def add_group_coordinator(self, group, response): + """Update with metadata for a group coordinator + + group: name of group from GroupCoordinatorRequest + response: GroupCoordinatorResponse + + returns True if metadata is updated, False on error + """ + log.debug("Updating coordinator for %s: %s", group, response) + error_type = Errors.for_code(response.error_code) + if error_type is not Errors.NoError: + log.error("GroupCoordinatorResponse error: %s", error_type) + self._groups[group] = -1 + return False + + node_id = response.coordinator_id + coordinator = BrokerMetadata( + response.coordinator_id, + response.host, + response.port) + + # Assume that group coordinators are just brokers + # (this is true now, but could diverge in future) + if node_id not in self._brokers: + self._brokers[node_id] = coordinator + + # If this happens, either brokers have moved without + # changing IDs, or our assumption above is wrong + elif coordinator != self._brokers[node_id]: + log.error("GroupCoordinator metadata conflicts with existing" + " broker metadata. Coordinator: %s, Broker: %s", + coordinator, self._brokers[node_id]) + self._groups[group] = node_id + return False + + log.info("Group coordinator for %s is %s", group, coordinator) + self._groups[group] = node_id + return True + def __str__(self): return 'Cluster(brokers: %d, topics: %d, groups: %d)' % \ (len(self._brokers), len(self._partitions), len(self._groups)) -- cgit v1.2.1 From 76222f24b21ff10b57b9a355eb0378e68e417169 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 20 Dec 2015 10:20:24 -0800 Subject: Update BrokerConnection for use with async client - use descriptive names for ConnectionStates enum values - Change default send_buffer_bytes config to 131072 - add can_send_more() and max_in_flight_requests_per_connection config - add blacked_out() and reconnect_backoff_ms config - last_attempt and last_failure are now public attributes - raise TooManyInFlightRequests in conn.send() if cant send more --- kafka/common.py | 4 ++ kafka/conn.py | 113 +++++++++++++++++++++++++++++++++++++------------------- 2 files changed, 78 insertions(+), 39 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 9615981..eb612d7 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -109,6 +109,10 @@ class CorrelationIdError(KafkaError): pass +class TooManyInFlightRequests(KafkaError): + pass + + class BrokerResponseError(KafkaError): errno = None message = None diff --git a/kafka/conn.py b/kafka/conn.py index c2b8fb0..a05ce8e 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -26,9 +26,9 @@ DEFAULT_KAFKA_PORT = 9092 class ConnectionStates(object): - DISCONNECTED = 1 - CONNECTING = 2 - CONNECTED = 3 + DISCONNECTED = '' + CONNECTING = '' + CONNECTED = '' InFlightRequest = collections.namedtuple('InFlightRequest', @@ -37,10 +37,12 @@ InFlightRequest = collections.namedtuple('InFlightRequest', class BrokerConnection(object): _receive_buffer_bytes = 32768 - _send_buffer_bytes = 32768 + _send_buffer_bytes = 131072 _client_id = 'kafka-python-0.10.0' _correlation_id = 0 _request_timeout_ms = 40000 + _max_in_flight_requests_per_connection = 5 + _reconnect_backoff_ms = 50 def __init__(self, host, port, **kwargs): self.host = host @@ -48,7 +50,9 @@ class BrokerConnection(object): self.in_flight_requests = collections.deque() for config in ('receive_buffer_bytes', 'send_buffer_bytes', - 'client_id', 'correlation_id', 'request_timeout_ms'): + 'client_id', 'correlation_id', 'request_timeout_ms', + 'max_in_flight_requests_per_connection', + 'reconnect_backoff_ms'): if config in kwargs: setattr(self, '_' + config, kwargs.pop(config)) @@ -57,8 +61,9 @@ class BrokerConnection(object): self._rbuffer = io.BytesIO() self._receiving = False self._next_payload_bytes = 0 - self._last_connection_attempt = None - self._last_connection_failure = None + self.last_attempt = 0 + self.last_failure = 0 + self._processing = False def connect(self): """Attempt to connect and return ConnectionState""" @@ -69,34 +74,47 @@ class BrokerConnection(object): self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_SNDBUF, self._send_buffer_bytes) self._sock.setblocking(False) ret = self._sock.connect_ex((self.host, self.port)) - self._last_connection_attempt = time.time() + self.last_attempt = time.time() if not ret or ret is errno.EISCONN: self.state = ConnectionStates.CONNECTED elif ret in (errno.EINPROGRESS, errno.EALREADY): self.state = ConnectionStates.CONNECTING else: - log.error('Connect attempt returned error %s. Disconnecting.', ret) + log.error('Connect attempt to %s returned error %s.' + ' Disconnecting.', self, ret) self.close() - self._last_connection_failure = time.time() + self.last_failure = time.time() if self.state is ConnectionStates.CONNECTING: # in non-blocking mode, use repeated calls to socket.connect_ex # to check connection status - if time.time() > (self._request_timeout_ms / 1000.0) + self._last_connection_attempt: - log.error('Connection attempt timed out') + if time.time() > (self._request_timeout_ms / 1000.0) + self.last_attempt: + log.error('Connection attempt to %s timed out', self) self.close() # error=TimeoutError ? - self._last_connection_failure = time.time() + self.last_failure = time.time() ret = self._sock.connect_ex((self.host, self.port)) if not ret or ret is errno.EISCONN: self.state = ConnectionStates.CONNECTED elif ret is not errno.EALREADY: - log.error('Connect attempt returned error %s. Disconnecting.', ret) + log.error('Connect attempt to %s returned error %s.' + ' Disconnecting.', self, ret) self.close() - self._last_connection_failure = time.time() + self.last_failure = time.time() return self.state + def blacked_out(self): + """ + Return true if we are disconnected from the given node and can't + re-establish a connection yet + """ + if self.state is ConnectionStates.DISCONNECTED: + now = time.time() + if now - self.last_attempt < self._reconnect_backoff_ms / 1000.0: + return True + return False + def connected(self): return self.state is ConnectionStates.CONNECTED @@ -105,17 +123,15 @@ class BrokerConnection(object): self._sock.close() self._sock = None self.state = ConnectionStates.DISCONNECTED - + self._receiving = False + self._next_payload_bytes = 0 + self._rbuffer.seek(0) + self._rbuffer.truncate() if error is None: error = Errors.DisconnectError() while self.in_flight_requests: ifr = self.in_flight_requests.popleft() ifr.future.failure(error) - self.in_flight_requests.clear() - self._receiving = False - self._next_payload_bytes = 0 - self._rbuffer.seek(0) - self._rbuffer.truncate() def send(self, request, expect_response=True): """send request, return Future() @@ -125,6 +141,8 @@ class BrokerConnection(object): future = Future() if not self.connected(): return future.failure(Errors.DisconnectError()) + if not self.can_send_more(): + return future.failure(Errors.TooManyInFlightRequests()) self._correlation_id += 1 header = RequestHeader(request, correlation_id=self._correlation_id, @@ -142,10 +160,10 @@ class BrokerConnection(object): assert sent_bytes == len(message) self._sock.setblocking(False) except (AssertionError, socket.error) as e: - log.debug("Error in BrokerConnection.send(): %s", request) + log.exception("Error sending %s to %s", request, self) self.close(error=e) return future.failure(e) - log.debug('Request %d: %s', self._correlation_id, request) + log.debug('%s Request %d: %s', self, self._correlation_id, request) if expect_response: ifr = InFlightRequest(request=request, @@ -159,24 +177,35 @@ class BrokerConnection(object): return future + def can_send_more(self): + return len(self.in_flight_requests) < self._max_in_flight_requests_per_connection + def recv(self, timeout=0): """Non-blocking network receive Return response if available """ + if self._processing: + raise Errors.IllegalStateError('Recursive connection processing' + ' not supported') if not self.connected(): - log.warning('Cannot recv: socket not connected') + log.warning('%s cannot recv: socket not connected', self) # If requests are pending, we should close the socket and # fail all the pending request futures if self.in_flight_requests: self.close() return None - if not self.in_flight_requests: - log.warning('No in-flight-requests to recv') + elif not self.in_flight_requests: + log.warning('%s: No in-flight-requests to recv', self) return None - self._fail_timed_out_requests() + elif self._requests_timed_out(): + log.warning('%s timed out after %s ms. Closing connection.', + self, self._request_timeout_ms) + self.close(error=Errors.RequestTimedOutError( + 'Request timed out after %s ms' % self._request_timeout_ms)) + return None readable, _, _ = select([self._sock], [], [], timeout) if not readable: @@ -193,7 +222,8 @@ class BrokerConnection(object): # This shouldn't happen after selecting above # but just in case return None - log.exception("Error receiving 4-byte payload header - closing socket") + log.exception('%s: Error receiving 4-byte payload header -' + ' closing socket', self) self.close(error=e) return None @@ -216,7 +246,7 @@ class BrokerConnection(object): # header, but nothing to read in the body yet if e.errno == errno.EWOULDBLOCK: return None - log.exception() + log.exception('%s: Error in recv', self) self.close(error=e) return None @@ -236,6 +266,11 @@ class BrokerConnection(object): return response def _process_response(self, read_buffer): + if self._processing: + raise Errors.IllegalStateError('Recursive connection processing' + ' not supported') + else: + self._processing = True ifr = self.in_flight_requests.popleft() # verify send/recv correlation ids match @@ -246,23 +281,23 @@ class BrokerConnection(object): % (ifr.correlation_id, recv_correlation_id)) ifr.future.fail(error) self.close() + self._processing = False return None # decode response response = ifr.response_type.decode(read_buffer) + log.debug('%s Response %d: %s', self, ifr.correlation_id, response) ifr.future.success(response) - log.debug('Response %d: %s', ifr.correlation_id, response) + self._processing = False return response - def _fail_timed_out_requests(self): - now = time.time() - while self.in_flight_requests: - next_timeout = self.in_flight_requests[0].timestamp + (self._request_timeout_ms / 1000.0) - if now < next_timeout: - break - timed_out = self.in_flight_requests.popleft() - error = Errors.RequestTimedOutError('Request timed out after %s ms' % self._request_timeout_ms) - timed_out.future.failure(error) + def _requests_timed_out(self): + if self.in_flight_requests: + oldest_at = self.in_flight_requests[0].timestamp + timeout = self._request_timeout_ms / 1000.0 + if time.time() >= oldest_at + timeout: + return True + return False def __repr__(self): return "" % (self.host, self.port) -- cgit v1.2.1 From fd1801907f85ac7686b9452c08ae908c3a88cd51 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 20 Dec 2015 10:27:52 -0800 Subject: New async KafkaClient, based on java NetworkClient and ConsumerClient - Support async client bootstrap retries after exponential backoff - Attempt to finish_connect if still connecting in KafkaClient.ready(node_id) - Run full async loop while waiting for futures in KafkaClient.poll() - Add more fallbacks to KafkaClient.least_loaded_node; worst case, retry boostrap --- kafka/client_async.py | 502 ++++++++++++++++++++++++++++++++++++++++++++++++++ kafka/common.py | 8 + 2 files changed, 510 insertions(+) create mode 100644 kafka/client_async.py diff --git a/kafka/client_async.py b/kafka/client_async.py new file mode 100644 index 0000000..5c11fc5 --- /dev/null +++ b/kafka/client_async.py @@ -0,0 +1,502 @@ +import heapq +import itertools +import logging +import random +import select +import sys +import time + +import six + +import kafka.common as Errors # TODO: make Errors a separate class + +from .cluster import ClusterMetadata +from .conn import BrokerConnection, ConnectionStates, collect_hosts +from .future import Future +from .protocol.metadata import MetadataRequest +from .protocol.produce import ProduceRequest + +log = logging.getLogger(__name__) + + +class KafkaClient(object): + """ + A network client for asynchronous request/response network i/o. + This is an internal class used to implement the + user-facing producer and consumer clients. + + This class is not thread-safe! + """ + _bootstrap_servers = 'localhost' + _client_id = 'kafka-python-0.10.0' + _reconnect_backoff_ms = 50 + _retry_backoff_ms = 100 + _send_buffer_bytes = 131072 + _receive_buffer_bytes = 32768 + _request_timeout_ms = 40000 + _max_in_flight_requests_per_connection=5 + + def __init__(self, **kwargs): + for config in ( + 'client_id', 'max_in_flight_requests_per_connection', + 'reconnect_backoff_ms', 'retry_backoff_ms', + 'send_buffer_bytes', 'receive_buffer_bytes', + 'request_timeout_ms', 'bootstrap_servers' + ): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) + + self.cluster = ClusterMetadata(**kwargs) + self._topics = set() # empty set will fetch all topic metadata + self._metadata_refresh_in_progress = False + self._conns = {} + self._connecting = set() + self._delayed_tasks = DelayedTaskQueue() + self._last_bootstrap = 0 + self._bootstrap_fails = 0 + self._bootstrap(collect_hosts(self._bootstrap_servers)) + + def _bootstrap(self, hosts): + # Exponential backoff if bootstrap fails + backoff_ms = self._reconnect_backoff_ms * 2 ** self._bootstrap_fails + next_at = self._last_bootstrap + backoff_ms / 1000.0 + now = time.time() + if next_at > now: + log.debug("Sleeping %0.4f before bootstrapping again", next_at - now) + time.sleep(next_at - now) + self._last_bootstrap = time.time() + + metadata_request = MetadataRequest([]) + for host, port in hosts: + log.debug("Attempting to bootstrap via node at %s:%s", host, port) + bootstrap = BrokerConnection( + host, port, + client_id=self._client_id, + receive_buffer_bytes=self._receive_buffer_bytes, + send_buffer_bytes=self._send_buffer_bytes, + request_timeout_ms=self._request_timeout_ms, + max_in_flight_requests_per_connection=self._max_in_flight_requests_per_connection, + reconnect_backoff_ms=self._reconnect_backoff_ms + ) + bootstrap.connect() + while bootstrap.state is ConnectionStates.CONNECTING: + bootstrap.connect() + if bootstrap.state is not ConnectionStates.CONNECTED: + bootstrap.close() + continue + future = bootstrap.send(metadata_request) + while not future.is_done: + bootstrap.recv() + if future.failed(): + bootstrap.close() + continue + self.cluster.update_metadata(future.value) + + # A cluster with no topics can return no broker metadata + # in that case, we should keep the bootstrap connection + if not len(self.cluster.brokers()): + self._conns['bootstrap'] = bootstrap + self._bootstrap_fails = 0 + break + # No bootstrap found... + else: + log.error('Unable to bootstrap from %s', hosts) + # Max exponential backoff is 2^12, x4000 (50ms -> 200s) + self._bootstrap_fails = min(self._bootstrap_fails + 1, 12) + + def _can_connect(self, node_id): + if node_id not in self._conns: + if self.cluster.broker_metadata(node_id): + return True + return False + conn = self._conns[node_id] + return conn.state is ConnectionStates.DISCONNECTED and not conn.blacked_out() + + def _initiate_connect(self, node_id): + """Initiate a connection to the given node""" + broker = self.cluster.broker_metadata(node_id) + if not broker: + raise Errors.IllegalArgumentError('Broker %s not found in current cluster metadata', node_id) + + if node_id not in self._conns: + log.debug("Initiating connection to node %s at %s:%s", + node_id, broker.host, broker.port) + self._conns[node_id] = BrokerConnection( + broker.host, broker.port, + client_id=self._client_id, + receive_buffer_bytes=self._receive_buffer_bytes, + send_buffer_bytes=self._send_buffer_bytes, + request_timeout_ms=self._request_timeout_ms, + max_in_flight_requests_per_connection=self._max_in_flight_requests_per_connection, + reconnect_backoff_ms=self._reconnect_backoff_ms + ) + return self._finish_connect(node_id) + + def _finish_connect(self, node_id): + if node_id not in self._conns: + raise Errors.IllegalArgumentError('Node %s not found in connections', node_id) + state = self._conns[node_id].connect() + if state is ConnectionStates.CONNECTING: + self._connecting.add(node_id) + elif node_id in self._connecting: + log.debug("Node %s connection state is %s", node_id, state) + self._connecting.remove(node_id) + return state + + def ready(self, node_id): + """ + Begin connecting to the given node, return true if we are already + connected and ready to send to that node. + + @param node_id The id of the node to check + @return True if we are ready to send to the given node + """ + if self.is_ready(node_id): + return True + + if self._can_connect(node_id): + # if we are interested in sending to a node + # and we don't have a connection to it, initiate one + self._initiate_connect(node_id) + + if node_id in self._connecting: + self._finish_connect(node_id) + + return self.is_ready(node_id) + + def close(self, node_id=None): + """Closes the connection to a particular node (if there is one). + + @param node_id The id of the node + """ + if node_id is None: + for conn in self._conns.values(): + conn.close() + elif node_id in self._conns: + self._conns[node_id].close() + else: + log.warning("Node %s not found in current connection list; skipping", node_id) + return + + def connection_delay(self, node_id): + """ + Returns the number of milliseconds to wait, based on the connection + state, before attempting to send data. When disconnected, this respects + the reconnect backoff time. When connecting or connected, this handles + slow/stalled connections. + + @param node_id The id of the node to check + @return The number of milliseconds to wait. + """ + if node_id not in self._conns: + return 0 + + conn = self._conns[node_id] + time_waited_ms = time.time() - (conn.last_attempt or 0) + if conn.state is ConnectionStates.DISCONNECTED: + return max(self._reconnect_backoff_ms - time_waited_ms, 0) + else: + return sys.maxint + + def connection_failed(self, node_id): + """ + Check if the connection of the node has failed, based on the connection + state. Such connection failures are usually transient and can be resumed + in the next ready(node) call, but there are cases where transient + failures need to be caught and re-acted upon. + + @param node_id the id of the node to check + @return true iff the connection has failed and the node is disconnected + """ + if node_id not in self._conns: + return False + return self._conns[node_id].state is ConnectionStates.DISCONNECTED + + def is_ready(self, node_id): + """ + Check if the node with the given id is ready to send more requests. + + @param node_id The id of the node + @return true if the node is ready + """ + # if we need to update our metadata now declare all requests unready to + # make metadata requests first priority + if not self._metadata_refresh_in_progress and not self.cluster.ttl() == 0: + if self._can_send_request(node_id): + return True + return False + + def _can_send_request(self, node_id): + if node_id not in self._conns: + return False + conn = self._conns[node_id] + return conn.connected() and conn.can_send_more() + + def send(self, node_id, request): + """ + Send the given request. Requests can only be sent out to ready nodes. + + @param node destination node + @param request The request + @param now The current timestamp + """ + if not self._can_send_request(node_id): + raise Errors.IllegalStateError("Attempt to send a request to node %s which is not ready." % node_id) + + # Every request gets a response, except one special case: + expect_response = True + if isinstance(request, ProduceRequest) and request.required_acks == 0: + expect_response = False + + return self._conns[node_id].send(request, expect_response=expect_response) + + def poll(self, timeout_ms=None, future=None): + """Do actual reads and writes to sockets. + + @param timeout_ms The maximum amount of time to wait (in ms) for + responses if there are none available immediately. + Must be non-negative. The actual timeout will be the + minimum of timeout, request timeout and metadata + timeout. If unspecified, default to request_timeout_ms + @param future Optionally block until the provided future completes. + @return The list of responses received. + """ + if timeout_ms is None: + timeout_ms = self._request_timeout_ms + + responses = [] + + # Loop for futures, break after first loop if None + while True: + + # Attempt to complete pending connections + for node_id in list(self._connecting): + self._finish_connect(node_id) + + # Send a metadata request if needed + metadata_timeout = self._maybe_refresh_metadata() + + # Send scheduled tasks + for task in self._delayed_tasks.pop_ready(): + try: + task() + except Exception as e: + log.error("Task %s failed: %s", task, e) + + timeout = min(timeout_ms, metadata_timeout, self._request_timeout_ms) + timeout /= 1000.0 + + responses.extend(self._poll(timeout)) + if not future or future.is_done: + break + + return responses + + def _poll(self, timeout): + # select on reads across all connected sockets, blocking up to timeout + sockets = [conn._sock for conn in six.itervalues(self._conns) + if (conn.state is ConnectionStates.CONNECTED and + conn.in_flight_requests)] + if sockets: + select.select(sockets, [], [], timeout) + + responses = [] + # list, not iterator, because inline callbacks may add to self._conns + for conn in list(self._conns.values()): + if conn.state is ConnectionStates.CONNECTING: + conn.connect() + + if conn.in_flight_requests: + response = conn.recv() # This will run callbacks / errbacks + if response: + responses.append(response) + return responses + + def in_flight_request_count(self, node_id=None): + """Get the number of in-flight requests""" + if node_id is not None: + if node_id not in self._conns: + return 0 + return len(self._conns[node_id].in_flight_requests) + else: + return sum([len(conn.in_flight_requests) for conn in self._conns.values()]) + + def least_loaded_node(self): + """ + Choose the node with the fewest outstanding requests which is at least + eligible for connection. This method will prefer a node with an + existing connection, but will potentially choose a node for which we + don't yet have a connection if all existing connections are in use. + This method will never choose a node for which there is no existing + connection and from which we have disconnected within the reconnect + backoff period. + + @return The node_id with the fewest in-flight requests. + """ + nodes = list(self._conns.keys()) + random.shuffle(nodes) + inflight = sys.maxint + found = None + for node_id in nodes: + conn = self._conns[node_id] + curr_inflight = len(conn.in_flight_requests) + if curr_inflight == 0 and conn.connected(): + # if we find an established connection with no in-flight requests we can stop right away + return node_id + elif not conn.blacked_out() and curr_inflight < inflight: + # otherwise if this is the best we have found so far, record that + inflight = curr_inflight + found = node_id + + if found is not None: + return found + + # if we found no connected node, return a disconnected one + log.debug("No connected nodes found. Trying disconnected nodes.") + for node_id in nodes: + if not self._conns[node_id].is_blacked_out(): + return node_id + + # if still no luck, look for a node not in self._conns yet + log.debug("No luck. Trying all broker metadata") + for broker in self.cluster.brokers(): + if broker.nodeId not in self._conns: + return broker.nodeId + + # Last option: try to bootstrap again + log.error('No nodes found in metadata -- retrying bootstrap') + self._bootstrap(collect_hosts(self._bootstrap_servers)) + return None + + def set_topics(self, topics): + """ + Set specific topics to track for metadata + + Returns a future that will complete after metadata request/response + """ + if set(topics).difference(self._topics): + future = self.cluster.request_update() + else: + future = Future().success(set(topics)) + self._topics = set(topics) + return future + + # request metadata update on disconnect and timedout + def _maybe_refresh_metadata(self): + """Send a metadata request if needed""" + ttl = self.cluster.ttl() + if ttl > 0: + return ttl + + if self._metadata_refresh_in_progress: + return sys.maxint + + node_id = self.least_loaded_node() + + if self._can_send_request(node_id): + request = MetadataRequest(list(self._topics)) + log.debug("Sending metadata request %s to node %s", request, node_id) + future = self.send(node_id, request) + future.add_callback(self.cluster.update_metadata) + future.add_errback(self.cluster.failed_update) + + self._metadata_refresh_in_progress = True + def refresh_done(val_or_error): + self._metadata_refresh_in_progress = False + future.add_callback(refresh_done) + future.add_errback(refresh_done) + + elif self._can_connect(node_id): + log.debug("Initializing connection to node %s for metadata request", node_id) + self._initiate_connect(node_id) + + return 0 + + def schedule(self, task, at): + """ + Schedule a new task to be executed at the given time. + + This is "best-effort" scheduling and should only be used for coarse + synchronization. A task cannot be scheduled for multiple times + simultaneously; any previously scheduled instance of the same task + will be cancelled. + + @param task The task to be scheduled -- function or implement __call__ + @param at Epoch seconds when it should run (see time.time()) + @returns Future + """ + return self._delayed_tasks.add(task, at) + + def unschedule(self, task): + """ + Unschedule a task. This will remove all instances of the task from the task queue. + This is a no-op if the task is not scheduled. + + @param task The task to be unscheduled. + """ + self._delayed_tasks.remove(task) + + +class DelayedTaskQueue(object): + # see https://docs.python.org/2/library/heapq.html + def __init__(self): + self._tasks = [] # list of entries arranged in a heap + self._task_map = {} # mapping of tasks to entries + self._counter = itertools.count() # unique sequence count + + def add(self, task, at): + """Add a task to run at a later time + + task: anything + at: seconds from epoch to schedule task (see time.time()) + """ + if task in self._task_map: + self.remove(task) + count = next(self._counter) + future = Future() + entry = [at, count, (task, future)] + self._task_map[task] = entry + heapq.heappush(self._tasks, entry) + return future + + def remove(self, task): + """Remove a previously scheduled task + + Raises KeyError if task is not found + """ + entry = self._task_map.pop(task) + task, future = entry[-1] + future.failure(Errors.Cancelled) + entry[-1] = 'REMOVED' + + def _drop_removed(self): + while self._tasks and self._tasks[0][-1] is 'REMOVED': + at, count, task = heapq.heappop(self._tasks) + + def _pop_next(self): + self._drop_removed() + if not self._tasks: + raise KeyError('pop from an empty DelayedTaskQueue') + _, _, maybe_task = heapq.heappop(self._tasks) + if maybe_task is 'REMOVED': + raise ValueError('popped a removed tasks from queue - bug') + else: + task, future = maybe_task + del self._task_map[task] + return task + + def next_at(self): + """Number of seconds until next task is ready""" + self._drop_removed() + if not self._tasks: + return sys.maxint + else: + return max(self._tasks[0][0] - time.time(), 0) + + def pop_ready(self): + """Pop and return a list of all ready (task, future) tuples""" + self._drop_removed() + ready_tasks = [] + while self._tasks and self._tasks[0][0] < time.time(): + ready_tasks.append(self._pop_next()) + return ready_tasks diff --git a/kafka/common.py b/kafka/common.py index eb612d7..f79150b 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -97,6 +97,10 @@ class IllegalStateError(KafkaError): pass +class IllegalArgumentError(KafkaError): + pass + + class RetriableError(KafkaError): pass @@ -109,6 +113,10 @@ class CorrelationIdError(KafkaError): pass +class Cancelled(KafkaError): + pass + + class TooManyInFlightRequests(KafkaError): pass -- cgit v1.2.1 From c9e6f17e42e410adfe583d987c516149bdcdcdae Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 01:37:25 -0800 Subject: Rename TopicAndPartition -> TopicPartition --- kafka/client.py | 8 ++++---- kafka/common.py | 2 +- kafka/producer/base.py | 4 ++-- test/test_client.py | 20 ++++++++++---------- test/test_failover_integration.py | 4 ++-- test/test_producer.py | 10 +++++----- test/test_protocol.py | 2 +- test/test_util.py | 2 +- 8 files changed, 26 insertions(+), 26 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 7f9969e..2f070cd 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -8,7 +8,7 @@ import time import six import kafka.common -from kafka.common import (TopicAndPartition, BrokerMetadata, UnknownError, +from kafka.common import (TopicPartition, BrokerMetadata, UnknownError, ConnectionError, FailedPayloadsError, KafkaTimeoutError, KafkaUnavailableError, LeaderNotAvailableError, UnknownTopicOrPartitionError, @@ -41,7 +41,7 @@ class KafkaClient(object): self._conns = {} self.brokers = {} # broker_id -> BrokerMetadata - self.topics_to_brokers = {} # TopicAndPartition -> BrokerMetadata + self.topics_to_brokers = {} # TopicPartition -> BrokerMetadata self.topic_partitions = {} # topic -> partition -> PartitionMetadata self.load_metadata_for_topics() # bootstrap with all metadata @@ -77,7 +77,7 @@ class KafkaClient(object): no current leader """ - key = TopicAndPartition(topic, partition) + key = TopicPartition(topic, partition) # Use cached metadata if it is there if self.topics_to_brokers.get(key) is not None: @@ -511,7 +511,7 @@ class KafkaClient(object): self.topic_partitions[topic][partition] = leader # Populate topics_to_brokers dict - topic_part = TopicAndPartition(topic, partition) + topic_part = TopicPartition(topic, partition) # Check for partition errors if error: diff --git a/kafka/common.py b/kafka/common.py index f79150b..6a32372 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -72,7 +72,7 @@ OffsetAndMessage = namedtuple("OffsetAndMessage", Message = namedtuple("Message", ["magic", "attributes", "key", "value"]) -TopicAndPartition = namedtuple("TopicAndPartition", +TopicPartition = namedtuple("TopicPartition", ["topic", "partition"]) KafkaMessage = namedtuple("KafkaMessage", diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 595ac37..4972cd4 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -15,7 +15,7 @@ from threading import Thread, Event import six from kafka.common import ( - ProduceRequestPayload, ProduceResponsePayload, TopicAndPartition, RetryOptions, + ProduceRequestPayload, ProduceResponsePayload, TopicPartition, RetryOptions, kafka_errors, UnsupportedCodecError, FailedPayloadsError, RequestTimedOutError, AsyncProducerQueueFull, UnknownError, RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES @@ -386,7 +386,7 @@ class Producer(object): if self.async: for idx, m in enumerate(msg): try: - item = (TopicAndPartition(topic, partition), m, key) + item = (TopicPartition(topic, partition), m, key) if self.async_queue_put_timeout == 0: self.queue.put_nowait(item) else: diff --git a/test/test_client.py b/test/test_client.py index 00e888c..8c62eb9 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -9,7 +9,7 @@ from kafka import KafkaClient from kafka.common import ( ProduceRequestPayload, BrokerMetadata, - TopicAndPartition, KafkaUnavailableError, + TopicPartition, KafkaUnavailableError, LeaderNotAvailableError, UnknownTopicOrPartitionError, KafkaTimeoutError, ConnectionError ) @@ -145,12 +145,12 @@ class TestKafkaClient(unittest.TestCase): # client loads metadata at init client = KafkaClient(hosts=['broker_1:4567']) self.assertDictEqual({ - TopicAndPartition('topic_1', 0): brokers[1], - TopicAndPartition('topic_noleader', 0): None, - TopicAndPartition('topic_noleader', 1): None, - TopicAndPartition('topic_3', 0): brokers[0], - TopicAndPartition('topic_3', 1): brokers[1], - TopicAndPartition('topic_3', 2): brokers[0]}, + TopicPartition('topic_1', 0): brokers[1], + TopicPartition('topic_noleader', 0): None, + TopicPartition('topic_noleader', 1): None, + TopicPartition('topic_3', 0): brokers[0], + TopicPartition('topic_3', 1): brokers[1], + TopicPartition('topic_3', 2): brokers[0]}, client.topics_to_brokers) # if we ask for metadata explicitly, it should raise errors @@ -260,7 +260,7 @@ class TestKafkaClient(unittest.TestCase): self.assertEqual(brokers[0], leader) self.assertDictEqual({ - TopicAndPartition('topic_one_partition', 0): brokers[0]}, + TopicPartition('topic_one_partition', 0): brokers[0]}, client.topics_to_brokers) @patch('kafka.client.KafkaClient._get_conn') @@ -312,8 +312,8 @@ class TestKafkaClient(unittest.TestCase): client = KafkaClient(hosts=['broker_1:4567']) self.assertDictEqual( { - TopicAndPartition('topic_noleader', 0): None, - TopicAndPartition('topic_noleader', 1): None + TopicPartition('topic_noleader', 0): None, + TopicPartition('topic_noleader', 1): None }, client.topics_to_brokers) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 8c5efe2..04c9e2b 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -3,7 +3,7 @@ import os import time from kafka import KafkaClient, SimpleConsumer, KeyedProducer -from kafka.common import TopicAndPartition, FailedPayloadsError, ConnectionError +from kafka.common import TopicPartition, FailedPayloadsError, ConnectionError from kafka.producer.base import Producer from test.fixtures import ZookeeperFixture, KafkaFixture @@ -202,7 +202,7 @@ class TestFailover(KafkaIntegrationTestCase): break def _kill_leader(self, topic, partition): - leader = self.client.topics_to_brokers[TopicAndPartition(topic, partition)] + leader = self.client.topics_to_brokers[TopicPartition(topic, partition)] broker = self.brokers[leader.nodeId] broker.close() return broker diff --git a/test/test_producer.py b/test/test_producer.py index cbc1773..f62b97a 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -10,7 +10,7 @@ from . import unittest from kafka import KafkaClient, SimpleProducer, KeyedProducer from kafka.common import ( AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError, - ProduceResponsePayload, RetryOptions, TopicAndPartition + ProduceResponsePayload, RetryOptions, TopicPartition ) from kafka.producer.base import Producer, _send_upstream from kafka.protocol import CODEC_NONE @@ -156,7 +156,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # lets create a queue and add 10 messages for 1 partition for i in range(10): - self.queue.put((TopicAndPartition("test", 0), "msg %i", "key %i")) + self.queue.put((TopicPartition("test", 0), "msg %i", "key %i")) self._run_process() @@ -172,7 +172,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # lets create a queue and add 10 messages for 10 different partitions # to show how retries should work ideally for i in range(10): - self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + self.queue.put((TopicPartition("test", i), "msg %i", "key %i")) # Mock offsets counter for closure offsets = collections.defaultdict(lambda: collections.defaultdict(lambda: 0)) @@ -206,7 +206,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): # lets create a queue and add 10 messages for 10 different partitions # to show how retries should work ideally for i in range(10): - self.queue.put((TopicAndPartition("test", i), "msg %i" % i, "key %i" % i)) + self.queue.put((TopicPartition("test", i), "msg %i" % i, "key %i" % i)) def send_side_effect(reqs, *args, **kwargs): return [FailedPayloadsError(req) for req in reqs] @@ -226,7 +226,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def test_async_producer_not_leader(self): for i in range(10): - self.queue.put((TopicAndPartition("test", i), "msg %i", "key %i")) + self.queue.put((TopicPartition("test", i), "msg %i", "key %i")) # Mock offsets counter for closure offsets = collections.defaultdict(lambda: collections.defaultdict(lambda: 0)) diff --git a/test/test_protocol.py b/test/test_protocol.py index 6c79829..4c5f379 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -12,7 +12,7 @@ from kafka.common import ( OffsetResponsePayload, OffsetCommitResponsePayload, OffsetFetchResponsePayload, ProduceRequestPayload, FetchRequestPayload, Message, ChecksumError, ProduceResponsePayload, FetchResponsePayload, OffsetAndMessage, - BrokerMetadata, TopicMetadata, PartitionMetadata, TopicAndPartition, + BrokerMetadata, TopicMetadata, PartitionMetadata, KafkaUnavailableError, UnsupportedCodecError, ConsumerFetchSizeTooSmall, ProtocolError, ConsumerMetadataResponse ) diff --git a/test/test_util.py b/test/test_util.py index ea3783e..7f0432b 100644 --- a/test/test_util.py +++ b/test/test_util.py @@ -104,7 +104,7 @@ class UtilTest(unittest.TestCase): kafka.util.relative_unpack('>hh', '\x00', 0) def test_group_by_topic_and_partition(self): - t = kafka.common.TopicAndPartition + t = kafka.common.TopicPartition l = [ t("a", 1), -- cgit v1.2.1 From d2af5b37ce53fcde6e9f3ee1b587c1ee1de19d2c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 01:43:57 -0800 Subject: Use attributes on exception classes to signal retriable and invalid metadata errors --- kafka/common.py | 39 ++++++++++++++++++++++++++++----------- kafka/future.py | 7 +++++-- 2 files changed, 33 insertions(+), 13 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 6a32372..cd93ff6 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -90,7 +90,9 @@ RetryOptions = namedtuple("RetryOptions", class KafkaError(RuntimeError): - pass + retriable = False + # whether metadata should be refreshed on error + invalid_metadata = False class IllegalStateError(KafkaError): @@ -101,24 +103,30 @@ class IllegalArgumentError(KafkaError): pass -class RetriableError(KafkaError): - pass +class DisconnectError(KafkaError): + retriable = True + invalid_metadata = True -class DisconnectError(KafkaError): - pass +class NodeNotReadyError(KafkaError): + retriable = True class CorrelationIdError(KafkaError): - pass + retriable = True class Cancelled(KafkaError): - pass + retriable = True class TooManyInFlightRequests(KafkaError): - pass + retriable = True + + +class StaleMetadata(KafkaError): + retriable = True + invalid_metadata = True class BrokerResponseError(KafkaError): @@ -161,6 +169,7 @@ class UnknownTopicOrPartitionError(BrokerResponseError): message = 'UNKNOWN_TOPIC_OR_PARTITON' description = ('This request is for a topic or partition that does not' ' exist on this broker.') + invalid_metadata = True class InvalidFetchRequestError(BrokerResponseError): @@ -173,8 +182,10 @@ class LeaderNotAvailableError(BrokerResponseError): errno = 5 message = 'LEADER_NOT_AVAILABLE' description = ('This error is thrown if we are in the middle of a' - 'leadership election and there is currently no leader for' - 'this partition and hence it is unavailable for writes.') + ' leadership election and there is currently no leader for' + ' this partition and hence it is unavailable for writes.') + retriable = True + invalid_metadata = True class NotLeaderForPartitionError(BrokerResponseError): @@ -184,6 +195,8 @@ class NotLeaderForPartitionError(BrokerResponseError): ' messages to a replica that is not the leader for some' ' partition. It indicates that the clients metadata is out' ' of date.') + retriable = True + invalid_metadata = True class RequestTimedOutError(BrokerResponseError): @@ -191,6 +204,7 @@ class RequestTimedOutError(BrokerResponseError): message = 'REQUEST_TIMED_OUT' description = ('This error is thrown if the request exceeds the' ' user-specified time limit in the request.') + retriable = True class BrokerNotAvailableError(BrokerResponseError): @@ -212,7 +226,7 @@ class MessageSizeTooLargeError(BrokerResponseError): description = ('The server has a configurable maximum message size to avoid' ' unbounded memory allocation. This error is thrown if the' ' client attempt to produce a message larger than this' - 'maximum.') + ' maximum.') class StaleControllerEpochError(BrokerResponseError): @@ -242,6 +256,7 @@ class GroupLoadInProgressError(BrokerResponseError): ' change for that offsets topic partition), or in response' ' to group membership requests (such as heartbeats) when' ' group metadata is being loaded by the coordinator.') + retriable = True class GroupCoordinatorNotAvailableError(BrokerResponseError): @@ -251,6 +266,7 @@ class GroupCoordinatorNotAvailableError(BrokerResponseError): ' requests, offset commits, and most group management' ' requests if the offsets topic has not yet been created, or' ' if the group coordinator is not active.') + retriable = True class NotCoordinatorForGroupError(BrokerResponseError): @@ -259,6 +275,7 @@ class NotCoordinatorForGroupError(BrokerResponseError): description = ('The broker returns this error code if it receives an offset' ' fetch or commit request for a group that it is not a' ' coordinator for.') + retriable = True class InvalidTopicError(BrokerResponseError): diff --git a/kafka/future.py b/kafka/future.py index 24173bb..20c31cf 100644 --- a/kafka/future.py +++ b/kafka/future.py @@ -1,4 +1,4 @@ -from kafka.common import RetriableError, IllegalStateError +from kafka.common import IllegalStateError class Future(object): @@ -16,7 +16,10 @@ class Future(object): return self.is_done and self.exception def retriable(self): - return isinstance(self.exception, RetriableError) + try: + return self.exception.retriable + except AttributeError: + return False def success(self, value): if self.is_done: -- cgit v1.2.1 From a57ab55a5c593c099453f988dc12a1ee3552ecc3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 01:44:53 -0800 Subject: Add OffsetResetStrategy enum class to kafka.protocol.offset --- kafka/protocol/offset.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/kafka/protocol/offset.py b/kafka/protocol/offset.py index 776de39..606f1f1 100644 --- a/kafka/protocol/offset.py +++ b/kafka/protocol/offset.py @@ -1,6 +1,11 @@ from .struct import Struct from .types import Array, Int16, Int32, Int64, Schema, String +class OffsetResetStrategy(object): + LATEST = -1 + EARLIEST = -2 + NONE = 0 + class OffsetResponse(Struct): SCHEMA = Schema( -- cgit v1.2.1 From 97c7d8829b96f37ae8ee9d3980cc443d77845534 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 01:45:23 -0800 Subject: Add Message.validate_crc() method --- kafka/protocol/message.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index a67d7f5..f6cbb33 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -42,6 +42,13 @@ class Message(Struct): return cls(fields[4], key=fields[3], magic=fields[1], attributes=fields[2], crc=fields[0]) + def validate_crc(self): + raw_msg = self._encode_self(recalc_crc=False) + crc = crc32(raw_msg[4:]) + if crc == self.crc: + return True + return False + class PartialMessage(bytes): def __repr__(self): -- cgit v1.2.1 From 87f0181b2c6c51318a6f950e9478d14a7efe963f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 09:13:58 -0800 Subject: Add OffsetAndMetadata namedtuple --- kafka/common.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/common.py b/kafka/common.py index cd93ff6..33f24a4 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -78,6 +78,9 @@ TopicPartition = namedtuple("TopicPartition", KafkaMessage = namedtuple("KafkaMessage", ["topic", "partition", "offset", "key", "value"]) +OffsetAndMetadata = namedtuple("OffsetAndMetadata", + ["offset", "metadata"]) + # Define retry policy for async producer # Limit value: int >= 0, 0 means no retries RetryOptions = namedtuple("RetryOptions", -- cgit v1.2.1 From b643ba1a1ed7838625012e6f75a1ed9b35ffd022 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 09:14:57 -0800 Subject: New class SubscriptionState, based on upstream Java client --- kafka/consumer/subscription_state.py | 304 +++++++++++++++++++++++++++++++++++ 1 file changed, 304 insertions(+) create mode 100644 kafka/consumer/subscription_state.py diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py new file mode 100644 index 0000000..6ebd925 --- /dev/null +++ b/kafka/consumer/subscription_state.py @@ -0,0 +1,304 @@ +from __future__ import absolute_import + +import logging +import re + +import six + +from kafka.common import IllegalStateError, OffsetAndMetadata +from kafka.protocol.offset import OffsetResetStrategy + +log = logging.getLogger(__name__) + + +class SubscriptionState(object): + """ + A class for tracking the topics, partitions, and offsets for the consumer. + A partition is "assigned" either directly with assign_from_user() (manual + assignment) or with assign_from_subscribed() (automatic assignment from + subscription). + + Once assigned, the partition is not considered "fetchable" until its initial + position has been set with seek(). Fetchable partitions track a fetch + position which is used to set the offset of the next fetch, and a consumed + position which is the last offset that has been returned to the user. You + can suspend fetching from a partition through pause() without affecting the + fetched/consumed offsets. The partition will remain unfetchable until the + resume() is used. You can also query the pause state independently with + is_paused(). + + Note that pause state as well as fetch/consumed positions are not preserved + when partition assignment is changed whether directly by the user or + through a group rebalance. + + This class also maintains a cache of the latest commit position for each of + the assigned partitions. This is updated through committed() and can be used + to set the initial fetch position (e.g. Fetcher._reset_offset() ). + """ + _SUBSCRIPTION_EXCEPTION_MESSAGE = ("Subscription to topics, partitions and" + " pattern are mutually exclusive") + + def __init__(self, offset_reset_strategy='earliest'): + """Initialize a SubscriptionState instance + + offset_reset_strategy: 'earliest' or 'latest', otherwise + exception will be raised when fetching an offset + that is no longer available. + Defaults to earliest. + """ + try: + offset_reset_strategy = getattr(OffsetResetStrategy, + offset_reset_strategy.upper()) + except AttributeError: + log.warning('Unrecognized offset_reset_strategy, using NONE') + offset_reset_strategy = OffsetResetStrategy.NONE + self._default_offset_reset_strategy = offset_reset_strategy + + self.subscription = None # set() or None + self.subscribed_pattern = None # regex str or None + self._group_subscription = set() + self._user_assignment = set() + self.assignment = dict() + self.needs_partition_assignment = False + self.listener = None + + # initialize to true for the consumers to fetch offset upon starting up + self.needs_fetch_committed_offsets = True + + def subscribe(self, topics=(), pattern=None, listener=None): + """Subscribe to a list of topics, or a topic regex pattern + + Partitions will be assigned via a group coordinator + (incompatible with assign_from_user) + + Optionally include listener callback, which must be a + ConsumerRebalanceListener and will be called before and + after each rebalance operation. + """ + if self._user_assignment or (topics and pattern): + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + if not (topics or pattern): + raise IllegalStateError('Must provide topics or a pattern') + + if pattern: + log.info('Subscribing to pattern: /%s/', pattern) + self.subscription = set() + self.subscribed_pattern = re.compile(pattern) + else: + self.change_subscription(topics) + self.listener = listener + + def change_subscription(self, topics): + if self._user_assignment: + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + + if self.subscription == set(topics): + log.warning("subscription unchanged by change_subscription(%s)", + topics) + return + + log.info('Updating subscribed topics to: %s', topics) + self.subscription = set(topics) + self._group_subscription.update(topics) + self.needs_partition_assignment = True + + # Remove any assigned partitions which are no longer subscribed to + for tp in set(self.assignment.keys()): + if tp.topic not in self.subscription: + del self.assignment[tp] + + def group_subscribe(self, topics): + """Add topics to the current group subscription. + + This is used by the group leader to ensure that it receives metadata + updates for all topics that any member of the group is subscribed to. + + @param topics list of topics to add to the group subscription + """ + if self._user_assignment: + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + self._group_subscription.update(topics) + + def mark_for_reassignment(self): + self._group_subscription.intersection_update(self.subscription) + self.needs_partition_assignment = True + + def assign_from_user(self, partitions): + """ + Change the assignment to the specified partitions provided by the user, + note this is different from assign_from_subscribed() + whose input partitions are provided from the subscribed topics. + + @param partitions: list (or iterable) of TopicPartition() + """ + if self.subscription is not None: + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + + self._user_assignment.clear() + self._user_assignment.update(partitions) + + for partition in partitions: + if partition not in self.assignment: + self.add_assigned_partition(partition) + + for tp in set(self.assignment.keys()) - self._user_assignment: + del self.assignment[tp] + + self.needs_partition_assignment = False + + def assign_from_subscribed(self, assignments): + """ + Change the assignment to the specified partitions returned from the coordinator, + note this is different from {@link #assignFromUser(Collection)} which directly set the assignment from user inputs + """ + if self.subscription is None: + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + + for tp in assignments: + if tp.topic not in self.subscription: + raise ValueError("Assigned partition %s for non-subscribed topic." % tp) + self.assignment.clear() + for tp in assignments: + self.add_assigned_partition(tp) + self.needs_partition_assignment = False + + def unsubscribe(self): + self.subscription = None + self._user_assignment.clear() + self.assignment.clear() + self.needs_partition_assignment = True + self.subscribed_pattern = None + + def group_subscription(self): + """Get the topic subscription for the group. + + For the leader, this will include the union of all member subscriptions. + For followers, it is the member's subscription only. + + This is used when querying topic metadata to detect metadata changes + that would require rebalancing (the leader fetches metadata for all + topics in the group so that it can do partition assignment). + + @return set of topics + """ + return self._group_subscription + + def seek(self, partition, offset): + self.assignment[partition].seek(offset) + + def assigned_partitions(self): + return set(self.assignment.keys()) + + def fetchable_partitions(self): + fetchable = set() + for partition, state in six.iteritems(self.assignment): + if state.is_fetchable(): + fetchable.add(partition) + return fetchable + + def partitions_auto_assigned(self): + return self.subscription is not None + + def all_consumed_offsets(self): + """Returns consumed offsets as {TopicPartition: OffsetAndMetadata}""" + all_consumed = {} + for partition, state in six.iteritems(self.assignment): + if state.has_valid_position: + all_consumed[partition] = OffsetAndMetadata(state.consumed, '') + return all_consumed + + def need_offset_reset(self, partition, offset_reset_strategy=None): + if offset_reset_strategy is None: + offset_reset_strategy = self._default_offset_reset_strategy + self.assignment[partition].await_reset(offset_reset_strategy) + + def has_default_offset_reset_policy(self): + return self._default_offset_reset_strategy != OffsetResetStrategy.NONE + + def is_offset_reset_needed(self, partition): + return self.assignment[partition].awaiting_reset + + def has_all_fetch_positions(self): + for state in self.assignment.values(): + if not state.has_valid_position: + return False + return True + + def missing_fetch_positions(self): + missing = set() + for partition, state in six.iteritems(self.assignment): + if not state.has_valid_position: + missing.add(partition) + return missing + + def is_assigned(self, partition): + return partition in self.assignment + + def is_paused(self, partition): + return partition in self.assignment and self.assignment[partition].paused + + def is_fetchable(self, partition): + return partition in self.assignment and self.assignment[partition].is_fetchable() + + def pause(self, partition): + self.assignment[partition].pause() + + def resume(self, partition): + self.assignment[partition].resume() + + def add_assigned_partition(self, partition): + self.assignment[partition] = TopicPartitionState() + + +class TopicPartitionState(object): + def __init__(self): + self.committed = None # last committed position + self.has_valid_position = False # whether we have valid consumed and fetched positions + self.paused = False # whether this partition has been paused by the user + self.awaiting_reset = False # whether we are awaiting reset + self.reset_strategy = None # the reset strategy if awaitingReset is set + self._consumed = None # offset exposed to the user + self._fetched = None # current fetch position + + def _set_fetched(self, offset): + if not self.has_valid_position: + raise IllegalStateError("Cannot update fetch position without valid consumed/fetched positions") + self._fetched = offset + + def _get_fetched(self): + return self._fetched + + fetched = property(_get_fetched, _set_fetched, None, "current fetch position") + + def _set_consumed(self, offset): + if not self.has_valid_position: + raise IllegalStateError("Cannot update consumed position without valid consumed/fetched positions") + self._consumed = offset + + def _get_consumed(self): + return self._consumed + + consumed = property(_get_consumed, _set_consumed, None, "last consumed position") + + def await_reset(self, strategy): + self.awaiting_reset = True + self.reset_strategy = strategy + self._consumed = None + self._fetched = None + self.has_valid_position = False + + def seek(self, offset): + self._consumed = offset + self._fetched = offset + self.awaiting_reset = False + self.reset_strategy = None + self.has_valid_position = True + + def pause(self): + self.paused = True + + def resume(self): + self.paused = False + + def is_fetchable(self): + return not self.paused and self.has_valid_position -- cgit v1.2.1 From 87257bddf23850774794d4dc070a15ddddbd7830 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 13:24:37 -0800 Subject: Improvements to kafka.future.Future - log exceptions in callbacks/errors - dont raise - guarantee future.exception is an instance, not a class/type - *args, **kwargs in add_callback / errback (functools.partial) - add_both(f, *args, **kwargs) to add same f as callback and errback - chain(new_future) to trigger new_future on success / failure of this --- kafka/future.py | 45 +++++++++++++++++++++++++++++++++++++-------- 1 file changed, 37 insertions(+), 8 deletions(-) diff --git a/kafka/future.py b/kafka/future.py index 20c31cf..1f22cb7 100644 --- a/kafka/future.py +++ b/kafka/future.py @@ -1,4 +1,9 @@ -from kafka.common import IllegalStateError +import functools +import logging + +import kafka.common as Errors + +log = logging.getLogger(__name__) class Future(object): @@ -23,32 +28,56 @@ class Future(object): def success(self, value): if self.is_done: - raise IllegalStateError('Invalid attempt to complete a request future which is already complete') + raise Errors.IllegalStateError('Invalid attempt to complete a' + ' request future which is already' + ' complete') self.value = value self.is_done = True for f in self._callbacks: - f(value) + try: + f(value) + except Exception: + log.exception('Error processing callback') return self def failure(self, e): if self.is_done: - raise IllegalStateError('Invalid attempt to complete a request future which is already complete') - self.exception = e + raise Errors.IllegalStateError('Invalid attempt to complete a' + ' request future which is already' + ' complete') + self.exception = e if type(e) is not type else e() self.is_done = True for f in self._errbacks: - f(e) + try: + f(e) + except Exception: + log.exception('Error processing errback') return self - def add_callback(self, f): + def add_callback(self, f, *args, **kwargs): + if args or kwargs: + f = functools.partial(f, *args, **kwargs) if self.is_done and not self.exception: f(self.value) else: self._callbacks.append(f) return self - def add_errback(self, f): + def add_errback(self, f, *args, **kwargs): + if args or kwargs: + f = functools.partial(f, *args, **kwargs) if self.is_done and self.exception: f(self.exception) else: self._errbacks.append(f) return self + + def add_both(self, f, *args, **kwargs): + self.add_callback(f, *args, **kwargs) + self.add_errback(f, *args, **kwargs) + return self + + def chain(self, future): + self.add_callback(future.success) + self.add_errback(future.failure) + return self -- cgit v1.2.1 From d0782dfecf53e39f7ca6461f5d8ab4c857af17ca Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 21 Dec 2015 09:28:22 -0800 Subject: New class: kafka.consumer.fetcher.Fetcher - manages fetch requests via async client and subscription_state - based on upstream java client --- kafka/consumer/fetcher.py | 523 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 523 insertions(+) create mode 100644 kafka/consumer/fetcher.py diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py new file mode 100644 index 0000000..ea9c8b9 --- /dev/null +++ b/kafka/consumer/fetcher.py @@ -0,0 +1,523 @@ +from __future__ import absolute_import + +import collections +import logging + +import six + +import kafka.common as Errors +from kafka.common import TopicPartition +from kafka.future import Future +from kafka.protocol.fetch import FetchRequest +from kafka.protocol.message import PartialMessage +from kafka.protocol.offset import OffsetRequest, OffsetResetStrategy + +log = logging.getLogger(__name__) + + +ConsumerRecord = collections.namedtuple("ConsumerRecord", + ["topic", "partition", "offset", "key", "value"]) + + +class NoOffsetForPartitionError(Errors.KafkaError): + pass + + +class RecordTooLargeError(Errors.KafkaError): + pass + + +class Fetcher(object): + _key_deserializer = None + _value_deserializer = None + _fetch_min_bytes = 1024 + _fetch_max_wait_ms = 500 + _max_partition_fetch_bytes = 1048576 + _check_crcs = True + _retry_backoff_ms = 100 + + def __init__(self, client, subscriptions, **kwargs): + #metrics=None, + #metric_group_prefix='consumer', + + self._client = client + self._subscriptions = subscriptions + for config in ('key_deserializer', 'value_deserializer', + 'fetch_min_bytes', 'fetch_max_wait_ms', + 'max_partition_fetch_bytes', 'check_crcs', + 'retry_backoff_ms'): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) + + self._records = collections.deque() # (offset, topic_partition, messages) + self._unauthorized_topics = set() + self._offset_out_of_range_partitions = dict() # {topic_partition: offset} + self._record_too_large_partitions = dict() # {topic_partition: offset} + + #self.sensors = FetchManagerMetrics(metrics, metric_group_prefix) + + def init_fetches(self): + """Send FetchRequests asynchronously for all assigned partitions""" + futures = [] + for node_id, request in six.iteritems(self._create_fetch_requests()): + if self._client.ready(node_id): + log.debug("Sending FetchRequest to node %s", node_id) + future = self._client.send(node_id, request) + future.add_callback(self._handle_fetch_response, request) + future.add_errback(log.error, 'Fetch to node %s failed: %s', node_id) + futures.append(future) + return futures + + def update_fetch_positions(self, partitions): + """Update the fetch positions for the provided partitions. + + @param partitions: iterable of TopicPartitions + @raises NoOffsetForPartitionError If no offset is stored for a given + partition and no reset policy is available + """ + # reset the fetch position to the committed position + for tp in partitions: + if not self._subscriptions.is_assigned(tp): + log.warning("partition %s is not assigned - skipping offset" + " update", tp) + continue + elif self._subscriptions.is_fetchable(tp): + log.warning("partition %s is still fetchable -- skipping offset" + " update", tp) + continue + + # TODO: If there are several offsets to reset, + # we could submit offset requests in parallel + # for now, each call to _reset_offset will block + if self._subscriptions.is_offset_reset_needed(tp): + self._reset_offset(tp) + elif self._subscriptions.assignment[tp].committed is None: + # there's no committed position, so we need to reset with the + # default strategy + self._subscriptions.need_offset_reset(tp) + self._reset_offset(tp) + else: + committed = self._subscriptions.assignment[tp].committed + log.debug("Resetting offset for partition %s to the committed" + " offset %s", tp, committed) + self._subscriptions.seek(tp, committed) + + def _reset_offset(self, partition): + """Reset offsets for the given partition using the offset reset strategy. + + @param partition The given partition that needs reset offset + @raises NoOffsetForPartitionError If no offset reset strategy is defined + """ + timestamp = self._subscriptions.assignment[partition].reset_strategy + if timestamp is OffsetResetStrategy.EARLIEST: + strategy = 'earliest' + elif timestamp is OffsetResetStrategy.LATEST: + strategy = 'latest' + else: + raise NoOffsetForPartitionError(partition) + + log.debug("Resetting offset for partition %s to %s offset.", + partition, strategy) + offset = self._offset(partition, timestamp) + + # we might lose the assignment while fetching the offset, + # so check it is still active + if self._subscriptions.is_assigned(partition): + self._subscriptions.seek(partition, offset) + + def _offset(self, partition, timestamp): + """Fetch a single offset before the given timestamp for the partition. + + Blocks until offset is obtained, or a non-retriable exception is raised + + @param partition The partition that needs fetching offset. + @param timestamp The timestamp for fetching offset. + @raises exceptions + @return The offset of the message that is published before the given + timestamp + """ + while True: + future = self._send_offset_request(partition, timestamp) + self._client.poll(future=future) + + if future.succeeded(): + return future.value + + if not future.retriable(): + raise future.exception # pylint: disable-msg=raising-bad-type + + if future.exception.invalid_metadata: + refresh_future = self._client.cluster.request_update() + self._client.poll(future=refresh_future) + + def _raise_if_offset_out_of_range(self): + """ + If any partition from previous FetchResponse contains + OffsetOutOfRangeError and the default_reset_policy is None, + raise OffsetOutOfRangeError + """ + current_out_of_range_partitions = {} + + # filter only the fetchable partitions + for partition, offset in self._offset_out_of_range_partitions: + if not self._subscriptions.is_fetchable(partition): + log.debug("Ignoring fetched records for %s since it is no" + " longer fetchable", partition) + continue + consumed = self._subscriptions.assignment[partition].consumed + # ignore partition if its consumed offset != offset in FetchResponse + # e.g. after seek() + if consumed is not None and offset == consumed: + current_out_of_range_partitions[partition] = offset + + self._offset_out_of_range_partitions.clear() + if current_out_of_range_partitions: + raise Errors.OffsetOutOfRangeError(current_out_of_range_partitions) + + def _raise_if_unauthorized_topics(self): + """ + If any topic from previous FetchResponse contains an Authorization + error, raise an exception + + @raise TopicAuthorizationFailedError + """ + if self._unauthorized_topics: + topics = set(self._unauthorized_topics) + self._unauthorized_topics.clear() + raise Errors.TopicAuthorizationFailedError(topics) + + def _raise_if_record_too_large(self): + """ + If any partition from previous FetchResponse gets a RecordTooLarge + error, raise RecordTooLargeError + + @raise RecordTooLargeError If there is a message larger than fetch size + and hence cannot be ever returned + """ + copied_record_too_large_partitions = dict(self._record_too_large_partitions) + self._record_too_large_partitions.clear() + + if copied_record_too_large_partitions: + raise RecordTooLargeError( + "There are some messages at [Partition=Offset]: %s " + " whose size is larger than the fetch size %s" + " and hence cannot be ever returned." + " Increase the fetch size, or decrease the maximum message" + " size the broker will allow.", + copied_record_too_large_partitions, self._max_partition_fetch_bytes) + + def fetched_records(self): + """Returns previously fetched records and updates consumed offsets + + NOTE: returning empty records guarantees the consumed position are NOT updated. + + @return {TopicPartition: deque([messages])} + @raises OffsetOutOfRangeError if no subscription offset_reset_strategy + """ + if self._subscriptions.needs_partition_assignment: + return {} + + drained = collections.defaultdict(collections.deque) + self._raise_if_offset_out_of_range() + self._raise_if_unauthorized_topics() + self._raise_if_record_too_large() + + # Loop over the records deque + while self._records: + (fetch_offset, tp, messages) = self._records.popleft() + + if not self._subscriptions.is_assigned(tp): + # this can happen when a rebalance happened before + # fetched records are returned to the consumer's poll call + log.debug("Not returning fetched records for partition %s" + " since it is no longer assigned", tp) + continue + + # note that the consumed position should always be available + # as long as the partition is still assigned + consumed = self._subscriptions.assignment[tp].consumed + if not self._subscriptions.is_fetchable(tp): + # this can happen when a partition consumption paused before + # fetched records are returned to the consumer's poll call + log.debug("Not returning fetched records for assigned partition" + " %s since it is no longer fetchable", tp) + + # we also need to reset the fetch positions to pretend we did + # not fetch this partition in the previous request at all + self._subscriptions.assignment[tp].fetched = consumed + elif fetch_offset == consumed: + next_offset = messages[-1][0] + 1 + log.debug("Returning fetched records for assigned partition %s" + " and update consumed position to %s", tp, next_offset) + self._subscriptions.assignment[tp].consumed = next_offset + + # TODO: handle compressed messages + for offset, size, msg in messages: + if msg.attributes: + raise Errors.KafkaError('Compressed messages not supported yet') + elif self._check_crcs and not msg.validate_crc(): + raise Errors.InvalidMessageError(msg) + + key, value = self._deserialize(msg) + record = ConsumerRecord(tp.topic, tp.partition, offset, key, value) + drained[tp].append(record) + else: + # these records aren't next in line based on the last consumed + # position, ignore them they must be from an obsolete request + log.debug("Ignoring fetched records for %s at offset %s", + tp, fetch_offset) + return dict(drained) + + def _deserialize(self, msg): + if self._key_deserializer: + key = self._key_deserializer(msg.key) # pylint: disable-msg=not-callable + else: + key = msg.key + if self._value_deserializer: + value = self._value_deserializer(msg.value) # pylint: disable-msg=not-callable + else: + value = msg.value + return key, value + + def _send_offset_request(self, partition, timestamp): + """ + Fetch a single offset before the given timestamp for the partition. + + @param partition The TopicPartition that needs fetching offset. + @param timestamp The timestamp for fetching offset. + @return A future which can be polled to obtain the corresponding offset. + """ + node_id = self._client.cluster.leader_for_partition(partition) + if node_id is None: + log.debug("Partition %s is unknown for fetching offset," + " wait for metadata refresh", partition) + return Future().failure(Errors.StaleMetadata(partition)) + elif node_id == -1: + log.debug("Leader for partition %s unavailable for fetching offset," + " wait for metadata refresh", partition) + return Future().failure(Errors.LeaderNotAvailableError(partition)) + + request = OffsetRequest( + -1, [(partition.topic, [(partition.partition, timestamp, 1)])] + ) + # Client returns a future that only fails on network issues + # so create a separate future and attach a callback to update it + # based on response error codes + future = Future() + if not self._client.ready(node_id): + return future.failure(Errors.NodeNotReadyError(node_id)) + + _f = self._client.send(node_id, request) + _f.add_callback(self._handle_offset_response, partition, future) + _f.add_errback(lambda e: future.failure(e)) + return future + + def _handle_offset_response(self, partition, future, response): + """Callback for the response of the list offset call above. + + @param partition The partition that was fetched + @param future the future to update based on response + @param response The OffsetResponse from the server + + @raises IllegalStateError if response does not match partition + """ + topic, partition_info = response.topics[0] + if len(response.topics) != 1 or len(partition_info) != 1: + raise Errors.IllegalStateError("OffsetResponse should only be for" + " a single topic-partition") + + part, error_code, offsets = partition_info[0] + if topic != partition.topic or part != partition.partition: + raise Errors.IllegalStateError("OffsetResponse partition does not" + " match OffsetRequest partition") + + error_type = Errors.for_code(error_code) + if error_type is Errors.NoError: + if len(offsets) != 1: + raise Errors.IllegalStateError("OffsetResponse should only" + " return a single offset") + offset = offsets[0] + log.debug("Fetched offset %d for partition %s", offset, partition) + future.success(offset) + elif error_type in (Errors.NotLeaderForPartitionError, + Errors.UnknownTopicOrPartitionError): + log.warning("Attempt to fetch offsets for partition %s failed due" + " to obsolete leadership information, retrying.", + partition) + future.failure(error_type(partition)) + else: + log.error("Attempt to fetch offsets for partition %s failed due to:" + " %s", partition, error_type) + future.failure(error_type(partition)) + + def _create_fetch_requests(self): + """ + Create fetch requests for all assigned partitions, grouped by node + Except where no leader, node has requests in flight, or we have + not returned all previously fetched records to consumer + """ + # create the fetch info as a dict of lists of partition info tuples + # which can be passed to FetchRequest() via .items() + fetchable = collections.defaultdict(lambda: collections.defaultdict(list)) + + for partition in self._subscriptions.fetchable_partitions(): + node_id = self._client.cluster.leader_for_partition(partition) + if node_id is None or node_id == -1: + log.debug("No leader found for partition %s." + " Requesting metadata update", partition) + self._client.cluster.request_update() + elif self._client.in_flight_request_count(node_id) == 0: + # if there is a leader and no in-flight requests, + # issue a new fetch but only fetch data for partitions whose + # previously fetched data has been consumed + fetched = self._subscriptions.assignment[partition].fetched + consumed = self._subscriptions.assignment[partition].consumed + if consumed == fetched: + partition_info = ( + partition.partition, + fetched, + self._max_partition_fetch_bytes + ) + fetchable[node_id][partition.topic].append(partition_info) + else: + log.debug("Skipping FetchRequest to %s because previously" + " fetched offsets (%s) have not been fully" + " consumed yet (%s)", node_id, fetched, consumed) + + requests = {} + for node_id, partition_data in six.iteritems(fetchable): + requests[node_id] = FetchRequest( + -1, # replica_id + self._fetch_max_wait_ms, + self._fetch_min_bytes, + partition_data.items()) + return requests + + def _handle_fetch_response(self, request, response): + """The callback for fetch completion""" + #total_bytes = 0 + #total_count = 0 + + fetch_offsets = {} + for topic, partitions in request.topics: + for partition, offset, _ in partitions: + fetch_offsets[TopicPartition(topic, partition)] = offset + + for topic, partitions in response.topics: + for partition, error_code, highwater, messages in partitions: + tp = TopicPartition(topic, partition) + error_type = Errors.for_code(error_code) + if not self._subscriptions.is_fetchable(tp): + # this can happen when a rebalance happened or a partition + # consumption paused while fetch is still in-flight + log.debug("Ignoring fetched records for partition %s" + " since it is no longer fetchable", tp) + elif error_type is Errors.NoError: + fetch_offset = fetch_offsets[tp] + + # we are interested in this fetch only if the beginning + # offset matches the current consumed position + consumed = self._subscriptions.assignment[tp].consumed + if consumed is None: + continue + elif consumed != fetch_offset: + # the fetched position has gotten out of sync with the + # consumed position (which might happen when a + # rebalance occurs with a fetch in-flight), so we need + # to reset the fetch position so the next fetch is right + self._subscriptions.assignment[tp].fetched = consumed + continue + + partial = None + if messages and isinstance(messages[-1][-1], PartialMessage): + partial = messages.pop() + + if messages: + last_offset, _, _ = messages[-1] + self._subscriptions.assignment[tp].fetched = last_offset + 1 + self._records.append((fetch_offset, tp, messages)) + #self.sensors.records_fetch_lag.record(highwater - last_offset) + elif partial: + # we did not read a single message from a non-empty + # buffer because that message's size is larger than + # fetch size, in this case record this exception + self._record_too_large_partitions[tp] = fetch_offset + + # TODO: bytes metrics + #self.sensors.record_topic_fetch_metrics(tp.topic, num_bytes, parsed.size()); + #totalBytes += num_bytes; + #totalCount += parsed.size(); + elif error_type in (Errors.NotLeaderForPartitionError, + Errors.UnknownTopicOrPartitionError): + self._client.cluster.request_update() + elif error_type is Errors.OffsetOutOfRangeError: + fetch_offset = fetch_offsets[tp] + if self._subscriptions.has_default_offset_reset_policy(): + self._subscriptions.need_offset_reset(tp) + else: + self._offset_out_of_range_partitions[tp] = fetch_offset + log.info("Fetch offset %s is out of range, resetting offset", + self._subscriptions.assignment[tp].fetched) + elif error_type is Errors.TopicAuthorizationFailedError: + log.warn("Not authorized to read from topic %s.", tp.topic) + self._unauthorized_topics.add(tp.topic) + elif error_type is Errors.UnknownError: + log.warn("Unknown error fetching data for topic-partition %s", tp) + else: + raise Errors.IllegalStateError("Unexpected error code %s" + " while fetching data" + % error_code) + + """TOOD - metrics + self.sensors.bytesFetched.record(totalBytes) + self.sensors.recordsFetched.record(totalCount) + self.sensors.fetchThrottleTimeSensor.record(response.getThrottleTime()) + self.sensors.fetchLatency.record(resp.requestLatencyMs()) + + +class FetchManagerMetrics(object): + def __init__(self, metrics, prefix): + self.metrics = metrics + self.group_name = prefix + "-fetch-manager-metrics" + + self.bytes_fetched = metrics.sensor("bytes-fetched") + self.bytes_fetched.add(metrics.metricName("fetch-size-avg", self.group_name, + "The average number of bytes fetched per request"), metrics.Avg()) + self.bytes_fetched.add(metrics.metricName("fetch-size-max", self.group_name, + "The maximum number of bytes fetched per request"), metrics.Max()) + self.bytes_fetched.add(metrics.metricName("bytes-consumed-rate", self.group_name, + "The average number of bytes consumed per second"), metrics.Rate()) + + self.records_fetched = self.metrics.sensor("records-fetched") + self.records_fetched.add(metrics.metricName("records-per-request-avg", self.group_name, + "The average number of records in each request"), metrics.Avg()) + self.records_fetched.add(metrics.metricName("records-consumed-rate", self.group_name, + "The average number of records consumed per second"), metrics.Rate()) + + self.fetch_latency = metrics.sensor("fetch-latency") + self.fetch_latency.add(metrics.metricName("fetch-latency-avg", self.group_name, + "The average time taken for a fetch request."), metrics.Avg()) + self.fetch_latency.add(metrics.metricName("fetch-latency-max", self.group_name, + "The max time taken for any fetch request."), metrics.Max()) + self.fetch_latency.add(metrics.metricName("fetch-rate", self.group_name, + "The number of fetch requests per second."), metrics.Rate(metrics.Count())) + + self.records_fetch_lag = metrics.sensor("records-lag") + self.records_fetch_lag.add(metrics.metricName("records-lag-max", self.group_name, + "The maximum lag in terms of number of records for any partition in self window"), metrics.Max()) + + self.fetch_throttle_time_sensor = metrics.sensor("fetch-throttle-time") + self.fetch_throttle_time_sensor.add(metrics.metricName("fetch-throttle-time-avg", self.group_name, + "The average throttle time in ms"), metrics.Avg()) + self.fetch_throttle_time_sensor.add(metrics.metricName("fetch-throttle-time-max", self.group_name, + "The maximum throttle time in ms"), metrics.Max()) + + def record_topic_fetch_metrics(topic, num_bytes, num_records): + # record bytes fetched + name = '.'.join(["topic", topic, "bytes-fetched"]) + self.metrics[name].record(num_bytes); + + # record records fetched + name = '.'.join(["topic", topic, "records-fetched"]) + self.metrics[name].record(num_records) + """ -- cgit v1.2.1 From 264fc8bb7ccd314f6361e9bd223b1faa0354afbc Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 13:39:10 -0800 Subject: Add DEFAULT_GENERATION_ID and DEFAULT_RETENTION_TIME to OffsetCommitRequest_v2 --- kafka/protocol/commit.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py index 2955de1..a32f8d3 100644 --- a/kafka/protocol/commit.py +++ b/kafka/protocol/commit.py @@ -28,6 +28,8 @@ class OffsetCommitRequest_v2(Struct): ('offset', Int64), ('metadata', String('utf-8')))))) ) + DEFAULT_GENERATION_ID = -1 + DEFAULT_RETENTION_TIME = -1 class OffsetCommitRequest_v1(Struct): -- cgit v1.2.1 From 35eb8c5eaadbbb81f5e553d0ab10c5221a675378 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 13:39:27 -0800 Subject: Add UNKNOWN_MEMBER_ID to JoinGroupRequest --- kafka/protocol/group.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index 63e4a11..72de005 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -28,6 +28,7 @@ class JoinGroupRequest(Struct): ('protocol_name', String('utf-8')), ('protocol_metadata', Bytes))) ) + UNKNOWN_MEMBER_ID = '' class ProtocolMetadata(Struct): -- cgit v1.2.1 From 9820c5d55398bdb49ffbcd0e6a997bde9f8891fe Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 14:54:50 -0800 Subject: Define ConsumerRebalanceListener abstract class --- kafka/consumer/subscription_state.py | 84 ++++++++++++++++++++++++++++++++++++ 1 file changed, 84 insertions(+) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 6ebd925..a90d9b3 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -1,5 +1,6 @@ from __future__ import absolute_import +import abc import logging import re @@ -86,6 +87,9 @@ class SubscriptionState(object): self.subscribed_pattern = re.compile(pattern) else: self.change_subscription(topics) + + if listener and not isinstance(listener, ConsumerRebalanceListener): + raise TypeError('listener must be a ConsumerRebalanceListener') self.listener = listener def change_subscription(self, topics): @@ -302,3 +306,83 @@ class TopicPartitionState(object): def is_fetchable(self): return not self.paused and self.has_valid_position + + +class ConsumerRebalanceListener(object): + """ + A callback interface that the user can implement to trigger custom actions + when the set of partitions assigned to the consumer changes. + + This is applicable when the consumer is having Kafka auto-manage group + membership. If the consumer's directly assign partitions, those + partitions will never be reassigned and this callback is not applicable. + + When Kafka is managing the group membership, a partition re-assignment will + be triggered any time the members of the group changes or the subscription + of the members changes. This can occur when processes die, new process + instances are added or old instances come back to life after failure. + Rebalances can also be triggered by changes affecting the subscribed + topics (e.g. when then number of partitions is administratively adjusted). + + There are many uses for this functionality. One common use is saving offsets + in a custom store. By saving offsets in the on_partitions_revoked(), call we + can ensure that any time partition assignment changes the offset gets saved. + + Another use is flushing out any kind of cache of intermediate results the + consumer may be keeping. For example, consider a case where the consumer is + subscribed to a topic containing user page views, and the goal is to count + the number of page views per users for each five minute window. Let's say + the topic is partitioned by the user id so that all events for a particular + user will go to a single consumer instance. The consumer can keep in memory + a running tally of actions per user and only flush these out to a remote + data store when its cache gets too big. However if a partition is reassigned + it may want to automatically trigger a flush of this cache, before the new + owner takes over consumption. + + This callback will execute in the user thread as part of the Consumer.poll() + whenever partition assignment changes. + + It is guaranteed that all consumer processes will invoke + on_partitions_revoked() prior to any process invoking + on_partitions_assigned(). So if offsets or other state is saved in the + on_partitions_revoked() call, it should be saved by the time the process + taking over that partition has their on_partitions_assigned() callback + called to load the state. + """ + __metaclass__ = abc.ABCMeta + + @abc.abstractmethod + def on_partitions_revoked(self, revoked): + """ + A callback method the user can implement to provide handling of offset + commits to a customized store on the start of a rebalance operation. + This method will be called before a rebalance operation starts and + after the consumer stops fetching data. It is recommended that offsets + should be committed in this callback to either Kafka or a custom offset + store to prevent duplicate data. + + NOTE: This method is only called before rebalances. It is not called + prior to KafkaConsumer.close() + + @param partitions The list of partitions that were assigned to the + consumer on the last rebalance + """ + pass + + @abc.abstractmethod + def on_partitions_assigned(self, assigned): + """ + A callback method the user can implement to provide handling of + customized offsets on completion of a successful partition + re-assignment. This method will be called after an offset re-assignment + completes and before the consumer starts fetching data. + + It is guaranteed that all the processes in a consumer group will execute + their on_partitions_revoked() callback before any instance executes its + on_partitions_assigned() callback. + + @param partitions The list of partitions that are now assigned to the + consumer (may include partitions previously assigned + to the consumer) + """ + pass -- cgit v1.2.1 From baab076c7e70a721d958f588c4199acbaae41481 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 15:05:08 -0800 Subject: AbstractCoordinator (modelled after upstream Java class) - Heartbeat class manages HeartbeatRequest / Response state - metrics are still TODO --- kafka/common.py | 5 + kafka/coordinator/__init__.py | 0 kafka/coordinator/abstract.py | 642 +++++++++++++++++++++++++++++++++++++++++ kafka/coordinator/heartbeat.py | 43 +++ 4 files changed, 690 insertions(+) create mode 100644 kafka/coordinator/__init__.py create mode 100644 kafka/coordinator/abstract.py create mode 100644 kafka/coordinator/heartbeat.py diff --git a/kafka/common.py b/kafka/common.py index 33f24a4..597fb5c 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -111,6 +111,11 @@ class DisconnectError(KafkaError): invalid_metadata = True +class NoBrokersAvailable(KafkaError): + retriable = True + invalid_metadata = True + + class NodeNotReadyError(KafkaError): retriable = True diff --git a/kafka/coordinator/__init__.py b/kafka/coordinator/__init__.py new file mode 100644 index 0000000..e69de29 diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py new file mode 100644 index 0000000..1713433 --- /dev/null +++ b/kafka/coordinator/abstract.py @@ -0,0 +1,642 @@ +import abc +import logging +import time + +import six + +import kafka.common as Errors +from kafka.future import Future +from kafka.protocol.commit import (GroupCoordinatorRequest, + OffsetCommitRequest_v2 as OffsetCommitRequest) +from kafka.protocol.group import (HeartbeatRequest, JoinGroupRequest, + LeaveGroupRequest, SyncGroupRequest) +from .heartbeat import Heartbeat + +log = logging.getLogger(__name__) + + +class AbstractCoordinator(object): + """ + AbstractCoordinator implements group management for a single group member + by interacting with a designated Kafka broker (the coordinator). Group + semantics are provided by extending this class. See ConsumerCoordinator + for example usage. + + From a high level, Kafka's group management protocol consists of the + following sequence of actions: + + 1. Group Registration: Group members register with the coordinator providing + their own metadata (such as the set of topics they are interested in). + + 2. Group/Leader Selection: The coordinator select the members of the group + and chooses one member as the leader. + + 3. State Assignment: The leader collects the metadata from all the members + of the group and assigns state. + + 4. Group Stabilization: Each member receives the state assigned by the + leader and begins processing. + + To leverage this protocol, an implementation must define the format of + metadata provided by each member for group registration in group_protocols() + and the format of the state assignment provided by the leader in + _perform_assignment() and which becomes available to members in + _on_join_complete(). + """ + + _session_timeout_ms = 30000 + _heartbeat_interval_ms = 3000 + _retry_backoff_ms = 100 + + def __init__(self, client, group_id, **kwargs): + if not client: + raise Errors.IllegalStateError('a client is required to use' + ' Group Coordinator') + if not group_id: + raise Errors.IllegalStateError('a group_id is required to use' + ' Group Coordinator') + for config in ('session_timeout_ms', + 'heartbeat_interval_ms', + 'retry_backoff_ms'): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) + + self._client = client + self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + self.group_id = group_id + self.coordinator_id = None + self.rejoin_needed = True + self.needs_join_prepare = True + self.heartbeat = Heartbeat( + session_timeout_ms=self._session_timeout_ms, + heartbeat_interval_ms=self._heartbeat_interval_ms) + self.heartbeat_task = HeartbeatTask(self) + #self.sensors = GroupCoordinatorMetrics(metrics, metric_group_prefix, metric_tags) + + @abc.abstractmethod + def protocol_type(self): + """ + Unique identifier for the class of protocols implements + (e.g. "consumer" or "connect"). + + @return str protocol type name + """ + pass + + @abc.abstractmethod + def group_protocols(self): + """Return the list of supported group protocols and metadata. + + This list is submitted by each group member via a JoinGroupRequest. + The order of the protocols in the list indicates the preference of the + protocol (the first entry is the most preferred). The coordinator takes + this preference into account when selecting the generation protocol + (generally more preferred protocols will be selected as long as all + members support them and there is no disagreement on the preference). + + Note: metadata must be type bytes or support an encode() method + + @return [(protocol, metadata), ...] + """ + pass + + @abc.abstractmethod + def _on_join_prepare(self, generation, member_id): + """Invoked prior to each group join or rejoin. + + This is typically used to perform any cleanup from the previous + generation (such as committing offsets for the consumer) + + @param generation The previous generation or -1 if there was none + @param member_id The identifier of this member in the previous group + or '' if there was none + """ + pass + + @abc.abstractmethod + def _perform_assignment(self, leader_id, protocol, members): + """Perform assignment for the group. + + This is used by the leader to push state to all the members of the group + (e.g. to push partition assignments in the case of the new consumer) + + @param leader_id: The id of the leader (which is this member) + @param protocol: the chosen group protocol (assignment strategy) + @param members: [(member_id, metadata_bytes)] from JoinGroupResponse. + metadata_bytes are associated with the chosen group + protocol, and the Coordinator subclass is responsible + for decoding metadata_bytes based on that protocol. + + @return dict of {member_id: assignment}; assignment must either be bytes + or have an encode() method to convert to bytes + """ + pass + + @abc.abstractmethod + def _on_join_complete(self, generation, member_id, protocol, + member_assignment_bytes): + """Invoked when a group member has successfully joined a group. + + @param generation The generation that was joined + @param member_id The identifier for the local member in the group + @param protocol The protocol selected by the coordinator + @param member_assignment_bytes The protocol-encoded assignment + propagated from the group leader. The Coordinator instance is + responsible for decoding based on the chosen protocol. + """ + pass + + def coordinator_unknown(self): + """ + Check if we know who the coordinator is and we have an active connection + + Side-effect: reset coordinator_id to None if connection failed + + @return True if the coordinator is unknown + """ + if self.coordinator_id is None: + return True + + if self._client.connection_failed(self.coordinator_id): + self.coordinator_dead() + return True + + return not self._client.ready(self.coordinator_id) + + def ensure_coordinator_known(self): + """Block until the coordinator for this group is known + (and we have an active connection -- java client uses unsent queue). + """ + while self.coordinator_unknown(): + + # Dont look for a new coordinator node if we are just waiting + # for connection to finish + if self.coordinator_id is not None: + self._client.poll() + continue + + future = self.send_group_metadata_request() + self._client.poll(future=future) + + if future.failed(): + if future.retriable(): + metadata_update = self._client.cluster.request_update() + self._client.poll(future=metadata_update) + else: + raise future.exception # pylint: disable-msg=raising-bad-type + + def need_rejoin(self): + """ + Check whether the group should be rejoined (e.g. if metadata changes) + @return True if it should, False otherwise + """ + return self.rejoin_needed + + def ensure_active_group(self): + """Ensure that the group is active (i.e. joined and synced)""" + if not self.need_rejoin(): + return + + if self.needs_join_prepare: + self._on_join_prepare(self.generation, self.member_id) + self.needs_join_prepare = False + + while self.need_rejoin(): + self.ensure_coordinator_known() + + future = self.perform_group_join() + self._client.poll(future=future) + + if future.succeeded(): + member_assignment_bytes = future.value + self._on_join_complete(self.generation, self.member_id, + self.protocol, member_assignment_bytes) + self.needs_join_prepare = True + self.heartbeat_task.reset() + else: + exception = future.exception + if isinstance(exception, (Errors.UnknownMemberIdError, + Errors.RebalanceInProgressError, + Errors.IllegalGenerationError)): + continue + elif not future.retriable(): + raise exception # pylint: disable-msg=raising-bad-type + time.sleep(self._retry_backoff_ms / 1000.0) + + def perform_group_join(self): + """Join the group and return the assignment for the next generation. + + This function handles both JoinGroup and SyncGroup, delegating to + _perform_assignment() if elected leader by the coordinator. + + @return Future() of the assignment returned from the group leader + """ + if self.coordinator_unknown(): + e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) + return Future().failure(e) + + # send a join group request to the coordinator + log.debug("(Re-)joining group %s", self.group_id) + request = JoinGroupRequest( + self.group_id, + self._session_timeout_ms, + self.member_id, + self.protocol_type(), + [(protocol, + metadata if isinstance(metadata, bytes) else metadata.encode()) + for protocol, metadata in self.group_protocols()]) + + # create the request for the coordinator + log.debug("Issuing request (%s) to coordinator %s", request, self.coordinator_id) + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_join_group_response, future) + _f.add_errback(self._failed_request, future) + return future + + def _failed_request(self, future, error): + self.coordinator_dead() + future.failure(error) + + def _handle_join_group_response(self, future, response): + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.debug("Joined group: %s", response) + self.member_id = response.member_id + self.generation = response.generation_id + self.rejoin_needed = False + self.protocol = response.group_protocol + #self.sensors.join_latency.record(response.requestLatencyMs()) + if response.leader_id == response.member_id: + self.on_join_leader(response).chain(future) + else: + self.on_join_follower().chain(future) + + elif error_type is Errors.GroupLoadInProgressError: + log.debug("Attempt to join group %s rejected since coordinator is" + " loading the group.", self.group_id) + # backoff and retry + future.failure(error_type(response)) + elif error_type is Errors.UnknownMemberIdError: + # reset the member id and retry immediately + error = error_type(self.member_id) + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + log.info("Attempt to join group %s failed due to unknown member id," + " resetting and retrying.", self.group_id) + future.failure(error) + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError): + # re-discover the coordinator and retry with backoff + self.coordinator_dead() + log.info("Attempt to join group %s failed due to obsolete " + "coordinator information, retrying.", self.group_id) + future.failure(error_type()) + elif error_type in (Errors.InconsistentGroupProtocolError, + Errors.InvalidSessionTimeoutError, + Errors.InvalidGroupIdError): + # log the error and re-throw the exception + error = error_type(response) + log.error("Attempt to join group %s failed due to: %s", + self.group_id, error) + future.failure(error) + elif error_type is Errors.GroupAuthorizationFailedError: + future.failure(error_type(self.group_id)) + else: + # unexpected error, throw the exception + error = error_type() + log.error("Unexpected error in join group response: %s", error) + future.failure(error) + + def on_join_follower(self): + # send follower's sync group with an empty assignment + request = SyncGroupRequest( + self.group_id, + self.generation, + self.member_id, + {}) + log.debug("Issuing follower SyncGroup (%s) to coordinator %s", + request, self.coordinator_id) + return self.send_sync_group_request(request) + + def on_join_leader(self, response): + """ + Perform leader synchronization and send back the assignment + for the group via SyncGroupRequest + """ + try: + group_assignment = self._perform_assignment(response.leader_id, + response.group_protocol, + response.members) + except Exception as e: + raise + return Future().failure(e) + + request = SyncGroupRequest( + self.group_id, + self.generation, + self.member_id, + [(member_id, + assignment if isinstance(assignment, bytes) else assignment.encode()) + for member_id, assignment in six.iteritems(group_assignment)]) + + log.debug("Issuing leader SyncGroup (%s) to coordinator %s", + request, self.coordinator_id) + return self.send_sync_group_request(request) + + def send_sync_group_request(self, request): + if self.coordinator_unknown(): + return Future().failure(Errors.GroupCoordinatorNotAvailableError()) + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_sync_group_response, future) + _f.add_errback(self._failed_request, future) + return future + + def _handle_sync_group_response(self, future, response): + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.debug("Received successful sync group response for group %s: %s", + self.group_id, response) + #self.sensors.syncLatency.record(response.requestLatencyMs()) + future.success(response.member_assignment) + return + + # Always rejoin on error + self.rejoin_needed = True + if error_type is Errors.GroupAuthorizationFailedError: + future.failure(error_type(self.group_id)) + elif error_type is Errors.RebalanceInProgressError: + log.info("SyncGroup for group %s failed due to coordinator" + " rebalance, rejoining the group", self.group_id) + future.failure(error_type(self.group_id)) + elif error_type in (Errors.UnknownMemberIdError, + Errors.IllegalGenerationError): + error = error_type() + log.info("SyncGroup for group %s failed due to %s," + " rejoining the group", self.group_id, error) + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + future.failure(error) + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError): + error = error_type() + log.info("SyncGroup for group %s failed due to %s, will find new" + " coordinator and rejoin", self.group_id, error) + self.coordinator_dead() + future.failure(error) + else: + error = error_type() + log.error("Unexpected error from SyncGroup: %s", error) + future.failure(error) + + def send_group_metadata_request(self): + """Discover the current coordinator for the group. + + Sends a GroupMetadata request to one of the brokers. The returned future + should be polled to get the result of the request. + + @return future indicating the completion of the metadata request + """ + node_id = self._client.least_loaded_node() + if node_id is None or not self._client.ready(node_id): + return Future().failure(Errors.NoBrokersAvailable()) + + log.debug("Issuing group metadata request to broker %s", node_id) + request = GroupCoordinatorRequest(self.group_id) + future = Future() + _f = self._client.send(node_id, request) + _f.add_callback(self._handle_group_coordinator_response, future) + _f.add_errback(self._failed_request, future) + return future + + def _handle_group_coordinator_response(self, future, response): + log.debug("Group metadata response %s", response) + if not self.coordinator_unknown(): + # We already found the coordinator, so ignore the request + log.debug("Coordinator already known -- ignoring metadata response") + future.success(self.coordinator_id) + return + + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + ok = self._client.cluster.add_group_coordinator(self.group_id, response) + if not ok: + # This could happen if coordinator metadata is different + # than broker metadata + future.failure(Errors.IllegalStateError()) + return + + self.coordinator_id = response.coordinator_id + self._client.ready(self.coordinator_id) + + # start sending heartbeats only if we have a valid generation + if self.generation > 0: + self.heartbeat_task.reset() + future.success(None) + elif error_type is Errors.GroupAuthorizationFailedError: + error = error_type(self.group_id) + log.error("Group Coordinator Request failed: %s", error) + future.failure(error) + else: + error = error_type() + log.error("Unrecognized failure in Group Coordinator Request: %s", + error) + future.failure(error) + + def coordinator_dead(self, error=None): + """Mark the current coordinator as dead.""" + if self.coordinator_id is not None: + log.info("Marking the coordinator dead (node %s): %s.", + self.coordinator_id, error) + self.coordinator_id = None + + def close(self): + """Close the coordinator, leave the current group + and reset local generation/memberId.""" + try: + self._client.unschedule(self.heartbeat_task) + except KeyError: + pass + if not self.coordinator_unknown() and self.generation > 0: + # this is a minimal effort attempt to leave the group. we do not + # attempt any resending if the request fails or times out. + request = LeaveGroupRequest(self.group_id, self.member_id) + future = self._client.send(self.coordinator_id, request) + future.add_callback(self._handle_leave_group_response) + future.add_errback(log.error, "LeaveGroup request failed: %s") + self._client.poll(future=future) + + self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + self.rejoin_needed = True + + def _handle_leave_group_response(self, response): + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.info("LeaveGroup request succeeded") + else: + log.error("LeaveGroup request failed: %s", error_type()) + + def send_heartbeat_request(self): + """Send a heartbeat request now (visible only for testing).""" + request = HeartbeatRequest(self.group_id, self.generation, self.member_id) + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_heartbeat_response, future) + _f.add_errback(self._failed_request, future) + return future + + def _handle_heartbeat_response(self, future, response): + #self.sensors.heartbeat_latency.record(response.requestLatencyMs()) + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.debug("Received successful heartbeat response.") + future.success(None) + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError): + log.info("Attempt to heart beat failed since coordinator is either" + " not started or not valid; marking it as dead.") + self.coordinator_dead() + future.failure(error_type()) + elif error_type is Errors.RebalanceInProgressError: + log.info("Attempt to heart beat failed since the group is" + " rebalancing; try to re-join group.") + self.rejoin_needed = True + future.failure(error_type()) + elif error_type is Errors.IllegalGenerationError: + log.info("Attempt to heart beat failed since generation id" + " is not legal; try to re-join group.") + self.rejoin_needed = True + future.failure(error_type()) + elif error_type is Errors.UnknownMemberIdError: + log.info("Attempt to heart beat failed since member id" + " is not valid; reset it and try to re-join group.") + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + self.rejoin_needed = True + future.failure(error_type) + elif error_type is Errors.GroupAuthorizationFailedError: + error = error_type(self.group_id) + log.error("Attempt to heart beat failed authorization: %s", error) + future.failure(error) + else: + error = error_type() + log.error("Unknown error in heart beat response: %s", error) + future.failure(error) + + +class HeartbeatTask(object): + def __init__(self, coordinator): + self._coordinator = coordinator + self._heartbeat = coordinator.heartbeat + self._client = coordinator._client + self._request_in_flight = False + + def reset(self): + # start or restart the heartbeat task to be executed at the next chance + self._heartbeat.reset_session_timeout() + try: + self._client.unschedule(self) + except KeyError: + pass + if not self._request_in_flight: + self._client.schedule(self, time.time()) + + def __call__(self): + log.debug("Running Heartbeat task") + if (self._coordinator.generation < 0 or + self._coordinator.need_rejoin() or + self._coordinator.coordinator_unknown()): + # no need to send the heartbeat we're not using auto-assignment + # or if we are awaiting a rebalance + log.debug("Skipping heartbeat: no auto-assignment" + " or waiting on rebalance") + return + + if self._heartbeat.session_expired(): + # we haven't received a successful heartbeat in one session interval + # so mark the coordinator dead + log.error("Heartbeat session expired") + self._coordinator.coordinator_dead() + return + + if not self._heartbeat.should_heartbeat(): + # we don't need to heartbeat now, so reschedule for when we do + ttl = self._heartbeat.ttl() + log.debug("Heartbeat unneeded now, retrying in %s", ttl) + self._client.schedule(self, time.time() + ttl) + else: + log.debug("Sending HeartbeatRequest") + self._heartbeat.sent_heartbeat() + self._request_in_flight = True + future = self._coordinator.send_heartbeat_request() + future.add_callback(self._handle_heartbeat_success) + future.add_errback(self._handle_heartbeat_failure) + + def _handle_heartbeat_success(self, v): + log.debug("Received successful heartbeat") + self.request_in_flight = False + self._heartbeat.received_heartbeat() + ttl = self._heartbeat.ttl() + self._client.schedule(self, time.time() + ttl) + + def _handle_heartbeat_failure(self, e): + log.debug("Heartbeat failed; retrying") + self._request_in_flight = False + etd = time.time() + self._coordinator._retry_backoff_ms / 1000.0 + self._client.schedule(self, etd) + + +class GroupCoordinatorMetrics(object): + def __init__(self, metrics, prefix, tags=None): + self.metrics = metrics + self.group_name = prefix + "-coordinator-metrics" + + self.heartbeat_latency = metrics.sensor("heartbeat-latency") + self.heartbeat_latency.add(metrics.metricName( + "heartbeat-response-time-max", self.group_name, + "The max time taken to receive a response to a heartbeat request", + tags), metrics.Max()) + self.heartbeat_latency.add(metrics.metricName( + "heartbeat-rate", self.group_name, + "The average number of heartbeats per second", + tags), metrics.Rate(metrics.Count())) + + self.join_latency = metrics.sensor("join-latency") + self.join_latency.add(metrics.metricName( + "join-time-avg", self.group_name, + "The average time taken for a group rejoin", + tags), metrics.Avg()) + self.join_latency.add(metrics.metricName( + "join-time-max", self.group_name, + "The max time taken for a group rejoin", + tags), metrics.Avg()) + self.join_latency.add(metrics.metricName( + "join-rate", self.group_name, + "The number of group joins per second", + tags), metrics.Rate(metrics.Count())) + + self.sync_latency = metrics.sensor("sync-latency") + self.sync_latency.add(metrics.metricName( + "sync-time-avg", self.group_name, + "The average time taken for a group sync", + tags), metrics.Avg()) + self.sync_latency.add(metrics.MetricName( + "sync-time-max", self.group_name, + "The max time taken for a group sync", + tags), metrics.Avg()) + self.sync_latency.add(metrics.metricName( + "sync-rate", self.group_name, + "The number of group syncs per second", + tags), metrics.Rate(metrics.Count())) + + """ + lastHeartbeat = Measurable( + measure=lambda _, value: value - heartbeat.last_heartbeat_send() + ) + metrics.addMetric(metrics.metricName( + "last-heartbeat-seconds-ago", self.group_name, + "The number of seconds since the last controller heartbeat", + tags), lastHeartbeat) + """ + + diff --git a/kafka/coordinator/heartbeat.py b/kafka/coordinator/heartbeat.py new file mode 100644 index 0000000..c153ddd --- /dev/null +++ b/kafka/coordinator/heartbeat.py @@ -0,0 +1,43 @@ +import time + +import kafka.common as Errors + + +class Heartbeat(object): + _heartbeat_interval_ms = 3000 + _session_timeout_ms = 30000 + + def __init__(self, **kwargs): + for config in ('heartbeat_interval_ms', 'session_timeout_ms'): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) + + if self._heartbeat_interval_ms > self._session_timeout_ms: + raise Errors.IllegalArgumentError("Heartbeat interval must be set" + " lower than the session timeout") + + self.interval = self._heartbeat_interval_ms / 1000.0 + self.timeout = self._session_timeout_ms / 1000.0 + self.last_send = 0 + self.last_receive = 0 + self.last_reset = time.time() + + def sent_heartbeat(self): + self.last_send = time.time() + + def received_heartbeat(self): + self.last_receive = time.time() + + def ttl(self): + last_beat = max(self.last_send, self.last_reset) + return max(0, last_beat + self.interval - time.time()) + + def should_heartbeat(self): + return self.ttl() == 0 + + def session_expired(self): + last_recv = max(self.last_receive, self.last_reset) + return (time.time() - last_recv) > self.timeout + + def reset_session_timeout(self): + self.last_reset = time.time() -- cgit v1.2.1 From 161d9ffcf8e879bf65c44ea55851c72ef0b80aa6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 15:16:55 -0800 Subject: ConsumerCoordinator (based on upstream Java client) - Use RoundRobinPartitionAssignor by default - Define AbstractPartitionAssignor for custom assignors - metrics still TODO --- kafka/coordinator/assignors/__init__.py | 0 kafka/coordinator/assignors/abstract.py | 35 ++ kafka/coordinator/assignors/roundrobin.py | 63 ++++ kafka/coordinator/consumer.py | 605 ++++++++++++++++++++++++++++++ 4 files changed, 703 insertions(+) create mode 100644 kafka/coordinator/assignors/__init__.py create mode 100644 kafka/coordinator/assignors/abstract.py create mode 100644 kafka/coordinator/assignors/roundrobin.py create mode 100644 kafka/coordinator/consumer.py diff --git a/kafka/coordinator/assignors/__init__.py b/kafka/coordinator/assignors/__init__.py new file mode 100644 index 0000000..e69de29 diff --git a/kafka/coordinator/assignors/abstract.py b/kafka/coordinator/assignors/abstract.py new file mode 100644 index 0000000..ed09a6e --- /dev/null +++ b/kafka/coordinator/assignors/abstract.py @@ -0,0 +1,35 @@ +import abc +import logging + +log = logging.getLogger(__name__) + + +class AbstractPartitionAssignor(object): + """ + Abstract assignor implementation which does some common grunt work (in particular collecting + partition counts which are always needed in assignors). + """ + + @abc.abstractproperty + def name(self): + """.name should be a string identifying the assignor""" + pass + + @abc.abstractmethod + def assign(self, cluster, members): + """Perform group assignment given cluster metadata and member subscriptions + + @param cluster: cluster metadata + @param members: {member_id: subscription} + @return {member_id: MemberAssignment} + """ + pass + + @abc.abstractmethod + def metadata(self, topics): + """return ProtocolMetadata to be submitted via JoinGroupRequest""" + pass + + @abc.abstractmethod + def on_assignment(self, assignment): + pass diff --git a/kafka/coordinator/assignors/roundrobin.py b/kafka/coordinator/assignors/roundrobin.py new file mode 100644 index 0000000..2927f3e --- /dev/null +++ b/kafka/coordinator/assignors/roundrobin.py @@ -0,0 +1,63 @@ +import collections +import itertools +import logging + +import six + +from .abstract import AbstractPartitionAssignor +from ...common import TopicPartition +from ..consumer import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment + +log = logging.getLogger(__name__) + + +class RoundRobinPartitionAssignor(AbstractPartitionAssignor): + name = 'roundrobin' + version = 0 + + @classmethod + def assign(cls, cluster, member_metadata): + all_topics = set() + for metadata in six.itervalues(member_metadata): + all_topics.update(metadata.subscription) + + all_topic_partitions = [] + for topic in all_topics: + partitions = cluster.partitions_for_topic(topic) + if partitions is None: + log.warning('No partition metadata for topic %s', topic) + continue + for partition in partitions: + all_topic_partitions.append(TopicPartition(topic, partition)) + all_topic_partitions.sort() + + # construct {member_id: {topic: [partition, ...]}} + assignment = collections.defaultdict(lambda: collections.defaultdict(list)) + + member_iter = itertools.cycle(sorted(member_metadata.keys())) + for partition in all_topic_partitions: + member_id = member_iter.next() + + # Because we constructed all_topic_partitions from the set of + # member subscribed topics, we should be safe assuming that + # each topic in all_topic_partitions is in at least one member + # subscription; otherwise this could yield an infinite loop + while partition.topic not in member_metadata[member_id].subscription: + member_id = member_iter.next() + assignment[member_id][partition.topic].append(partition.partition) + + protocol_assignment = {} + for member_id in member_metadata: + protocol_assignment[member_id] = ConsumerProtocolMemberAssignment( + cls.version, + assignment[member_id].items(), + b'') + return protocol_assignment + + @classmethod + def metadata(cls, topics): + return ConsumerProtocolMemberMetadata(cls.version, list(topics), b'') + + @classmethod + def on_assignment(cls, assignment): + pass diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py new file mode 100644 index 0000000..c17c593 --- /dev/null +++ b/kafka/coordinator/consumer.py @@ -0,0 +1,605 @@ +import collections +import logging +import time + +import six + +from .abstract import AbstractCoordinator +import kafka.common as Errors +from kafka.common import OffsetAndMetadata, TopicPartition +from kafka.future import Future +from kafka.protocol.commit import OffsetCommitRequest_v2, OffsetFetchRequest_v1 +from kafka.protocol.struct import Struct +from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String + +log = logging.getLogger(__name__) + + +class ConsumerProtocolMemberMetadata(Struct): + SCHEMA = Schema( + ('version', Int16), + ('subscription', Array(String('utf-8'))), + ('user_data', Bytes)) + + +class ConsumerProtocolMemberAssignment(Struct): + SCHEMA = Schema( + ('version', Int16), + ('assignment', Array( + ('topic', String('utf-8')), + ('partitions', Array(Int32)))), + ('user_data', Bytes)) + + def partitions(self): + return [TopicPartition(topic, partition) + for topic, partitions in self.assignment # pylint: disable-msg=no-member + for partition in partitions] + + +class ConsumerProtocol(object): + PROTOCOL_TYPE = 'consumer' + ASSIGNMENT_STRATEGIES = ('roundrobin',) + METADATA = ConsumerProtocolMemberMetadata + ASSIGNMENT = ConsumerProtocolMemberAssignment + + +class ConsumerCoordinator(AbstractCoordinator): + """This class manages the coordination process with the consumer coordinator.""" + _enable_auto_commit = True + _auto_commit_interval_ms = 60 * 1000 + _default_offset_commit_callback = lambda offsets, error: True + _assignors = () + #_heartbeat_interval_ms = 3000 + #_session_timeout_ms = 30000 + #_retry_backoff_ms = 100 + + def __init__(self, client, group_id, subscription, **kwargs): + """Initialize the coordination manager.""" + super(ConsumerCoordinator, self).__init__(client, group_id, **kwargs) + for config in ('enable_auto_commit', 'auto_commit_interval_ms', + 'default_offset_commit_callback', 'assignors'): + if config in kwargs: + setattr(self, '_' + config, kwargs.pop(config)) + + self._cluster = client.cluster + self._subscription = subscription + self._partitions_per_topic = {} + self._auto_commit_task = None + if not self._assignors: + raise Errors.IllegalStateError('Coordinator requires assignors') + + self._cluster.request_update() + self._cluster.add_listener(self._handle_metadata_update) #TODO + + if self._enable_auto_commit: + interval = self._auto_commit_interval_ms / 1000.0 + self._auto_commit_task = AutoCommitTask(self, interval) + + # metrics=None, + # metric_group_prefix=None, + # metric_tags=None, + # self.sensors = ConsumerCoordinatorMetrics(metrics, metric_group_prefix, metric_tags) + + def protocol_type(self): + return ConsumerProtocol.PROTOCOL_TYPE + + def group_protocols(self): + """Returns list of preferred (protocols, metadata)""" + topics = self._subscription.subscription + metadata_list = [] + for assignor in self._assignors: + metadata = assignor.metadata(topics) + group_protocol = (assignor.name, metadata) + metadata_list.append(group_protocol) + return metadata_list + + def _handle_metadata_update(self, cluster): + # if we encounter any unauthorized topics, raise an exception + # TODO + #if self._cluster.unauthorized_topics: + # raise Errors.TopicAuthorizationError(self._cluster.unauthorized_topics) + + if self._subscription.subscribed_pattern: + topics = [] + for topic in cluster.topics(): + if self._subscription.subscribed_pattern.match(topic): + topics.append(topic) + + self._subscription.change_subscription(topics) + self._client.set_topics(self._subscription.group_subscription()) + + # check if there are any changes to the metadata which should trigger a rebalance + if self._subscription_metadata_changed(): + self._subscription.mark_for_reassignment() + + def _subscription_metadata_changed(self): + if not self._subscription.partitions_auto_assigned(): + return False + + old_partitions_per_topic = self._partitions_per_topic + self._partitions_per_topic = {} + for topic in self._subscription.group_subscription(): + self._partitions_per_topic[topic] = set(self._cluster.partitions_for_topic(topic)) + + if self._partitions_per_topic != old_partitions_per_topic: + return True + return False + + def _lookup_assignor(self, name): + for assignor in self._assignors: + if assignor.name == name: + return assignor + return None + + def _on_join_complete(self, generation, member_id, protocol, + member_assignment_bytes): + assignor = self._lookup_assignor(protocol) + if not assignor: + raise Errors.IllegalStateError("Coordinator selected invalid" + " assignment protocol: %s" + % protocol) + + assignment = ConsumerProtocol.ASSIGNMENT.decode(member_assignment_bytes) + + # set the flag to refresh last committed offsets + self._subscription.needs_fetch_committed_offsets = True + + # update partition assignment + self._subscription.assign_from_subscribed(assignment.partitions()) + + # give the assignor a chance to update internal state + # based on the received assignment + assignor.on_assignment(assignment) + + # restart the autocommit task if needed + if self._enable_auto_commit: + self._auto_commit_task.enable() + + assigned = set(self._subscription.assigned_partitions()) + log.debug("Set newly assigned partitions %s", assigned) + + # execute the user's callback after rebalance + if self._subscription.listener: + try: + self._subscriptions.listener.on_partitions_assigned(assigned) + except Exception: + log.exception("User provided listener failed on partition" + " assignment: %s", assigned) + + def _perform_assignment(self, leader_id, assignment_strategy, members): + assignor = self._lookup_assignor(assignment_strategy) + if not assignor: + raise Errors.IllegalStateError("Coordinator selected invalid" + " assignment protocol: %s" + % assignment_strategy) + member_metadata = {} + all_subscribed_topics = set() + for member_id, metadata_bytes in members: + metadata = ConsumerProtocol.METADATA.decode(metadata_bytes) + member_metadata[member_id] = metadata + all_subscribed_topics.update(metadata.subscription) # pylint: disable-msg=no-member + + # the leader will begin watching for changes to any of the topics + # the group is interested in, which ensures that all metadata changes + # will eventually be seen + self._subscription.group_subscribe(all_subscribed_topics) + future = self._client.set_topics(self._subscription.group_subscription()) + self._client.poll(future=future) + + log.debug("Performing %s assignment for subscriptions %s", + assignor.name, member_metadata) + + assignments = assignor.assign(self._cluster, member_metadata) + + log.debug("Finished assignment: %s", assignments) + + group_assignment = {} + for member_id, assignment in six.iteritems(assignments): + group_assignment[member_id] = assignment + return group_assignment + + def _on_join_prepare(self, generation, member_id): + # commit offsets prior to rebalance if auto-commit enabled + self._maybe_auto_commit_offsets_sync() + + # execute the user's callback before rebalance + log.debug("Revoking previously assigned partitions %s", + self._subscription.assigned_partitions()) + if self._subscription.listener: + try: + revoked = set(self._subscription.assigned_partitions()) + self._subscription.listener.on_partitions_revoked(revoked) + except Exception: + log.exception("User provided subscription listener failed" + " on_partitions_revoked") + + self._subscription.mark_for_reassignment() + + def need_rejoin(self): + """ + Check whether the group should be rejoined (e.g. if metadata changes) + @return True if it should, False otherwise + """ + return (self._subscription.partitions_auto_assigned() and + (super(ConsumerCoordinator, self).need_rejoin() or + self._subscription.needs_partition_assignment)) + + def refresh_committed_offsets_if_needed(self): + """Fetch committed offsets for assigned partitions.""" + if self._subscription.needs_fetch_committed_offsets: + offsets = self.fetch_committed_offsets(self._subscription.assigned_partitions()) + for partition, offset in six.iteritems(offsets): + # verify assignment is still active + if self._subscription.is_assigned(partition): + self._subscription.assignment[partition].committed = offset.offset + self._subscription.needs_fetch_committed_offsets = False + + def fetch_committed_offsets(self, partitions): + """ + Fetch the current committed offsets from the coordinator for a set of + partitions. + + @param partitions The partitions to fetch offsets for + @return dict of {TopicPartition: OffsetMetadata} + """ + while True: + self.ensure_coordinator_known() + + # contact coordinator to fetch committed offsets + future = self._send_offset_fetch_request(partitions) + self._client.poll(future=future) + + if future.succeeded(): + return future.value + + if not future.retriable(): + raise future.exception # pylint: disable-msg=raising-bad-type + + time.sleep(self._retry_backoff_ms / 1000.0) + + def ensure_partition_assignment(self): + """Ensure that we have a valid partition assignment from the coordinator.""" + if self._subscription.partitions_auto_assigned(): + self.ensure_active_group() + + def close(self): + try: + self._maybe_auto_commit_offsets_sync() + finally: + super(ConsumerCoordinator, self).close() + + def commit_offsets_async(self, offsets, callback=None): + """ + @param offsets: dict of {TopicPartition: OffsetAndMetadata} to commit + @param callback: called as callback(offsets, response), with response + as either an Exception or a OffsetCommitResponse + struct. This callback can be used to trigger custom + actions when a commit request completes. + @returns Future + """ + self._subscription.needs_fetch_committed_offsets = True + future = self._send_offset_commit_request(offsets) + cb = callback if callback else self._default_offset_commit_callback + future.add_both(cb, offsets) + + def commit_offsets_sync(self, offsets): + """ + Commit offsets synchronously. This method will retry until the commit + completes successfully or an unrecoverable error is encountered. + + @param offsets dict of {TopicPartition: OffsetAndMetadata} to commit + @raises TopicAuthorizationError if the consumer is not authorized to the + group or to any of the specified partitions + @raises CommitFailedError if an unrecoverable error occurs before the + commit can be completed + """ + if not offsets: + return + + while True: + self.ensure_coordinator_known() + + future = self._send_offset_commit_request(offsets) + self._client.poll(future=future) + + if future.succeeded(): + return + + if not future.retriable(): + raise future.exception # pylint: disable-msg=raising-bad-type + + time.sleep(self._retry_backoff_ms / 1000.0) + + def _maybe_auto_commit_offsets_sync(self): + if self._enable_auto_commit: + # disable periodic commits prior to committing synchronously. note that they will + # be re-enabled after a rebalance completes + self._auto_commit_task.disable() + + try: + self.commit_offsets_sync(self._subscription.all_consumed_offsets()) + except Exception: + # consistent with async auto-commit failures, we do not propagate the exception + log.exception("Auto offset commit failed") + + def _send_offset_commit_request(self, offsets): + """Commit offsets for the specified list of topics and partitions. + + This is a non-blocking call which returns a request future that can be + polled in the case of a synchronous commit or ignored in the + asynchronous case. + + @param offsets dict of {TopicPartition: OffsetAndMetadata} that should + be committed + @return Future indicating whether the commit was successful or not + """ + if self.coordinator_unknown(): + return Future().failure(Errors.GroupCoordinatorNotAvailableError) + + if not offsets: + return Future().failure(None) + + # create the offset commit request + offset_data = collections.defaultdict(dict) + for tp, offset in six.iteritems(offsets): + offset_data[tp.topic][tp.partition] = offset + + request = OffsetCommitRequest_v2( + self.group_id, + self.generation, + self.member_id, + OffsetCommitRequest_v2.DEFAULT_RETENTION_TIME, + [( + topic, [( + partition, + offset.offset, + offset.metadata + ) for partition, offset in six.iteritems(partitions)] + ) for topic, partitions in six.iteritems(offset_data)] + ) + + log.debug("Sending offset-commit request with %s to %s", + offsets, self.coordinator_id) + + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_offset_commit_response, offsets, future) + _f.add_errback(self._failed_request, future) + return future + + def _handle_offset_commit_response(self, offsets, future, response): + #self.sensors.commit_latency.record(response.requestLatencyMs()) + unauthorized_topics = set() + + for topic, partitions in response.topics: + for partition, error_code in partitions: + tp = TopicPartition(topic, partition) + offset = offsets[tp] + + error_type = Errors.for_code(error_code) + if error_type is Errors.NoError: + log.debug("Committed offset %s for partition %s", offset, tp) + if self._subscription.is_assigned(tp): + self._subscription.assignment[tp].committed = offset.offset + elif error_type is Errors.GroupAuthorizationFailedError: + log.error("Unauthorized to commit for group %s", self.group_id) + future.failure(error_type(self.group_id)) + return + elif error_type is Errors.TopicAuthorizationFailedError: + unauthorized_topics.add(topic) + elif error_type in (Errors.OffsetMetadataTooLargeError, + Errors.InvalidCommitOffsetSizeError): + # raise the error to the user + error = error_type() + log.info("Offset commit for group %s failed on partition" + " %s due to %s will retry", self.group_id, tp, error) + future.failure(error) + return + elif error_type is Errors.GroupLoadInProgressError: + # just retry + error = error_type(self.group_id) + log.info("Offset commit for group %s failed due to %s," + " will retry", self.group_id, error) + future.failure(error) + return + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError, + Errors.RequestTimedOutError): + error = error_type(self.group_id) + log.info("Offset commit for group %s failed due to %s," + " will find new coordinator and retry", + self.group_id, error) + self.coordinator_dead() + future.failure(error) + return + elif error_type in (Errors.UnknownMemberIdError, + Errors.IllegalGenerationError, + Errors.RebalanceInProgressError): + # need to re-join group + error = error_type(self.group_id) + log.error("Error %s occurred while committing offsets for" + " group %s", error, self.group_id) + self._subscription.mark_for_reassignment() + # Errors.CommitFailedError("Commit cannot be completed due to group rebalance")) + future.failure(error) + return + else: + error = error_type() + log.error("Unexpected error committing partition %s at" + " offset %s: %s", tp, offset, error) + future.failure(error) + return + + if unauthorized_topics: + log.error("Unauthorized to commit to topics %s", unauthorized_topics) + future.failure(Errors.TopicAuthorizationFailedError(unauthorized_topics)) + else: + future.success(True) + + def _send_offset_fetch_request(self, partitions): + """Fetch the committed offsets for a set of partitions. + + This is a non-blocking call. The returned future can be polled to get + the actual offsets returned from the broker. + + @param partitions list of TopicPartitions + @return Future of committed offsets dict: {TopicPartition: offset} + """ + if self.coordinator_unknown(): + return Future().failure(Errors.GroupCoordinatorNotAvailableError) + + log.debug("Fetching committed offsets for partitions: %s", partitions) + # construct the request + topic_partitions = collections.defaultdict(set) + for tp in partitions: + topic_partitions[tp.topic].add(tp.partition) + request = OffsetFetchRequest_v1( + self.group_id, + list(topic_partitions.items()) + ) + + # send the request with a callback + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_offset_fetch_response, future) + _f.add_errback(self._failed_request, future) + return future + + def _handle_offset_fetch_response(self, future, response): + offsets = {} + for topic, partitions in response.topics: + for partition, offset, metadata, error_code in partitions: + tp = TopicPartition(topic, partition) + error_type = Errors.for_code(error_code) + if error_type is not Errors.NoError: + error = error_type() + log.debug("Error fetching offset for %s: %s", tp, error_type()) + if error_type is Errors.GroupLoadInProgressError: + # just retry + future.failure(error) + elif error_type is Errors.NotCoordinatorForGroupError: + # re-discover the coordinator and retry + self.coordinator_dead() + future.failure(error) + elif error_type in (Errors.UnknownMemberIdError, + Errors.IllegalGenerationError): + # need to re-join group + self._subscription.mark_for_reassignment() + future.failure(error) + else: + log.error("Unknown error fetching offsets for %s: %s", + tp, error) + future.failure(error) + return + elif offset >= 0: + # record the position with the offset (-1 indicates no committed offset to fetch) + offsets[tp] = OffsetAndMetadata(offset, metadata) + else: + log.debug("No committed offset for partition %s", tp) + future.success(offsets) + + +class AutoCommitTask(object): + def __init__(self, coordinator, interval): + self._coordinator = coordinator + self._client = coordinator._client + self._interval = interval + self._enabled = False + self._request_in_flight = False + + def enable(self): + if self._enabled: + log.warning("AutoCommitTask is already enabled") + return + + self._enabled = True + if not self._request_in_flight: + self._client.schedule(self, time.time() + self._interval) + + def disable(self): + self._enabled = False + try: + self._client.unschedule(self) + except KeyError: + log.warning("AutoCommitTask was not previously scheduled") + + def _reschedule(self, at): + if self._enabled: + self._client.schedule(self, at) + else: + raise Errors.IllegalStateError('AutoCommitTask not enabled') + + def __call__(self): + if not self._enabled: + return + + if self._coordinator.coordinator_unknown(): + log.debug("Cannot auto-commit offsets because the coordinator is" + " unknown, will retry after backoff") + next_at = time.time() + self._coordinator._retry_backoff_ms / 1000.0 + self._client.schedule(self, next_at) + return + + self._request_in_flight = True + self._coordinator.commit_offsets_async( + self._coordinator._subscription.all_consumed_offsets(), + self._handle_commit_response) + + def _handle_commit_response(self, offsets, result): + self._request_in_flight = False + if result is True: + log.debug("Successfully auto-committed offsets") + next_at = time.time() + self._interval + elif not isinstance(result, BaseException): + raise Errors.IllegalStateError( + 'Unrecognized result in _handle_commit_response: %s' + % result) + elif hasattr(result, 'retriable') and result.retriable: + log.debug("Failed to auto-commit offsets: %s, will retry" + " immediately", result) + next_at = time.time() + else: + log.warning("Auto offset commit failed: %s", result) + next_at = time.time() + self._interval + + if not self._enabled: + log.warning("Skipping auto-commit reschedule -- it is disabled") + return + self._reschedule(next_at) + + +# TODO +""" +class ConsumerCoordinatorMetrics(object): + def __init__(self, metrics, prefix, tags): + self.metrics = metrics + self.group_name = prefix + "-coordinator-metrics" + + self.commit_latency = metrics.sensor("commit-latency") + self.commit_latency.add(metrics.MetricName( + "commit-latency-avg", self.group_name, + "The average time taken for a commit request", + tags), metrics.Avg()) + self.commit_latency.add(metrics.MetricName( + "commit-latency-max", self.group_name, + "The max time taken for a commit request", + tags), metrics.Max()) + self.commit_latency.add(metrics.MetricName( + "commit-rate", self.group_name, + "The number of commit calls per second", + tags), metrics.Rate(metrics.Count())) + + ''' + def _num_partitions(config, now): + new Measurable() { + public double measure(MetricConfig config, long now) { + return subscriptions.assignedPartitions().size(); + } + }; + metrics.addMetric(new MetricName("assigned-partitions", + this.metricGrpName, + "The number of partitions currently assigned to this consumer", + tags), + numParts); + ''' +""" -- cgit v1.2.1 From 70d2f2630da37ccdf616e28b2bfa8e6c2562960b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Dec 2015 23:10:00 -0800 Subject: kafka.consumer.group.KafkaConsumer: 0.9 coordinator-aware consumer - use ConsumerCoordinator to manage group membership - dynamically assign partitions via group using topic subscription - use upstream configuration parameters as __init__ kwargs - metrics and SSL are still TODO --- kafka/consumer/group.py | 1277 ++++++++++++++++++----------------------------- 1 file changed, 480 insertions(+), 797 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index dba5f60..abd9473 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -1,883 +1,566 @@ -#pylint: skip-file from __future__ import absolute_import -from collections import namedtuple -from copy import deepcopy import logging -import random -import sys import time -import six - -from kafka.cluster import Cluster -from kafka.common import ( - OffsetFetchRequest, OffsetCommitRequest, OffsetRequest, FetchRequest, - check_error, NotLeaderForPartitionError, UnknownTopicOrPartitionError, - OffsetOutOfRangeError, RequestTimedOutError, KafkaMessage, ConsumerTimeout, - FailedPayloadsError, KafkaUnavailableError, KafkaConfigurationError -) - -logger = logging.getLogger(__name__) - -OffsetsStruct = namedtuple("OffsetsStruct", ["fetch", "highwater", "commit", "task_done"]) - -NEW_CONSUMER_CONFIGS = { - 'bootstrap_servers': None, - 'client_id': None, - 'group_id': None, - 'key_deserializer': None, - 'value_deserializer': None, - 'auto_commit_interval_ms': 5000, - 'auto_offset_reset': 'latest', - 'check_crcs': True, # "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; - 'connections_max_idle_ms': 9 * 60 * 1000, - 'enable_auto_commit': True, - 'fetch_max_wait_ms': 500, - 'fetch_min_bytes': 1024, - 'heartbeat_interval_ms': 3000, - 'max_partition_fetch_bytes': 1 * 1024 * 1024, - 'metadata_max_age_ms': 5 * 60 * 1000, # >0 - 'metric_reporters': None, - 'metrics_num_samples': 2, - 'metrics_sample_window_ms': 30000, - 'partition_assignment_strategy': None, # This should default to something like 'roundrobin' or 'range' - 'reconnect_backoff_ms': 50, - 'request_timeout_ms': 40 * 1000, - 'retry_backoff_ms': 100, - 'send_buffer_bytes': 128 * 1024, - 'receive_buffer_bytes': 32 * 1024, - 'session_timeout_ms': 30000, # "The timeout used to detect failures when using Kafka's group management facilities."; -} - -DEFAULT_CONSUMER_CONFIG = { - 'client_id': __name__, - 'group_id': None, - 'bootstrap_servers': [], - 'socket_timeout_ms': 30 * 1000, - 'fetch_message_max_bytes': 1024 * 1024, - 'auto_offset_reset': 'largest', - 'fetch_min_bytes': 1, - 'fetch_wait_max_ms': 100, - 'refresh_leader_backoff_ms': 200, - 'deserializer_class': lambda msg: msg, - 'auto_commit_enable': False, - 'auto_commit_interval_ms': 60 * 1000, - 'auto_commit_interval_messages': None, - 'consumer_timeout_ms': -1, - - # Currently unused - 'socket_receive_buffer_bytes': 64 * 1024, - 'num_consumer_fetchers': 1, - 'default_fetcher_backoff_ms': 1000, - 'queued_max_message_chunks': 10, - 'rebalance_max_retries': 4, - 'rebalance_backoff_ms': 2000, -} - -DEPRECATED_CONFIG_KEYS = { - 'metadata_broker_list': 'bootstrap_servers', -} +import kafka.common as Errors -class KafkaConsumer(object): - """A simpler kafka consumer""" - - def __init__(self, *topics, **configs): - self._config = deepcopy(DEFAULT_CONSUMER_CONFIG) - self._topics = topics - self._partitions = [] - self._offsets = OffsetsStruct(fetch=dict(), commit=dict(), highwater=dict(), task_done=dict()) - self._consumer_timeout = False - self._uncommitted_message_count = 0 - self._next_commit_time = None - self._msg_iter = None - - self._configure(**configs) - self._cluster = Cluster(**self._config) - - def assign(self, topic_partitions): - pass +from kafka.client_async import KafkaClient +from kafka.consumer.fetcher import Fetcher +from kafka.consumer.subscription_state import SubscriptionState +from kafka.coordinator.consumer import ConsumerCoordinator +from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor +from kafka.protocol.offset import OffsetResetStrategy +from kafka.version import __version__ - def assignment(self): - """Get the set of partitions currently assigned to this consumer.""" - pass +log = logging.getLogger(__name__) - def close(self): - """Close the consumer, waiting indefinitely for any needed cleanup.""" - pass - def commitAsync(self, topic_partition_offsets_and_metadata=None, callback=None): - """ - Commit offsets the specified offsets, or those returned on the last poll(), - for all the subscribed list of topics and partition. Asynchronous. - """ - pass +class KafkaConsumer(object): + """Consumer for Kafka 0.9""" + _bootstrap_servers = 'localhost' + _client_id = 'kafka-python-' + __version__ + _group_id = 'kafka-python-default-group' + _key_deserializer = None + _value_deserializer = None + _fetch_max_wait_ms = 500 + _fetch_min_bytes = 1024 + _max_partition_fetch_bytes = 1 * 1024 * 1024 + _request_timeout_ms = 40 * 1000 + _retry_backoff_ms = 100 + _reconnect_backoff_ms = 50 + _auto_offset_reset = 'latest' + _enable_auto_commit = True + _auto_commit_interval_ms = 5000 + _check_crcs = True + _metadata_max_age_ms = 5 * 60 * 1000 + _partition_assignment_strategy = (RoundRobinPartitionAssignor,) + _heartbeat_interval_ms = 3000 + _session_timeout_ms = 30000 + _send_buffer_bytes = 128 * 1024 + _receive_buffer_bytes = 32 * 1024 + _connections_max_idle_ms = 9 * 60 * 1000 # not implemented yet + #_metric_reporters = None + #_metrics_num_samples = 2 + #_metrics_sample_window_ms = 30000 + + def __init__(self, *topics, **kwargs): + """A Kafka client that consumes records from a Kafka cluster. + + The consumer will transparently handle the failure of servers in the + Kafka cluster, and transparently adapt as partitions of data it fetches + migrate within the cluster. This client also interacts with the server + to allow groups of consumers to load balance consumption using consumer + groups. + + Requires Kafka Server >= 0.9.0.0 + + Configuration settings can be passed to constructor as kwargs, + otherwise defaults will be used: - def commitSync(self, topic_partition_offsets_and_metadata=None): - """ - Commit offsets the specified offsets, or those returned on the last poll(), - for all the subscribed list of topics and partition. Synchronous. - Blocks until either the commit succeeds or an unrecoverable error is - encountered (in which case it is thrown to the caller). - """ - pass + Keyword Arguments: + bootstrap_servers: 'host[:port]' string (or list of 'host[:port]' + strings) that the consumer should contact to bootstrap initial + cluster metadata. This does not have to be the full node list. + It just needs to have at least one broker that will respond to a + Metadata API Request. Default port is 9092. If no servers are + specified, will default to localhost:9092. + client_id (str): a name for this client. This string is passed in + each request to servers and can be used to identify specific + server-side log entries that correspond to this client. Also + submitted to GroupCoordinator for logging with respect to + consumer group administration. Default: 'kafka-python-{version}' + group_id (str): name of the consumer group to join for dynamic + partition assignment (if enabled), and to use for fetching and + committing offsets. Default: 'kafka-python-default-group' + key_deserializer (callable): Any callable that takes a + raw message key and returns a deserialized key. + value_deserializer (callable, optional): Any callable that takes a + raw message value and returns a deserialized value. + fetch_min_bytes (int): Minimum amount of data the server should + return for a fetch request, otherwise wait up to + fetch_wait_max_ms for more data to accumulate. Default: 1024. + fetch_wait_max_ms (int): The maximum amount of time in milliseconds + the server will block before answering the fetch request if + there isn't sufficient data to immediately satisfy the + requirement given by fetch_min_bytes. Default: 500. + max_partition_fetch_bytes (int): The maximum amount of data + per-partition the server will return. The maximum total memory + used for a request = #partitions * max_partition_fetch_bytes. + This size must be at least as large as the maximum message size + the server allows or else it is possible for the producer to + send messages larger than the consumer can fetch. If that + happens, the consumer can get stuck trying to fetch a large + message on a certain partition. Default: 1048576. + request_timeout_ms (int): Client request timeout in milliseconds. + Default: 40000. + retry_backoff_ms (int): Milliseconds to backoff when retrying on + errors. Default: 100. + reconnect_backoff_ms (int): The amount of time in milliseconds to + wait before attempting to reconnect to a given host. Defaults + to 50. + auto_offset_reset (str): A policy for resetting offsets on + OffsetOutOfRange errors: 'earliest' will move to the oldest + available message, 'latest' will move to the most recent. Any + ofther value will raise the exception. Default: 'latest'. + enable_auto_commit (bool): If true the consumer's offset will be + periodically committed in the background. Default: True. + auto_commit_interval_ms (int): milliseconds between automatic + offset commits, if enable_auto_commit is True. Default: 5000. + check_crcs (bool): Automatically check the CRC32 of the records + consumed. This ensures no on-the-wire or on-disk corruption to + the messages occurred. This check adds some overhead, so it may + be disabled in cases seeking extreme performance. Default: True + metadata_max_age_ms (int): The period of time in milliseconds after + which we force a refresh of metadata even if we haven't seen any + partition leadership changes to proactively discover any new + brokers or partitions. Default: 300000 + partition_assignment_strategy (list): List of objects to use to + distribute partition ownership amongst consumer instances when + group management is used. Default: [RoundRobinPartitionAssignor] + heartbeat_interval_ms (int): The expected time in milliseconds + between heartbeats to the consumer coordinator when using + Kafka's group management feature. Heartbeats are used to ensure + that the consumer's session stays active and to facilitate + rebalancing when new consumers join or leave the group. The + value must be set lower than session_timeout_ms, but typically + should be set no higher than 1/3 of that value. It can be + adjusted even lower to control the expected time for normal + rebalances. Default: 3000 + session_timeout_ms (int): The timeout used to detect failures when + using Kafka's group managementment facilities. Default: 30000 + send_buffer_bytes (int): The size of the TCP send buffer + (SO_SNDBUF) to use when sending data. Default: 131072 + receive_buffer_bytes (int): The size of the TCP receive buffer + (SO_RCVBUF) to use when reading data. Default: 32768 - def committed(self, topic_partition): - """ - Get the last committed offset for the given partition (whether the - commit happened by this process or another). - Returns: offset_and_metadata + Configuration parameters are described in more detail at + https://kafka.apache.org/090/configuration.html#newconsumerconfigs """ - pass + for config in ('bootstrap_servers', 'client_id', 'group_id', + 'key_deserializer', 'value_deserializer', + 'fetch_max_wait_ms', 'fetch_min_bytes', + 'max_partition_fetch_bytes', 'request_timeout_ms', + 'retry_backoff_ms', 'reconnect_backoff_ms', + 'auto_offset_reset', 'enable_auto_commit', + 'auto_commit_interval_ms', 'check_crcs', + 'metadata_max_age_ms', 'partition_assignment_strategy', + 'heartbeat_interval_ms', 'session_timeout_ms', + 'send_buffer_bytes', 'receive_buffer_bytes'): + if config in kwargs: + setattr(self, '_' + config, kwargs[config]) + + self._client = KafkaClient(**kwargs) + self._subscription = SubscriptionState(self._auto_offset_reset) + self._fetcher = Fetcher( + self._client, self._subscription, **kwargs) + self._coordinator = ConsumerCoordinator( + self._client, self._group_id, self._subscription, + assignors=self._partition_assignment_strategy, + **kwargs) + self._closed = False + + #self.metrics = None + if topics: + self._subscription.subscribe(topics=topics) + self._client.set_topics(topics) + + def assign(self, partitions): + """Manually assign a list of TopicPartitions to this consumer. + + This interface does not allow for incremental assignment and will + replace the previous assignment (if there was one). + + Manual topic assignment through this method does not use the consumer's + group management functionality. As such, there will be no rebalance + operation triggered when group membership or cluster and topic metadata + change. Note that it is not possible to use both manual partition + assignment with assign() and group assignment with subscribe(). - def listTopics(self): - """ - Get metadata about partitions for all topics that the user is authorized - to view. - Returns: {topic: [partition_info]} - """ - pass + Arguments: + partitions (list of TopicPartition): assignment for this instance. - def metrics(self): - """ - Get the metrics kept by the consumer. - Returns: {metric_name: metric} + Raises: + IllegalStateError: if consumer has already called subscribe() """ - pass + self._subscription.assign_from_user(partitions) + self._client.set_topics([tp.topic for tp in partitions]) - def partitionsFor(self, topic): - """ - Get metadata about the partitions for a given topic. - Returns: [partition_info] - """ - pass + def assignment(self): + """Get the TopicPartitions currently assigned to this consumer. - def pause(self, *topic_partitions): - """Suspend fetching from the requested partitions.""" - pass + If partitions were directly assigning using assign(), then this will + simply return the same partitions that were assigned. + If topics were subscribed to using subscribe(), then this will give the + set of topic partitions currently assigned to the consumer (which may + be none if the assignment hasn't happened yet, or the partitions are in + the process of getting reassigned). - def poll(self, timeout): - """ - Fetch data for the topics or partitions specified using one of the - subscribe/assign APIs. - Returns: [consumer_records] + Returns: + set: {TopicPartition, ...} """ - pass + return self._subscription.assigned_partitions() - def position(self, topic_partition): - """Get the offset of the next record that will be fetched (if a record - with that offset exists).""" - pass - - def resume(self, *topic_partitions): - """Resume specified partitions which have been paused""" - pass - - def seek(self, topic_partition, offset): - """Overrides the fetch offsets that the consumer will use on the next - poll(timeout).""" - pass - - def seekToBeginning(self, *topic_partitions): - """Seek to the first offset for each of the given partitions.""" - pass - - def seekToEnd(self, *topic_partitions): - """Seek to the last offset for each of the given partitions.""" - pass + def close(self): + """Close the consumer, waiting indefinitely for any needed cleanup.""" + if self._closed: + return + log.debug("Closing the KafkaConsumer.") + self._closed = True + self._coordinator.close() + #self.metrics.close() + self._client.close() + try: + self._key_deserializer.close() + except AttributeError: + pass + try: + self._value_deserializer.close() + except AttributeError: + pass + log.debug("The KafkaConsumer has closed.") + + def commit_async(self, offsets=None, callback=None): + """Commit offsets to kafka asynchronously, optionally firing callback + + This commits offsets only to Kafka. The offsets committed using this API + will be used on the first fetch after every rebalance and also on + startup. As such, if you need to store offsets in anything other than + Kafka, this API should not be used. + + This is an asynchronous call and will not block. Any errors encountered + are either passed to the callback (if provided) or discarded. - def subscribe(self, topics, callback=None): - """Subscribe to the given list of topics or those matching a regex to get dynamically assigned - partitions.""" - pass + Arguments: + offsets (dict, optional): {TopicPartition: OffsetAndMetadata} dict + to commit with the configured group_id. Defaults to current + consumed offsets for all subscribed partitions. + callback (callable, optional): called as callback(offsets, response) + with response as either an Exception or a OffsetCommitResponse + struct. This callback can be used to trigger custom actions when + a commit request completes. - def subscription(self): - """ - Get the current subscription. - Returns: [topic] + Returns: + kafka.future.Future """ - pass - - def unsubscribe(self): - """Unsubscribe from topics currently subscribed with subscribe(List).""" - pass + if offsets is None: + offsets = self._subscription.all_consumed_offsets() + log.debug("Committing offsets: %s", offsets) + future = self._coordinator.commit_offsets_async( + offsets, callback=callback) + return future - def wakeup(self): - """Wakeup the consumer.""" - pass + def commit(self, offsets=None): + """Commit offsets to kafka, blocking until success or error - def _configure(self, **configs): - """Configure the consumer instance + This commits offsets only to Kafka. The offsets committed using this API + will be used on the first fetch after every rebalance and also on + startup. As such, if you need to store offsets in anything other than + Kafka, this API should not be used. - Configuration settings can be passed to constructor, - otherwise defaults will be used: - - Keyword Arguments: - bootstrap_servers (list): List of initial broker nodes the consumer - should contact to bootstrap initial cluster metadata. This does - not have to be the full node list. It just needs to have at - least one broker that will respond to a Metadata API Request. - client_id (str): a unique name for this client. Defaults to - 'kafka.consumer.kafka'. - group_id (str): the name of the consumer group to join, - Offsets are fetched / committed to this group name. - fetch_message_max_bytes (int, optional): Maximum bytes for each - topic/partition fetch request. Defaults to 1024*1024. - fetch_min_bytes (int, optional): Minimum amount of data the server - should return for a fetch request, otherwise wait up to - fetch_wait_max_ms for more data to accumulate. Defaults to 1. - fetch_wait_max_ms (int, optional): Maximum time for the server to - block waiting for fetch_min_bytes messages to accumulate. - Defaults to 100. - refresh_leader_backoff_ms (int, optional): Milliseconds to backoff - when refreshing metadata on errors (subject to random jitter). - Defaults to 200. - socket_timeout_ms (int, optional): TCP socket timeout in - milliseconds. Defaults to 30*1000. - auto_offset_reset (str, optional): A policy for resetting offsets on - OffsetOutOfRange errors. 'smallest' will move to the oldest - available message, 'largest' will move to the most recent. Any - ofther value will raise the exception. Defaults to 'largest'. - deserializer_class (callable, optional): Any callable that takes a - raw message value and returns a deserialized value. Defaults to - lambda msg: msg. - auto_commit_enable (bool, optional): Enabling auto-commit will cause - the KafkaConsumer to periodically commit offsets without an - explicit call to commit(). Defaults to False. - auto_commit_interval_ms (int, optional): If auto_commit_enabled, - the milliseconds between automatic offset commits. Defaults to - 60 * 1000. - auto_commit_interval_messages (int, optional): If - auto_commit_enabled, a number of messages consumed between - automatic offset commits. Defaults to None (disabled). - consumer_timeout_ms (int, optional): number of millisecond to throw - a timeout exception to the consumer if no message is available - for consumption. Defaults to -1 (dont throw exception). - - Configuration parameters are described in more detail at - http://kafka.apache.org/documentation.html#highlevelconsumerapi - """ - configs = self._deprecate_configs(**configs) - self._config.update(configs) + Blocks until either the commit succeeds or an unrecoverable error is + encountered (in which case it is thrown to the caller). - if self._config['auto_commit_enable']: - logger.info('Configuring consumer to auto-commit offsets') - self._reset_auto_commit() + Currently only supports kafka-topic offset storage (not zookeeper) - def set_topic_partitions(self, *topics): + Arguments: + offsets (dict, optional): {TopicPartition: OffsetAndMetadata} dict + to commit with the configured group_id. Defaults to current + consumed offsets for all subscribed partitions. """ - Set the topic/partitions to consume - Optionally specify offsets to start from - - Accepts types: + if offsets is None: + offsets = self._subscription.all_consumed_offsets() + self._coordinator.commit_offsets_sync(offsets) - * str (utf-8): topic name (will consume all available partitions) - * tuple: (topic, partition) - * dict: - - { topic: partition } - - { topic: [partition list] } - - { topic: (partition tuple,) } + def committed(self, partition): + """Get the last committed offset for the given partition - Optionally, offsets can be specified directly: + This offset will be used as the position for the consumer + in the event of a failure. - * tuple: (topic, partition, offset) - * dict: { (topic, partition): offset, ... } + This call may block to do a remote call if the partition in question + isn't assigned to this consumer or if the consumer hasn't yet + initialized its cache of committed offsets. - Example: - - .. code:: python - - kafka = KafkaConsumer() - - # Consume topic1-all; topic2-partition2; topic3-partition0 - kafka.set_topic_partitions("topic1", ("topic2", 2), {"topic3": 0}) - - # Consume topic1-0 starting at offset 12, and topic2-1 at offset 45 - # using tuples -- - kafka.set_topic_partitions(("topic1", 0, 12), ("topic2", 1, 45)) - - # using dict -- - kafka.set_topic_partitions({ ("topic1", 0): 12, ("topic2", 1): 45 }) + Arguments: + partition (TopicPartition): the partition to check + Returns: + The last committed offset, or None if there was no prior commit. """ - self._cluster.refresh_metadata() - - # Handle different topic types - for arg in topics: - - # Topic name str -- all partitions - if isinstance(arg, (six.string_types, six.binary_type)): - topic = arg - for partition in self._cluster.partitions_for_topic(topic): - self._consume_topic_partition(topic, partition) - - # (topic, partition [, offset]) tuple - elif isinstance(arg, tuple): - topic = arg[0] - partition = arg[1] - self._consume_topic_partition(topic, partition) - if len(arg) == 3: - offset = arg[2] - self._offsets.fetch[(topic, partition)] = offset - - # { topic: partitions, ... } dict - elif isinstance(arg, dict): - for key, value in six.iteritems(arg): - - # key can be string (a topic) - if isinstance(key, (six.string_types, six.binary_type)): - topic = key - - # topic: partition - if isinstance(value, int): - self._consume_topic_partition(topic, value) - - # topic: [ partition1, partition2, ... ] - elif isinstance(value, (list, tuple)): - for partition in value: - self._consume_topic_partition(topic, partition) - else: - raise KafkaConfigurationError( - 'Unknown topic type ' - '(dict key must be int or list/tuple of ints)' - ) - - # (topic, partition): offset - elif isinstance(key, tuple): - topic = key[0] - partition = key[1] - self._consume_topic_partition(topic, partition) - self._offsets.fetch[(topic, partition)] = value - + if self._subscription.is_assigned: + committed = self._subscription.assignment[partition].committed + if committed is None: + self._coordinator.refresh_committed_offsets_if_needed() + committed = self._subscription.assignment[partition].committed + else: + commit_map = self._coordinator.fetch_committed_offsets([partition]) + if partition in commit_map: + committed = commit_map[partition].offset else: - raise KafkaConfigurationError('Unknown topic type (%s)' % type(arg)) - - # If we have a consumer group, try to fetch stored offsets - if self._config['group_id']: - self._get_commit_offsets() - - # Update missing fetch/commit offsets - for topic_partition in self._topics: - - # Commit offsets default is None - if topic_partition not in self._offsets.commit: - self._offsets.commit[topic_partition] = None - - # Skip if we already have a fetch offset from user args - if topic_partition not in self._offsets.fetch: - - # Fetch offsets default is (1) commit - if self._offsets.commit[topic_partition] is not None: - self._offsets.fetch[topic_partition] = self._offsets.commit[topic_partition] - - # or (2) auto reset - else: - self._offsets.fetch[topic_partition] = self._reset_partition_offset(topic_partition) + committed = None + return committed - # highwater marks (received from server on fetch response) - # and task_done (set locally by user) - # should always get initialized to None - self._reset_highwater_offsets() - self._reset_task_done_offsets() + def _ensure_not_closed(self): + if self._closed: + raise Errors.IllegalStateError("This consumer has already been closed.") - # Reset message iterator in case we were in the middle of one - self._reset_message_iterator() + def topics(self): + """Get all topic metadata topics the user is authorized to view. - def next(self): - """Return the next available message - - Blocks indefinitely unless consumer_timeout_ms > 0 + [Not Implemented Yet] Returns: - a single KafkaMessage from the message iterator + {topic: [partition_info]} + """ + raise NotImplementedError('TODO') - Raises: - ConsumerTimeout after consumer_timeout_ms and no message + def partitions_for_topic(self, topic): + """Get metadata about the partitions for a given topic. - Note: - This is also the method called internally during iteration + Arguments: + topic (str): topic to check + Returns: + set: partition ids """ - self._set_consumer_timeout_start() - while True: - - try: - return six.next(self._get_message_iterator()) + return self._client.cluster.partitions_for_topic(topic) - # Handle batch completion - except StopIteration: - self._reset_message_iterator() + def poll(self, timeout_ms=0): + """ + Fetch data for the topics or partitions specified using one of the + subscribe/assign APIs. It is an error to not have subscribed to any + topics or partitions before polling for data. - self._check_consumer_timeout() + On each poll, consumer will try to use the last consumed offset as the + starting offset and fetch sequentially. The last consumed offset can be + manually set through seek(partition, offset) or automatically set as + the last committed offset for the subscribed list of partitions. - def fetch_messages(self): - """Sends FetchRequests for all topic/partitions set for consumption + Arguments: + timeout_ms (int, optional): milliseconds to spend waiting in poll if + data is not available. If 0, returns immediately with any + records that are available now. Must not be negative. Default: 0 Returns: - Generator that yields KafkaMessage structs - after deserializing with the configured `deserializer_class` - - Note: - Refreshes metadata on errors, and resets fetch offset on - OffsetOutOfRange, per the configured `auto_offset_reset` policy - - See Also: - Key KafkaConsumer configuration parameters: - * `fetch_message_max_bytes` - * `fetch_max_wait_ms` - * `fetch_min_bytes` - * `deserializer_class` - * `auto_offset_reset` - + dict: topic to deque of records since the last fetch for the + subscribed list of topics and partitions """ + if timeout_ms < 0: + raise Errors.IllegalArgumentError("Timeout must not be negative") - max_bytes = self._config['fetch_message_max_bytes'] - max_wait_time = self._config['fetch_wait_max_ms'] - min_bytes = self._config['fetch_min_bytes'] - - if not self._topics: - raise KafkaConfigurationError('No topics or partitions configured') - - if not self._offsets.fetch: - raise KafkaConfigurationError( - 'No fetch offsets found when calling fetch_messages' - ) - - fetches = [FetchRequest(topic, partition, - self._offsets.fetch[(topic, partition)], - max_bytes) - for (topic, partition) in self._topics] - - # send_fetch_request will batch topic/partition requests by leader - responses = self._client.send_fetch_request( - fetches, - max_wait_time=max_wait_time, - min_bytes=min_bytes, - fail_on_error=False - ) - - for resp in responses: - - if isinstance(resp, FailedPayloadsError): - logger.warning('FailedPayloadsError attempting to fetch data') - self._refresh_metadata_on_error() - continue - - topic = resp.topic - partition = resp.partition - try: - check_error(resp) - except OffsetOutOfRangeError: - logger.warning('OffsetOutOfRange: topic %s, partition %d, ' - 'offset %d (Highwatermark: %d)', - topic, partition, - self._offsets.fetch[(topic, partition)], - resp.highwaterMark) - # Reset offset - self._offsets.fetch[(topic, partition)] = ( - self._reset_partition_offset((topic, partition)) - ) - continue - - except NotLeaderForPartitionError: - logger.warning("NotLeaderForPartitionError for %s - %d. " - "Metadata may be out of date", - topic, partition) - self._refresh_metadata_on_error() - continue - - except RequestTimedOutError: - logger.warning("RequestTimedOutError for %s - %d", - topic, partition) - continue - - # Track server highwater mark - self._offsets.highwater[(topic, partition)] = resp.highwaterMark - - # Yield each message - # Kafka-python could raise an exception during iteration - # we are not catching -- user will need to address - for (offset, message) in resp.messages: - # deserializer_class could raise an exception here - val = self._config['deserializer_class'](message.value) - msg = KafkaMessage(topic, partition, offset, message.key, val) - - # in some cases the server will return earlier messages - # than we requested. skip them per kafka spec - if offset < self._offsets.fetch[(topic, partition)]: - logger.debug('message offset less than fetched offset ' - 'skipping: %s', msg) - continue - # Only increment fetch offset - # if we safely got the message and deserialized - self._offsets.fetch[(topic, partition)] = offset + 1 - - # Then yield to user - yield msg - - def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): - """Request available fetch offsets for a single topic/partition + # poll for new data until the timeout expires + start = time.time() + remaining = timeout_ms + while True: + records = self._poll_once(remaining) + if records: + # before returning the fetched records, we can send off the + # next round of fetches and avoid block waiting for their + # responses to enable pipelining while the user is handling the + # fetched records. + self._fetcher.init_fetches() + return records + + elapsed_ms = (time.time() - start) * 1000 + remaining = timeout_ms - elapsed_ms + + if remaining <= 0: + break + + def _poll_once(self, timeout_ms): + """ + Do one round of polling. In addition to checking for new data, this does + any needed heart-beating, auto-commits, and offset updates. - Keyword Arguments: - topic (str): topic for offset request - partition (int): partition for offset request - request_time_ms (int): Used to ask for all messages before a - certain time (ms). There are two special values. - Specify -1 to receive the latest offset (i.e. the offset of the - next coming message) and -2 to receive the earliest available - offset. Note that because offsets are pulled in descending - order, asking for the earliest offset will always return you a - single element. - max_num_offsets (int): Maximum offsets to include in the OffsetResponse + Arguments: + timeout_ms (int): The maximum time in milliseconds to block Returns: - a list of offsets in the OffsetResponse submitted for the provided - topic / partition. See: - https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI + dict: map of topic to deque of records (may be empty) """ - reqs = [OffsetRequest(topic, partition, request_time_ms, max_num_offsets)] + # TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) + self._coordinator.ensure_coordinator_known() - (resp,) = self._client.send_offset_request(reqs) + # ensure we have partitions assigned if we expect to + if self._subscription.partitions_auto_assigned(): + self._coordinator.ensure_active_group() - check_error(resp) + # fetch positions if we have partitions we're subscribed to that we + # don't know the offset for + if not self._subscription.has_all_fetch_positions(): + self._update_fetch_positions(self._subscription.missing_fetch_positions()) - # Just for sanity.. - # probably unnecessary - assert resp.topic == topic - assert resp.partition == partition + # init any new fetches (won't resend pending fetches) + records = self._fetcher.fetched_records() - return resp.offsets + # if data is available already, e.g. from a previous network client + # poll() call to commit, then just return it immediately + if records: + return records - def offsets(self, group=None): - """Get internal consumer offset values + self._fetcher.init_fetches() + self._client.poll(timeout_ms / 1000.0) + return self._fetcher.fetched_records() - Keyword Arguments: - group: Either "fetch", "commit", "task_done", or "highwater". - If no group specified, returns all groups. + def position(self, partition): + """Get the offset of the next record that will be fetched - Returns: - A copy of internal offsets struct + Arguments: + partition (TopicPartition): partition to check """ - if not group: - return { - 'fetch': self.offsets('fetch'), - 'commit': self.offsets('commit'), - 'task_done': self.offsets('task_done'), - 'highwater': self.offsets('highwater') - } - else: - return dict(deepcopy(getattr(self._offsets, group))) + if not self._subscription.is_assigned(partition): + raise Errors.IllegalStateError("You can only check the position for partitions assigned to this consumer.") + offset = self._subscription.assignment[partition].consumed + if offset is None: + self._update_fetch_positions(partition) + offset = self._subscription.assignment[partition].consumed + return offset - def task_done(self, message): - """Mark a fetched message as consumed. + def pause(self, *partitions): + """Suspend fetching from the requested partitions. - Offsets for messages marked as "task_done" will be stored back - to the kafka cluster for this consumer group on commit() + Future calls to poll() will not return any records from these partitions + until they have been resumed using resume(). Note that this method does + not affect partition subscription. In particular, it does not cause a + group rebalance when automatic assignment is used. Arguments: - message (KafkaMessage): the message to mark as complete - - Returns: - True, unless the topic-partition for this message has not - been configured for the consumer. In normal operation, this - should not happen. But see github issue 364. + *partitions (TopicPartition): partitions to pause """ - topic_partition = (message.topic, message.partition) - if topic_partition not in self._topics: - logger.warning('Unrecognized topic/partition in task_done message: ' - '{0}:{1}'.format(*topic_partition)) - return False + for partition in partitions: + log.debug("Pausing partition %s", partition) + self._subscription.pause(partition) - offset = message.offset + def resume(self, *partitions): + """Resume fetching from the specified (paused) partitions. - # Warn on non-contiguous offsets - prev_done = self._offsets.task_done[topic_partition] - if prev_done is not None and offset != (prev_done + 1): - logger.warning('Marking task_done on a non-continuous offset: %d != %d + 1', - offset, prev_done) + Arguments: + *partitions (TopicPartition): partitions to resume + """ + for partition in partitions: + log.debug("Resuming partition %s", partition) + self._subscription.resume(partition) + + def seek(self, partition, offset): + """Manually specify the fetch offset for a TopicPartition + + Overrides the fetch offsets that the consumer will use on the next + poll(). If this API is invoked for the same partition more than once, + the latest offset will be used on the next poll(). Note that you may + lose data if this API is arbitrarily used in the middle of consumption, + to reset the fetch offsets. + """ + if offset < 0: + raise Errors.IllegalStateError("seek offset must not be a negative number") + log.debug("Seeking to offset %s for partition %s", offset, partition) + self._subscription.assignment[partition].seek(offset) - # Warn on smaller offsets than previous commit - # "commit" offsets are actually the offset of the next message to fetch. - prev_commit = self._offsets.commit[topic_partition] - if prev_commit is not None and ((offset + 1) <= prev_commit): - logger.warning('Marking task_done on a previously committed offset?: %d (+1) <= %d', - offset, prev_commit) + def seek_to_beginning(self, *partitions): + """Seek to the oldest available offset for partitions. - self._offsets.task_done[topic_partition] = offset + Arguments: + *partitions: optionally provide specific TopicPartitions, otherwise + default to all assigned partitions + """ + if not partitions: + partitions = self._subscription.assigned_partitions() + for tp in partitions: + log.debug("Seeking to beginning of partition %s", tp) + self._subscription.need_offset_reset(tp, OffsetResetStrategy.EARLIEST) - # Check for auto-commit - if self._does_auto_commit_messages(): - self._incr_auto_commit_message_count() + def seek_to_end(self, *partitions): + """Seek to the most recent available offset for partitions. - if self._should_auto_commit(): - self.commit() + Arguments: + *partitions: optionally provide specific TopicPartitions, otherwise + default to all assigned partitions + """ + if not partitions: + partitions = self._subscription.assigned_partitions() + for tp in partitions: + log.debug("Seeking to end of partition %s", tp) + self._subscription.need_offset_reset(tp, OffsetResetStrategy.LATEST) - return True + def subscribe(self, topics=(), pattern=None, listener=None): + """Subscribe to a list of topics, or a topic regex pattern - def commit(self): - """Store consumed message offsets (marked via task_done()) - to kafka cluster for this consumer_group. + Partitions will be dynamically assigned via a group coordinator. + Topic subscriptions are not incremental: this list will replace the + current assignment (if there is one). - Returns: - True on success, or False if no offsets were found for commit + This method is incompatible with assign() - Note: - this functionality requires server version >=0.8.1.1 - https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + Arguments: + topics (list): List of topics for subscription. + pattern (str): Pattern to match available topics. You must provide + either topics or pattern, but not both. + listener (ConsumerRebalanceListener): Optionally include listener + callback, which will be called before and after each rebalance + operation. + + As part of group management, the consumer will keep track of the + list of consumers that belong to a particular group and will + trigger a rebalance operation if one of the following events + trigger: + + * Number of partitions change for any of the subscribed topics + * Topic is created or deleted + * An existing member of the consumer group dies + * A new member is added to the consumer group + + When any of these events are triggered, the provided listener + will be invoked first to indicate that the consumer's assignment + has been revoked, and then again when the new assignment has + been received. Note that this listener will immediately override + any listener set in a previous call to subscribe. It is + guaranteed, however, that the partitions revoked/assigned + through this interface are from topics subscribed in this call. """ - if not self._config['group_id']: - logger.warning('Cannot commit without a group_id!') - raise KafkaConfigurationError( - 'Attempted to commit offsets ' - 'without a configured consumer group (group_id)' - ) - - # API supports storing metadata with each commit - # but for now it is unused - metadata = b'' - - offsets = self._offsets.task_done - commits = [] - for topic_partition, task_done_offset in six.iteritems(offsets): - - # Skip if None - if task_done_offset is None: - continue - - # Commit offsets as the next offset to fetch - # which is consistent with the Java Client - # task_done is marked by messages consumed, - # so add one to mark the next message for fetching - commit_offset = (task_done_offset + 1) - - # Skip if no change from previous committed - if commit_offset == self._offsets.commit[topic_partition]: - continue - - commits.append( - OffsetCommitRequest(topic_partition[0], topic_partition[1], - commit_offset, metadata) - ) - - if commits: - logger.info('committing consumer offsets to group %s', self._config['group_id']) - resps = self._client.send_offset_commit_request( - self._config['group_id'], commits, - fail_on_error=False - ) - - for r in resps: - check_error(r) - topic_partition = (r.topic, r.partition) - task_done = self._offsets.task_done[topic_partition] - self._offsets.commit[topic_partition] = (task_done + 1) - - if self._config['auto_commit_enable']: - self._reset_auto_commit() - - return True - + if not topics: + self.unsubscribe() else: - logger.info('No new offsets found to commit in group %s', self._config['group_id']) - return False - - # - # Topic/partition management private methods - # - - def _consume_topic_partition(self, topic, partition): - if not isinstance(partition, int): - raise KafkaConfigurationError('Unknown partition type (%s) ' - '-- expected int' % type(partition)) - - if topic not in self._cluster.topics(): - raise UnknownTopicOrPartitionError("Topic %s not found in broker metadata" % topic) - if partition not in self._cluster.partitions_for_topic(topic): - raise UnknownTopicOrPartitionError("Partition %d not found in Topic %s " - "in broker metadata" % (partition, topic)) - logger.info("Configuring consumer to fetch topic '%s', partition %d", topic, partition) - self._topics.append((topic, partition)) - - def _refresh_metadata_on_error(self): - refresh_ms = self._config['refresh_leader_backoff_ms'] - jitter_pct = 0.20 - sleep_ms = random.randint( - int((1.0 - 0.5 * jitter_pct) * refresh_ms), - int((1.0 + 0.5 * jitter_pct) * refresh_ms) - ) - while True: - logger.info("Sleeping for refresh_leader_backoff_ms: %d", sleep_ms) - time.sleep(sleep_ms / 1000.0) - try: - self._client.load_metadata_for_topics() - except KafkaUnavailableError: - logger.warning("Unable to refresh topic metadata... cluster unavailable") - self._check_consumer_timeout() + self._subscription.subscribe(topics=topics, + pattern=pattern, + listener=listener) + # regex will need all topic metadata + if pattern is not None: + self._client.cluster.need_metadata_for_all = True + log.debug("Subscribed to topic pattern: %s", topics) else: - logger.info("Topic metadata refreshed") - return - - # - # Offset-managment private methods - # - - def _get_commit_offsets(self): - logger.info("Consumer fetching stored offsets") - for topic_partition in self._topics: - (resp,) = self._client.send_offset_fetch_request( - self._config['group_id'], - [OffsetFetchRequest(topic_partition[0], topic_partition[1])], - fail_on_error=False) - try: - check_error(resp) - # API spec says server wont set an error here - # but 0.8.1.1 does actually... - except UnknownTopicOrPartitionError: - pass - - # -1 offset signals no commit is currently stored - if resp.offset == -1: - self._offsets.commit[topic_partition] = None - - # Otherwise we committed the stored offset - # and need to fetch the next one - else: - self._offsets.commit[topic_partition] = resp.offset - - def _reset_highwater_offsets(self): - for topic_partition in self._topics: - self._offsets.highwater[topic_partition] = None - - def _reset_task_done_offsets(self): - for topic_partition in self._topics: - self._offsets.task_done[topic_partition] = None - - def _reset_partition_offset(self, topic_partition): - (topic, partition) = topic_partition - LATEST = -1 - EARLIEST = -2 - - request_time_ms = None - if self._config['auto_offset_reset'] == 'largest': - request_time_ms = LATEST - elif self._config['auto_offset_reset'] == 'smallest': - request_time_ms = EARLIEST - else: + self._client.set_topics(self._subscription.group_subscription()) + log.debug("Subscribed to topic(s): %s", topics) - # Let's raise an reasonable exception type if user calls - # outside of an exception context - if sys.exc_info() == (None, None, None): - raise OffsetOutOfRangeError('Cannot reset partition offsets without a ' - 'valid auto_offset_reset setting ' - '(largest|smallest)') + def subscription(self): + """Get the current topic subscription. - # Otherwise we should re-raise the upstream exception - # b/c it typically includes additional data about - # the request that triggered it, and we do not want to drop that - raise + Returns: + set: {topic, ...} + """ + return self._subscription.subscription - (offset, ) = self.get_partition_offsets(topic, partition, - request_time_ms, max_num_offsets=1) - return offset + def unsubscribe(self): + """Unsubscribe from all topics and clear all assigned partitions.""" + self._subscription.unsubscribe() + self._coordinator.close() + self._client.cluster.need_metadata_for_all_topics = False + log.debug("Unsubscribed all topics or patterns and assigned partitions") + + def _update_fetch_positions(self, partitions): + """ + Set the fetch position to the committed position (if there is one) + or reset it using the offset reset policy the user has configured. + + Arguments: + partitions (List[TopicPartition]): The partitions that need + updating fetch positions + + Raises: + NoOffsetForPartitionError: If no offset is stored for a given + partition and no offset reset policy is defined + """ + # refresh commits for all assigned partitions + self._coordinator.refresh_committed_offsets_if_needed() - # - # Consumer Timeout private methods - # - - def _set_consumer_timeout_start(self): - self._consumer_timeout = False - if self._config['consumer_timeout_ms'] >= 0: - self._consumer_timeout = time.time() + (self._config['consumer_timeout_ms'] / 1000.0) - - def _check_consumer_timeout(self): - if self._consumer_timeout and time.time() > self._consumer_timeout: - raise ConsumerTimeout('Consumer timed out after %d ms' % + self._config['consumer_timeout_ms']) - - # - # Autocommit private methods - # - - def _should_auto_commit(self): - if self._does_auto_commit_ms(): - if time.time() >= self._next_commit_time: - return True - - if self._does_auto_commit_messages(): - if self._uncommitted_message_count >= self._config['auto_commit_interval_messages']: - return True - - return False - - def _reset_auto_commit(self): - if not self._config['group_id']: - raise KafkaConfigurationError('auto_commit requires group_id') - self._uncommitted_message_count = 0 - self._next_commit_time = None - if self._does_auto_commit_ms(): - self._next_commit_time = time.time() + (self._config['auto_commit_interval_ms'] / 1000.0) - - def _incr_auto_commit_message_count(self, n=1): - self._uncommitted_message_count += n - - def _does_auto_commit_ms(self): - if not self._config['auto_commit_enable']: - return False - - conf = self._config['auto_commit_interval_ms'] - if conf is not None and conf > 0: - return True - return False - - def _does_auto_commit_messages(self): - if not self._config['auto_commit_enable']: - return False - - conf = self._config['auto_commit_interval_messages'] - if conf is not None and conf > 0: - return True - return False - - # - # Message iterator private methods - # - - def __iter__(self): - return self - - def __next__(self): - return self.next() - - def _get_message_iterator(self): - # Fetch a new batch if needed - if self._msg_iter is None: - self._msg_iter = self.fetch_messages() - - return self._msg_iter - - def _reset_message_iterator(self): - self._msg_iter = None - - # - # python private methods - # - - def __repr__(self): - return '<{0} topics=({1})>'.format( - self.__class__.__name__, - '|'.join(["%s-%d" % topic_partition - for topic_partition in self._topics]) - ) - - # - # other private methods - # - - def _deprecate_configs(self, **configs): - for old, new in six.iteritems(DEPRECATED_CONFIG_KEYS): - if old in configs: - logger.warning('Deprecated Kafka Consumer configuration: %s. ' - 'Please use %s instead.', old, new) - old_value = configs.pop(old) - if new not in configs: - configs[new] = old_value - return configs + # then do any offset lookups in case some positions are not known + self._fetcher.update_fetch_positions(partitions) -- cgit v1.2.1 From 80078c2ed4e8ef1c346b81a6487db997ae03d439 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 15:43:27 -0800 Subject: Fix HeartbeatTask request_in_flight handling --- kafka/coordinator/abstract.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 1713433..10df9a5 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -574,7 +574,7 @@ class HeartbeatTask(object): def _handle_heartbeat_success(self, v): log.debug("Received successful heartbeat") - self.request_in_flight = False + self._request_in_flight = False self._heartbeat.received_heartbeat() ttl = self._heartbeat.ttl() self._client.schedule(self, time.time() + ttl) -- cgit v1.2.1 From ab1e1d014f439af3684d8704f7b679e9e865bb88 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 15:44:39 -0800 Subject: Fix broken SubscriptionState.is_assigned call --- kafka/consumer/group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index abd9473..059c4ee 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -288,7 +288,7 @@ class KafkaConsumer(object): Returns: The last committed offset, or None if there was no prior commit. """ - if self._subscription.is_assigned: + if self._subscription.is_assigned(partition): committed = self._subscription.assignment[partition].committed if committed is None: self._coordinator.refresh_committed_offsets_if_needed() -- cgit v1.2.1 From ac57ada2bd73b5a7ab1c2060207c4c9c6f0ff9a9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 15:45:48 -0800 Subject: Pass auto commit parameters from KafkaConsumer to ConsumerCoordinator --- kafka/consumer/group.py | 2 ++ kafka/coordinator/consumer.py | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 059c4ee..63a1b2e 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -156,6 +156,8 @@ class KafkaConsumer(object): self._client, self._subscription, **kwargs) self._coordinator = ConsumerCoordinator( self._client, self._group_id, self._subscription, + enable_auto_commit=self._enable_auto_commit, + auto_commit_interval_ms=self._auto_commit_interval_ms, assignors=self._partition_assignment_strategy, **kwargs) self._closed = False diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index c17c593..119e372 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -46,7 +46,7 @@ class ConsumerProtocol(object): class ConsumerCoordinator(AbstractCoordinator): """This class manages the coordination process with the consumer coordinator.""" _enable_auto_commit = True - _auto_commit_interval_ms = 60 * 1000 + _auto_commit_interval_ms = 5000 _default_offset_commit_callback = lambda offsets, error: True _assignors = () #_heartbeat_interval_ms = 3000 -- cgit v1.2.1 From 6dea93dfdb0b15270d5fd0ed630e21f319d9cea8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 15:46:50 -0800 Subject: Remove recursive client.poll call in ConsumerCoordinator --- kafka/coordinator/consumer.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 119e372..d5e166b 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -182,9 +182,10 @@ class ConsumerCoordinator(AbstractCoordinator): # the leader will begin watching for changes to any of the topics # the group is interested in, which ensures that all metadata changes # will eventually be seen + # Because assignment typically happens within response callbacks, + # we cannot block on metadata updates here (no recursion into poll()) self._subscription.group_subscribe(all_subscribed_topics) - future = self._client.set_topics(self._subscription.group_subscription()) - self._client.poll(future=future) + self._client.set_topics(self._subscription.group_subscription()) log.debug("Performing %s assignment for subscriptions %s", assignor.name, member_metadata) -- cgit v1.2.1 From 357073cc180e3d80fa1f7ece58fc67dbb8f42af7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 16:58:08 -0800 Subject: Make more methods private(ish) in AbstractCoordinator --- kafka/coordinator/abstract.py | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 10df9a5..2f7b144 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -176,7 +176,7 @@ class AbstractCoordinator(object): self._client.poll() continue - future = self.send_group_metadata_request() + future = self._send_group_metadata_request() self._client.poll(future=future) if future.failed(): @@ -205,7 +205,7 @@ class AbstractCoordinator(object): while self.need_rejoin(): self.ensure_coordinator_known() - future = self.perform_group_join() + future = self._perform_group_join() self._client.poll(future=future) if future.succeeded(): @@ -224,7 +224,7 @@ class AbstractCoordinator(object): raise exception # pylint: disable-msg=raising-bad-type time.sleep(self._retry_backoff_ms / 1000.0) - def perform_group_join(self): + def _perform_group_join(self): """Join the group and return the assignment for the next generation. This function handles both JoinGroup and SyncGroup, delegating to @@ -269,9 +269,9 @@ class AbstractCoordinator(object): self.protocol = response.group_protocol #self.sensors.join_latency.record(response.requestLatencyMs()) if response.leader_id == response.member_id: - self.on_join_leader(response).chain(future) + self._on_join_leader(response).chain(future) else: - self.on_join_follower().chain(future) + self._on_join_follower().chain(future) elif error_type is Errors.GroupLoadInProgressError: log.debug("Attempt to join group %s rejected since coordinator is" @@ -308,7 +308,7 @@ class AbstractCoordinator(object): log.error("Unexpected error in join group response: %s", error) future.failure(error) - def on_join_follower(self): + def _on_join_follower(self): # send follower's sync group with an empty assignment request = SyncGroupRequest( self.group_id, @@ -317,9 +317,9 @@ class AbstractCoordinator(object): {}) log.debug("Issuing follower SyncGroup (%s) to coordinator %s", request, self.coordinator_id) - return self.send_sync_group_request(request) + return self._send_sync_group_request(request) - def on_join_leader(self, response): + def _on_join_leader(self, response): """ Perform leader synchronization and send back the assignment for the group via SyncGroupRequest @@ -342,9 +342,9 @@ class AbstractCoordinator(object): log.debug("Issuing leader SyncGroup (%s) to coordinator %s", request, self.coordinator_id) - return self.send_sync_group_request(request) + return self._send_sync_group_request(request) - def send_sync_group_request(self, request): + def _send_sync_group_request(self, request): if self.coordinator_unknown(): return Future().failure(Errors.GroupCoordinatorNotAvailableError()) future = Future() @@ -389,7 +389,7 @@ class AbstractCoordinator(object): log.error("Unexpected error from SyncGroup: %s", error) future.failure(error) - def send_group_metadata_request(self): + def _send_group_metadata_request(self): """Discover the current coordinator for the group. Sends a GroupMetadata request to one of the brokers. The returned future @@ -477,7 +477,7 @@ class AbstractCoordinator(object): else: log.error("LeaveGroup request failed: %s", error_type()) - def send_heartbeat_request(self): + def _send_heartbeat_request(self): """Send a heartbeat request now (visible only for testing).""" request = HeartbeatRequest(self.group_id, self.generation, self.member_id) future = Future() @@ -568,7 +568,7 @@ class HeartbeatTask(object): log.debug("Sending HeartbeatRequest") self._heartbeat.sent_heartbeat() self._request_in_flight = True - future = self._coordinator.send_heartbeat_request() + future = self._coordinator._send_heartbeat_request() future.add_callback(self._handle_heartbeat_success) future.add_errback(self._handle_heartbeat_failure) -- cgit v1.2.1 From aa35c4e424424687b3ff4a89a10e51d96f365279 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 16:58:38 -0800 Subject: Make add_assigned_partition private(ish) in SubscriptionState --- kafka/consumer/subscription_state.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index a90d9b3..a562093 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -143,7 +143,7 @@ class SubscriptionState(object): for partition in partitions: if partition not in self.assignment: - self.add_assigned_partition(partition) + self._add_assigned_partition(partition) for tp in set(self.assignment.keys()) - self._user_assignment: del self.assignment[tp] @@ -163,7 +163,7 @@ class SubscriptionState(object): raise ValueError("Assigned partition %s for non-subscribed topic." % tp) self.assignment.clear() for tp in assignments: - self.add_assigned_partition(tp) + self._add_assigned_partition(tp) self.needs_partition_assignment = False def unsubscribe(self): @@ -250,7 +250,7 @@ class SubscriptionState(object): def resume(self, partition): self.assignment[partition].resume() - def add_assigned_partition(self, partition): + def _add_assigned_partition(self, partition): self.assignment[partition] = TopicPartitionState() -- cgit v1.2.1 From b7d1ed3fb4644c3b255eea356b7de273b522d1f4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 16:59:59 -0800 Subject: Remove extra raise and old TODO comment in coordinators --- kafka/coordinator/abstract.py | 1 - kafka/coordinator/consumer.py | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 2f7b144..2dc8269 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -329,7 +329,6 @@ class AbstractCoordinator(object): response.group_protocol, response.members) except Exception as e: - raise return Future().failure(e) request = SyncGroupRequest( diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index d5e166b..ef5d2c6 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -69,7 +69,7 @@ class ConsumerCoordinator(AbstractCoordinator): raise Errors.IllegalStateError('Coordinator requires assignors') self._cluster.request_update() - self._cluster.add_listener(self._handle_metadata_update) #TODO + self._cluster.add_listener(self._handle_metadata_update) if self._enable_auto_commit: interval = self._auto_commit_interval_ms / 1000.0 -- cgit v1.2.1 From 2a2733d4fc725f04461a6c4d0ca0fc253f99caeb Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:01:41 -0800 Subject: Improve various docstrings --- kafka/consumer/subscription_state.py | 13 ++++++++++--- kafka/coordinator/abstract.py | 18 ++++++++++++------ kafka/coordinator/consumer.py | 34 +++++++++++++++++++++------------- 3 files changed, 43 insertions(+), 22 deletions(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index a562093..5330e9f 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -151,9 +151,16 @@ class SubscriptionState(object): self.needs_partition_assignment = False def assign_from_subscribed(self, assignments): - """ - Change the assignment to the specified partitions returned from the coordinator, - note this is different from {@link #assignFromUser(Collection)} which directly set the assignment from user inputs + """Update the assignment to the specified partitions + + This method is called by the coordinator to dynamically assign + partitions based on the consumer's topic subscription. This is different + from assign_from_user() which directly sets the assignment from a + user-supplied TopicPartition list. + + Arguments: + assignments (list of TopicPartition): partitions to assign to this + consumer instance. """ if self.subscription is None: raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 2dc8269..b0413d5 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -230,7 +230,9 @@ class AbstractCoordinator(object): This function handles both JoinGroup and SyncGroup, delegating to _perform_assignment() if elected leader by the coordinator. - @return Future() of the assignment returned from the group leader + Returns: + Future: resolves to the encoded-bytes assignment returned from the + group leader """ if self.coordinator_unknown(): e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) @@ -323,6 +325,12 @@ class AbstractCoordinator(object): """ Perform leader synchronization and send back the assignment for the group via SyncGroupRequest + + Arguments: + response (JoinResponse): broker response to parse + + Returns: + Future: resolves to member assignment encoded-bytes """ try: group_assignment = self._perform_assignment(response.leader_id, @@ -391,10 +399,8 @@ class AbstractCoordinator(object): def _send_group_metadata_request(self): """Discover the current coordinator for the group. - Sends a GroupMetadata request to one of the brokers. The returned future - should be polled to get the result of the request. - - @return future indicating the completion of the metadata request + Returns: + Future: resolves to the node id of the coordinator """ node_id = self._client.least_loaded_node() if node_id is None or not self._client.ready(node_id): @@ -477,7 +483,7 @@ class AbstractCoordinator(object): log.error("LeaveGroup request failed: %s", error_type()) def _send_heartbeat_request(self): - """Send a heartbeat request now (visible only for testing).""" + """Send a heartbeat request""" request = HeartbeatRequest(self.group_id, self.generation, self.member_id) future = Future() _f = self._client.send(self.coordinator_id, request) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index ef5d2c6..474c0e0 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -217,9 +217,10 @@ class ConsumerCoordinator(AbstractCoordinator): self._subscription.mark_for_reassignment() def need_rejoin(self): - """ - Check whether the group should be rejoined (e.g. if metadata changes) - @return True if it should, False otherwise + """Check whether the group should be rejoined + + Returns: + bool: True if consumer should rejoin group, False otherwise """ return (self._subscription.partitions_auto_assigned() and (super(ConsumerCoordinator, self).need_rejoin() or @@ -236,12 +237,13 @@ class ConsumerCoordinator(AbstractCoordinator): self._subscription.needs_fetch_committed_offsets = False def fetch_committed_offsets(self, partitions): - """ - Fetch the current committed offsets from the coordinator for a set of - partitions. + """Fetch the current committed offsets for specified partitions - @param partitions The partitions to fetch offsets for - @return dict of {TopicPartition: OffsetMetadata} + Arguments: + partitions (list of TopicPartition): partitions to fetch + + Returns: + dict: {TopicPartition: OffsetAndMetadata} """ while True: self.ensure_coordinator_known() @@ -330,9 +332,12 @@ class ConsumerCoordinator(AbstractCoordinator): polled in the case of a synchronous commit or ignored in the asynchronous case. - @param offsets dict of {TopicPartition: OffsetAndMetadata} that should - be committed - @return Future indicating whether the commit was successful or not + Arguments: + offsets (dict of {TopicPartition: OffsetAndMetadata}): what should + be committed + + Returns: + Future: indicating whether the commit was successful or not """ if self.coordinator_unknown(): return Future().failure(Errors.GroupCoordinatorNotAvailableError) @@ -443,8 +448,11 @@ class ConsumerCoordinator(AbstractCoordinator): This is a non-blocking call. The returned future can be polled to get the actual offsets returned from the broker. - @param partitions list of TopicPartitions - @return Future of committed offsets dict: {TopicPartition: offset} + Arguments: + partitions (list of TopicPartition): the partitions to fetch + + Returns: + Future: resolves to dict of offsets: {TopicPartition: int} """ if self.coordinator_unknown(): return Future().failure(Errors.GroupCoordinatorNotAvailableError) -- cgit v1.2.1 From ab1578444c79703610228531beef259478614338 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:04:24 -0800 Subject: Log as INFO: group join, leader election, partition assignments --- kafka/consumer/subscription_state.py | 1 + kafka/coordinator/abstract.py | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 5330e9f..38d4571 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -172,6 +172,7 @@ class SubscriptionState(object): for tp in assignments: self._add_assigned_partition(tp) self.needs_partition_assignment = False + log.info("Updated partition assignment: %s", assignments) def unsubscribe(self): self.subscription = None diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index b0413d5..78e8d74 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -264,13 +264,16 @@ class AbstractCoordinator(object): def _handle_join_group_response(self, future, response): error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - log.debug("Joined group: %s", response) self.member_id = response.member_id self.generation = response.generation_id self.rejoin_needed = False self.protocol = response.group_protocol + log.info("Joined group '%s' (generation %s) with member_id %s", + self.group_id, self.generation, self.member_id) #self.sensors.join_latency.record(response.requestLatencyMs()) if response.leader_id == response.member_id: + log.info("Elected group leader -- performing partition" + " assignments using %s", self.protocol) self._on_join_leader(response).chain(future) else: self._on_join_follower().chain(future) -- cgit v1.2.1 From f82eb8cdf6ff5b40631a04fd97934f65a358194d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:19:51 -0800 Subject: Improve heartbeat logging --- kafka/coordinator/abstract.py | 27 +++++++++++++-------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 78e8d74..c84475a 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -488,6 +488,8 @@ class AbstractCoordinator(object): def _send_heartbeat_request(self): """Send a heartbeat request""" request = HeartbeatRequest(self.group_id, self.generation, self.member_id) + log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, + request.member_id) future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_heartbeat_response, future) @@ -502,33 +504,32 @@ class AbstractCoordinator(object): future.success(None) elif error_type in (Errors.GroupCoordinatorNotAvailableError, Errors.NotCoordinatorForGroupError): - log.info("Attempt to heart beat failed since coordinator is either" - " not started or not valid; marking it as dead.") + log.info("Heartbeat failed: coordinator is either not started or" + " not valid; will refresh metadata and retry") self.coordinator_dead() future.failure(error_type()) elif error_type is Errors.RebalanceInProgressError: - log.info("Attempt to heart beat failed since the group is" - " rebalancing; try to re-join group.") + log.info("Heartbeat failed: group is rebalancing; re-joining group") self.rejoin_needed = True future.failure(error_type()) elif error_type is Errors.IllegalGenerationError: - log.info("Attempt to heart beat failed since generation id" - " is not legal; try to re-join group.") + log.info("Heartbeat failed: local generation id is not current;" + " re-joining group") self.rejoin_needed = True future.failure(error_type()) elif error_type is Errors.UnknownMemberIdError: - log.info("Attempt to heart beat failed since member id" - " is not valid; reset it and try to re-join group.") + log.info("Heartbeat failed: local member_id was not recognized;" + " resetting and re-joining group") self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID self.rejoin_needed = True future.failure(error_type) elif error_type is Errors.GroupAuthorizationFailedError: error = error_type(self.group_id) - log.error("Attempt to heart beat failed authorization: %s", error) + log.error("Heartbeat failed: authorization error: %s", error) future.failure(error) else: error = error_type() - log.error("Unknown error in heart beat response: %s", error) + log.error("Heartbeat failed: Unhandled error: %s", error) future.failure(error) @@ -550,7 +551,6 @@ class HeartbeatTask(object): self._client.schedule(self, time.time()) def __call__(self): - log.debug("Running Heartbeat task") if (self._coordinator.generation < 0 or self._coordinator.need_rejoin() or self._coordinator.coordinator_unknown()): @@ -563,17 +563,16 @@ class HeartbeatTask(object): if self._heartbeat.session_expired(): # we haven't received a successful heartbeat in one session interval # so mark the coordinator dead - log.error("Heartbeat session expired") + log.error("Heartbeat session expired - marking coordinator dead") self._coordinator.coordinator_dead() return if not self._heartbeat.should_heartbeat(): # we don't need to heartbeat now, so reschedule for when we do ttl = self._heartbeat.ttl() - log.debug("Heartbeat unneeded now, retrying in %s", ttl) + log.debug("Heartbeat task unneeded now, retrying in %s", ttl) self._client.schedule(self, time.time() + ttl) else: - log.debug("Sending HeartbeatRequest") self._heartbeat.sent_heartbeat() self._request_in_flight = True future = self._coordinator._send_heartbeat_request() -- cgit v1.2.1 From 8811326443496d5efcc784c67b9d39824ac0ecee Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:21:25 -0800 Subject: _handle_group_coordinator_response should resolve future with the coordinator id --- kafka/coordinator/abstract.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index c84475a..2431132 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -440,7 +440,7 @@ class AbstractCoordinator(object): # start sending heartbeats only if we have a valid generation if self.generation > 0: self.heartbeat_task.reset() - future.success(None) + future.success(self.coordinator_id) elif error_type is Errors.GroupAuthorizationFailedError: error = error_type(self.group_id) log.error("Group Coordinator Request failed: %s", error) -- cgit v1.2.1 From 39e7562b31a7058bdeeb6d61abbbbd7627799546 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:27:36 -0800 Subject: Log request failures in AbstractCoordinator._failed_request --- kafka/coordinator/abstract.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 2431132..03302a3 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -254,10 +254,13 @@ class AbstractCoordinator(object): future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_join_group_response, future) - _f.add_errback(self._failed_request, future) + _f.add_errback(self._failed_request, self.coordinator_id, + request, future) return future - def _failed_request(self, future, error): + def _failed_request(self, node_id, request, future, error): + log.error('Error sending %s to node %s [%s] -- marking coordinator dead', + request.__class__.__name__, node_id, error) self.coordinator_dead() future.failure(error) @@ -360,7 +363,8 @@ class AbstractCoordinator(object): future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_sync_group_response, future) - _f.add_errback(self._failed_request, future) + _f.add_errback(self._failed_request, self.coordinator_id, + request, future) return future def _handle_sync_group_response(self, future, response): @@ -414,7 +418,7 @@ class AbstractCoordinator(object): future = Future() _f = self._client.send(node_id, request) _f.add_callback(self._handle_group_coordinator_response, future) - _f.add_errback(self._failed_request, future) + _f.add_errback(self._failed_request, node_id, request, future) return future def _handle_group_coordinator_response(self, future, response): @@ -493,7 +497,8 @@ class AbstractCoordinator(object): future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_heartbeat_response, future) - _f.add_errback(self._failed_request, future) + _f.add_errback(self._failed_request, self.coordinator_id, + request, future) return future def _handle_heartbeat_response(self, future, response): -- cgit v1.2.1 From 6e20e0bb52143955e49a3edca77153b5aba58148 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:34:17 -0800 Subject: Improve OffsetCommit error logging Avoid printing full errors because they currently include long descriptions that are generally duplicative of our local error message. --- kafka/coordinator/consumer.py | 56 +++++++++++++++++++++++++------------------ 1 file changed, 33 insertions(+), 23 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 474c0e0..67b4b6d 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -321,9 +321,18 @@ class ConsumerCoordinator(AbstractCoordinator): try: self.commit_offsets_sync(self._subscription.all_consumed_offsets()) + + # The three main group membership errors are known and should not + # require a stacktrace -- just a warning + except (Errors.UnknownMemberIdError, + Errors.IllegalGenerationError, + Errors.RebalanceInProgressError): + log.warning("Offset commit failed: group membership out of date" + " This is likely to cause duplicate message" + " delivery.") except Exception: - # consistent with async auto-commit failures, we do not propagate the exception - log.exception("Auto offset commit failed") + log.exception("Offset commit failed: This is likely to cause" + " duplicate message delivery") def _send_offset_commit_request(self, offsets): """Commit offsets for the specified list of topics and partitions. @@ -388,7 +397,8 @@ class ConsumerCoordinator(AbstractCoordinator): if self._subscription.is_assigned(tp): self._subscription.assignment[tp].committed = offset.offset elif error_type is Errors.GroupAuthorizationFailedError: - log.error("Unauthorized to commit for group %s", self.group_id) + log.error("OffsetCommit failed for group %s - %s", + self.group_id, error_type.__name__) future.failure(error_type(self.group_id)) return elif error_type is Errors.TopicAuthorizationFailedError: @@ -396,48 +406,48 @@ class ConsumerCoordinator(AbstractCoordinator): elif error_type in (Errors.OffsetMetadataTooLargeError, Errors.InvalidCommitOffsetSizeError): # raise the error to the user - error = error_type() - log.info("Offset commit for group %s failed on partition" - " %s due to %s will retry", self.group_id, tp, error) - future.failure(error) + log.info("OffsetCommit failed for group %s on partition %s" + " due to %s, will retry", self.group_id, tp, + error_type.__name__) + future.failure(error_type()) return elif error_type is Errors.GroupLoadInProgressError: # just retry - error = error_type(self.group_id) - log.info("Offset commit for group %s failed due to %s," - " will retry", self.group_id, error) - future.failure(error) + log.info("OffsetCommit failed for group %s because group is" + " initializing (%s), will retry", self.group_id, + error_type.__name__) + future.failure(error_type(self.group_id)) return elif error_type in (Errors.GroupCoordinatorNotAvailableError, Errors.NotCoordinatorForGroupError, Errors.RequestTimedOutError): - error = error_type(self.group_id) - log.info("Offset commit for group %s failed due to %s," - " will find new coordinator and retry", - self.group_id, error) + log.info("OffsetCommit failed for group %s due to a" + " coordinator error (%s), will find new coordinator" + " and retry", self.group_id, error_type.__name__) self.coordinator_dead() - future.failure(error) + future.failure(error_type(self.group_id)) return elif error_type in (Errors.UnknownMemberIdError, Errors.IllegalGenerationError, Errors.RebalanceInProgressError): # need to re-join group error = error_type(self.group_id) - log.error("Error %s occurred while committing offsets for" - " group %s", error, self.group_id) + log.error("OffsetCommit failed for group %s due to group" + " error (%s), will rejoin", self.group_id, error) self._subscription.mark_for_reassignment() # Errors.CommitFailedError("Commit cannot be completed due to group rebalance")) future.failure(error) return else: - error = error_type() - log.error("Unexpected error committing partition %s at" - " offset %s: %s", tp, offset, error) - future.failure(error) + log.error("OffsetCommit failed for group % on partition %s" + " with offset %s: %s", tp, offset, + error_type.__name__) + future.failure(error_type()) return if unauthorized_topics: - log.error("Unauthorized to commit to topics %s", unauthorized_topics) + log.error("OffsetCommit failed for unauthorized topics %s", + unauthorized_topics) future.failure(Errors.TopicAuthorizationFailedError(unauthorized_topics)) else: future.success(True) -- cgit v1.2.1 From 563e38a84f75d2ce3df23f1734d08feaaefe24b6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:36:11 -0800 Subject: Dont warn in AutoCommitTask.disable if not previously scheduled --- kafka/coordinator/consumer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 67b4b6d..92a840d 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -540,7 +540,7 @@ class AutoCommitTask(object): try: self._client.unschedule(self) except KeyError: - log.warning("AutoCommitTask was not previously scheduled") + pass def _reschedule(self, at): if self._enabled: -- cgit v1.2.1 From c549a2b403900c0f305f67224dc13356aa65934c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:48:10 -0800 Subject: Use ConnectionError rather than new DisconnectError --- kafka/common.py | 8 ++------ kafka/conn.py | 4 ++-- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/kafka/common.py b/kafka/common.py index 597fb5c..84cf719 100644 --- a/kafka/common.py +++ b/kafka/common.py @@ -106,11 +106,6 @@ class IllegalArgumentError(KafkaError): pass -class DisconnectError(KafkaError): - retriable = True - invalid_metadata = True - - class NoBrokersAvailable(KafkaError): retriable = True invalid_metadata = True @@ -412,7 +407,8 @@ class FailedPayloadsError(KafkaError): class ConnectionError(KafkaError): - pass + retriable = True + invalid_metadata = True class BufferUnderflowError(KafkaError): diff --git a/kafka/conn.py b/kafka/conn.py index a05ce8e..5afd946 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -128,7 +128,7 @@ class BrokerConnection(object): self._rbuffer.seek(0) self._rbuffer.truncate() if error is None: - error = Errors.DisconnectError() + error = Errors.ConnectionError() while self.in_flight_requests: ifr = self.in_flight_requests.popleft() ifr.future.failure(error) @@ -140,7 +140,7 @@ class BrokerConnection(object): """ future = Future() if not self.connected(): - return future.failure(Errors.DisconnectError()) + return future.failure(Errors.ConnectionError()) if not self.can_send_more(): return future.failure(Errors.TooManyInFlightRequests()) self._correlation_id += 1 -- cgit v1.2.1 From 3748bade2893acd8870c98c810173776eced0068 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 17:49:04 -0800 Subject: Catch and retry on RequestTimedOutError in producer failover test --- test/test_failover_integration.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 04c9e2b..339a08b 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -3,7 +3,9 @@ import os import time from kafka import KafkaClient, SimpleConsumer, KeyedProducer -from kafka.common import TopicPartition, FailedPayloadsError, ConnectionError +from kafka.common import ( + TopicPartition, FailedPayloadsError, ConnectionError, RequestTimedOutError +) from kafka.producer.base import Producer from test.fixtures import ZookeeperFixture, KafkaFixture @@ -77,7 +79,7 @@ class TestFailover(KafkaIntegrationTestCase): producer.send_messages(topic, partition, b'success') log.debug("success!") recovered = True - except (FailedPayloadsError, ConnectionError): + except (FailedPayloadsError, ConnectionError, RequestTimedOutError): log.debug("caught exception sending message -- will retry") continue -- cgit v1.2.1 From a766495355cdcc046566b4f96545c4d0f71cb7ec Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 19:05:29 -0800 Subject: Clean more ConsumerCoordinator docstrings --- kafka/coordinator/consumer.py | 35 +++++++++++++++++++---------------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 92a840d..f7e55f6 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -272,13 +272,16 @@ class ConsumerCoordinator(AbstractCoordinator): super(ConsumerCoordinator, self).close() def commit_offsets_async(self, offsets, callback=None): - """ - @param offsets: dict of {TopicPartition: OffsetAndMetadata} to commit - @param callback: called as callback(offsets, response), with response - as either an Exception or a OffsetCommitResponse - struct. This callback can be used to trigger custom - actions when a commit request completes. - @returns Future + """Commit specific offsets asynchronously. + + Arguments: + offsets (dict {TopicPartition: OffsetAndMetadata}): what to commit + callback (callable, optional): called as callback(offsets, response) + response will be either an Exception or a OffsetCommitResponse + struct. This callback can be used to trigger custom actions when + a commit request completes. + Returns: + Future: indicating whether the commit was successful or not """ self._subscription.needs_fetch_committed_offsets = True future = self._send_offset_commit_request(offsets) @@ -286,15 +289,15 @@ class ConsumerCoordinator(AbstractCoordinator): future.add_both(cb, offsets) def commit_offsets_sync(self, offsets): - """ - Commit offsets synchronously. This method will retry until the commit - completes successfully or an unrecoverable error is encountered. - - @param offsets dict of {TopicPartition: OffsetAndMetadata} to commit - @raises TopicAuthorizationError if the consumer is not authorized to the - group or to any of the specified partitions - @raises CommitFailedError if an unrecoverable error occurs before the - commit can be completed + """Commit specific offsets synchronously. + + This method will retry until the commit completes successfully or an + unrecoverable error is encountered. + + Arguments: + offsets (dict {TopicPartition: OffsetAndMetadata}): what to commit + + Raises error on failure """ if not offsets: return -- cgit v1.2.1 From 8f0d1c1716205d82c8ee2c22baf60413936650c9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 19:06:04 -0800 Subject: Fix log statement bug (via pylint) --- kafka/coordinator/consumer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index f7e55f6..211d1d0 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -443,7 +443,7 @@ class ConsumerCoordinator(AbstractCoordinator): return else: log.error("OffsetCommit failed for group % on partition %s" - " with offset %s: %s", tp, offset, + " with offset %s: %s", self.group_id, tp, offset, error_type.__name__) future.failure(error_type()) return -- cgit v1.2.1 From e5c7d81e7c35e6b013cece347ef42d9f21d03aa6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 19:07:44 -0800 Subject: Use _next_correlation_id() method to avoid int overflows --- kafka/conn.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 5afd946..7979ba7 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -143,9 +143,9 @@ class BrokerConnection(object): return future.failure(Errors.ConnectionError()) if not self.can_send_more(): return future.failure(Errors.TooManyInFlightRequests()) - self._correlation_id += 1 + correlation_id = self._next_correlation_id() header = RequestHeader(request, - correlation_id=self._correlation_id, + correlation_id=correlation_id, client_id=self._client_id) message = b''.join([header.encode(), request.encode()]) size = Int32.encode(len(message)) @@ -163,11 +163,11 @@ class BrokerConnection(object): log.exception("Error sending %s to %s", request, self) self.close(error=e) return future.failure(e) - log.debug('%s Request %d: %s', self, self._correlation_id, request) + log.debug('%s Request %d: %s', self, correlation_id, request) if expect_response: ifr = InFlightRequest(request=request, - correlation_id=self._correlation_id, + correlation_id=correlation_id, response_type=request.RESPONSE_TYPE, future=future, timestamp=time.time()) @@ -299,6 +299,10 @@ class BrokerConnection(object): return True return False + def _next_correlation_id(self): + self._correlation_id = (self._correlation_id + 1) % 2**31 + return self._correlation_id + def __repr__(self): return "" % (self.host, self.port) -- cgit v1.2.1 From 3afdd285a3c92a2c4add5b2b1bd94cfcec4fedd9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Dec 2015 19:08:35 -0800 Subject: Switch configs from attributes to dict to make passing / inspecting easier --- kafka/client_async.py | 73 ++++++++++++---------------- kafka/cluster.py | 20 ++++---- kafka/conn.py | 56 ++++++++++++---------- kafka/consumer/fetcher.py | 50 ++++++++++---------- kafka/consumer/group.py | 105 ++++++++++++++++++++--------------------- kafka/coordinator/abstract.py | 34 ++++++------- kafka/coordinator/consumer.py | 58 ++++++++++++----------- kafka/coordinator/heartbeat.py | 26 +++++----- 8 files changed, 211 insertions(+), 211 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 5c11fc5..6fb5fdd 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -1,3 +1,4 @@ +import copy import heapq import itertools import logging @@ -15,6 +16,7 @@ from .conn import BrokerConnection, ConnectionStates, collect_hosts from .future import Future from .protocol.metadata import MetadataRequest from .protocol.produce import ProduceRequest +from .version import __version__ log = logging.getLogger(__name__) @@ -27,26 +29,23 @@ class KafkaClient(object): This class is not thread-safe! """ - _bootstrap_servers = 'localhost' - _client_id = 'kafka-python-0.10.0' - _reconnect_backoff_ms = 50 - _retry_backoff_ms = 100 - _send_buffer_bytes = 131072 - _receive_buffer_bytes = 32768 - _request_timeout_ms = 40000 - _max_in_flight_requests_per_connection=5 - - def __init__(self, **kwargs): - for config in ( - 'client_id', 'max_in_flight_requests_per_connection', - 'reconnect_backoff_ms', 'retry_backoff_ms', - 'send_buffer_bytes', 'receive_buffer_bytes', - 'request_timeout_ms', 'bootstrap_servers' - ): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) - - self.cluster = ClusterMetadata(**kwargs) + DEFAULT_CONFIG = { + 'bootstrap_servers': 'localhost', + 'client_id': 'kafka-python-' + __version__, + 'request_timeout_ms': 40000, + 'reconnect_backoff_ms': 50, + 'max_in_flight_requests_per_connection': 5, + 'receive_buffer_bytes': 32768, + 'send_buffer_bytes': 131072, + } + + def __init__(self, **configs): + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] + + self.cluster = ClusterMetadata(**self.config) self._topics = set() # empty set will fetch all topic metadata self._metadata_refresh_in_progress = False self._conns = {} @@ -54,11 +53,11 @@ class KafkaClient(object): self._delayed_tasks = DelayedTaskQueue() self._last_bootstrap = 0 self._bootstrap_fails = 0 - self._bootstrap(collect_hosts(self._bootstrap_servers)) + self._bootstrap(collect_hosts(self.config['bootstrap_servers'])) def _bootstrap(self, hosts): # Exponential backoff if bootstrap fails - backoff_ms = self._reconnect_backoff_ms * 2 ** self._bootstrap_fails + backoff_ms = self.config['reconnect_backoff_ms'] * 2 ** self._bootstrap_fails next_at = self._last_bootstrap + backoff_ms / 1000.0 now = time.time() if next_at > now: @@ -69,15 +68,7 @@ class KafkaClient(object): metadata_request = MetadataRequest([]) for host, port in hosts: log.debug("Attempting to bootstrap via node at %s:%s", host, port) - bootstrap = BrokerConnection( - host, port, - client_id=self._client_id, - receive_buffer_bytes=self._receive_buffer_bytes, - send_buffer_bytes=self._send_buffer_bytes, - request_timeout_ms=self._request_timeout_ms, - max_in_flight_requests_per_connection=self._max_in_flight_requests_per_connection, - reconnect_backoff_ms=self._reconnect_backoff_ms - ) + bootstrap = BrokerConnection(host, port, **self.config) bootstrap.connect() while bootstrap.state is ConnectionStates.CONNECTING: bootstrap.connect() @@ -121,15 +112,8 @@ class KafkaClient(object): if node_id not in self._conns: log.debug("Initiating connection to node %s at %s:%s", node_id, broker.host, broker.port) - self._conns[node_id] = BrokerConnection( - broker.host, broker.port, - client_id=self._client_id, - receive_buffer_bytes=self._receive_buffer_bytes, - send_buffer_bytes=self._send_buffer_bytes, - request_timeout_ms=self._request_timeout_ms, - max_in_flight_requests_per_connection=self._max_in_flight_requests_per_connection, - reconnect_backoff_ms=self._reconnect_backoff_ms - ) + self._conns[node_id] = BrokerConnection(broker.host, broker.port, + **self.config) return self._finish_connect(node_id) def _finish_connect(self, node_id): @@ -194,7 +178,7 @@ class KafkaClient(object): conn = self._conns[node_id] time_waited_ms = time.time() - (conn.last_attempt or 0) if conn.state is ConnectionStates.DISCONNECTED: - return max(self._reconnect_backoff_ms - time_waited_ms, 0) + return max(self.config['reconnect_backoff_ms'] - time_waited_ms, 0) else: return sys.maxint @@ -262,7 +246,7 @@ class KafkaClient(object): @return The list of responses received. """ if timeout_ms is None: - timeout_ms = self._request_timeout_ms + timeout_ms = self.config['request_timeout_ms'] responses = [] @@ -283,7 +267,8 @@ class KafkaClient(object): except Exception as e: log.error("Task %s failed: %s", task, e) - timeout = min(timeout_ms, metadata_timeout, self._request_timeout_ms) + timeout = min(timeout_ms, metadata_timeout, + self.config['request_timeout_ms']) timeout /= 1000.0 responses.extend(self._poll(timeout)) @@ -365,7 +350,7 @@ class KafkaClient(object): # Last option: try to bootstrap again log.error('No nodes found in metadata -- retrying bootstrap') - self._bootstrap(collect_hosts(self._bootstrap_servers)) + self._bootstrap(collect_hosts(self.config['bootstrap_servers'])) return None def set_topics(self, topics): diff --git a/kafka/cluster.py b/kafka/cluster.py index 5b5fd8e..84ad1d3 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -1,5 +1,6 @@ from __future__ import absolute_import +import copy import logging import random import time @@ -12,10 +13,12 @@ log = logging.getLogger(__name__) class ClusterMetadata(object): - _retry_backoff_ms = 100 - _metadata_max_age_ms = 300000 + DEFAULT_CONFIG = { + 'retry_backoff_ms': 100, + 'metadata_max_age_ms': 300000, + } - def __init__(self, **kwargs): + def __init__(self, **configs): self._brokers = {} self._partitions = {} self._groups = {} @@ -26,9 +29,10 @@ class ClusterMetadata(object): self._future = None self._listeners = set() - for config in ('retry_backoff_ms', 'metadata_max_age_ms'): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] def brokers(self): return set(self._brokers.values()) @@ -55,8 +59,8 @@ class ClusterMetadata(object): if self._need_update: ttl = 0 else: - ttl = self._last_successful_refresh_ms + self._metadata_max_age_ms - now - retry = self._last_refresh_ms + self._retry_backoff_ms - now + ttl = self._last_successful_refresh_ms + self.config['metadata_max_age_ms'] - now + retry = self._last_refresh_ms + self.config['retry_backoff_ms'] - now return max(ttl, retry, 0) def request_update(self): diff --git a/kafka/conn.py b/kafka/conn.py index 7979ba7..8ce4a6f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -17,6 +17,7 @@ from kafka.common import ConnectionError from kafka.future import Future from kafka.protocol.api import RequestHeader from kafka.protocol.types import Int32 +from kafka.version import __version__ log = logging.getLogger(__name__) @@ -36,25 +37,24 @@ InFlightRequest = collections.namedtuple('InFlightRequest', class BrokerConnection(object): - _receive_buffer_bytes = 32768 - _send_buffer_bytes = 131072 - _client_id = 'kafka-python-0.10.0' - _correlation_id = 0 - _request_timeout_ms = 40000 - _max_in_flight_requests_per_connection = 5 - _reconnect_backoff_ms = 50 - - def __init__(self, host, port, **kwargs): + DEFAULT_CONFIG = { + 'client_id': 'kafka-python-' + __version__, + 'request_timeout_ms': 40000, + 'reconnect_backoff_ms': 50, + 'max_in_flight_requests_per_connection': 5, + 'receive_buffer_bytes': 32768, + 'send_buffer_bytes': 131072, + } + + def __init__(self, host, port, **configs): self.host = host self.port = port self.in_flight_requests = collections.deque() - for config in ('receive_buffer_bytes', 'send_buffer_bytes', - 'client_id', 'correlation_id', 'request_timeout_ms', - 'max_in_flight_requests_per_connection', - 'reconnect_backoff_ms'): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] self.state = ConnectionStates.DISCONNECTED self._sock = None @@ -64,14 +64,17 @@ class BrokerConnection(object): self.last_attempt = 0 self.last_failure = 0 self._processing = False + self._correlation_id = 0 def connect(self): """Attempt to connect and return ConnectionState""" if self.state is ConnectionStates.DISCONNECTED: self.close() self._sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_RCVBUF, self._receive_buffer_bytes) - self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_SNDBUF, self._send_buffer_bytes) + self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_RCVBUF, + self.config['receive_buffer_bytes']) + self._sock.setsockopt(socket.SOL_SOCKET, socket.SO_SNDBUF, + self.config['send_buffer_bytes']) self._sock.setblocking(False) ret = self._sock.connect_ex((self.host, self.port)) self.last_attempt = time.time() @@ -89,7 +92,8 @@ class BrokerConnection(object): if self.state is ConnectionStates.CONNECTING: # in non-blocking mode, use repeated calls to socket.connect_ex # to check connection status - if time.time() > (self._request_timeout_ms / 1000.0) + self.last_attempt: + request_timeout = self.config['request_timeout_ms'] / 1000.0 + if time.time() > request_timeout + self.last_attempt: log.error('Connection attempt to %s timed out', self) self.close() # error=TimeoutError ? self.last_failure = time.time() @@ -110,8 +114,8 @@ class BrokerConnection(object): re-establish a connection yet """ if self.state is ConnectionStates.DISCONNECTED: - now = time.time() - if now - self.last_attempt < self._reconnect_backoff_ms / 1000.0: + backoff = self.config['reconnect_backoff_ms'] / 1000.0 + if time.time() < self.last_attempt + backoff: return True return False @@ -146,7 +150,7 @@ class BrokerConnection(object): correlation_id = self._next_correlation_id() header = RequestHeader(request, correlation_id=correlation_id, - client_id=self._client_id) + client_id=self.config['client_id']) message = b''.join([header.encode(), request.encode()]) size = Int32.encode(len(message)) try: @@ -178,7 +182,8 @@ class BrokerConnection(object): return future def can_send_more(self): - return len(self.in_flight_requests) < self._max_in_flight_requests_per_connection + max_ifrs = self.config['max_in_flight_requests_per_connection'] + return len(self.in_flight_requests) < max_ifrs def recv(self, timeout=0): """Non-blocking network receive @@ -202,9 +207,10 @@ class BrokerConnection(object): elif self._requests_timed_out(): log.warning('%s timed out after %s ms. Closing connection.', - self, self._request_timeout_ms) + self, self.config['request_timeout_ms']) self.close(error=Errors.RequestTimedOutError( - 'Request timed out after %s ms' % self._request_timeout_ms)) + 'Request timed out after %s ms' % + self.config['request_timeout_ms'])) return None readable, _, _ = select([self._sock], [], [], timeout) @@ -294,7 +300,7 @@ class BrokerConnection(object): def _requests_timed_out(self): if self.in_flight_requests: oldest_at = self.in_flight_requests[0].timestamp - timeout = self._request_timeout_ms / 1000.0 + timeout = self.config['request_timeout_ms'] / 1000.0 if time.time() >= oldest_at + timeout: return True return False diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index ea9c8b9..39e1244 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -1,6 +1,7 @@ from __future__ import absolute_import import collections +import copy import logging import six @@ -28,27 +29,25 @@ class RecordTooLargeError(Errors.KafkaError): class Fetcher(object): - _key_deserializer = None - _value_deserializer = None - _fetch_min_bytes = 1024 - _fetch_max_wait_ms = 500 - _max_partition_fetch_bytes = 1048576 - _check_crcs = True - _retry_backoff_ms = 100 - - def __init__(self, client, subscriptions, **kwargs): + DEFAULT_CONFIG = { + 'key_deserializer': None, + 'value_deserializer': None, + 'fetch_min_bytes': 1024, + 'fetch_max_wait_ms': 500, + 'max_partition_fetch_bytes': 1048576, + 'check_crcs': True, + } + + def __init__(self, client, subscriptions, **configs): #metrics=None, #metric_group_prefix='consumer', + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] self._client = client self._subscriptions = subscriptions - for config in ('key_deserializer', 'value_deserializer', - 'fetch_min_bytes', 'fetch_max_wait_ms', - 'max_partition_fetch_bytes', 'check_crcs', - 'retry_backoff_ms'): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) - self._records = collections.deque() # (offset, topic_partition, messages) self._unauthorized_topics = set() self._offset_out_of_range_partitions = dict() # {topic_partition: offset} @@ -204,7 +203,8 @@ class Fetcher(object): " and hence cannot be ever returned." " Increase the fetch size, or decrease the maximum message" " size the broker will allow.", - copied_record_too_large_partitions, self._max_partition_fetch_bytes) + copied_record_too_large_partitions, + self.config['max_partition_fetch_bytes']) def fetched_records(self): """Returns previously fetched records and updates consumed offsets @@ -255,7 +255,7 @@ class Fetcher(object): for offset, size, msg in messages: if msg.attributes: raise Errors.KafkaError('Compressed messages not supported yet') - elif self._check_crcs and not msg.validate_crc(): + elif self.config['check_crcs'] and not msg.validate_crc(): raise Errors.InvalidMessageError(msg) key, value = self._deserialize(msg) @@ -269,12 +269,12 @@ class Fetcher(object): return dict(drained) def _deserialize(self, msg): - if self._key_deserializer: - key = self._key_deserializer(msg.key) # pylint: disable-msg=not-callable + if self.config['key_deserializer']: + key = self.config['key_deserializer'](msg.key) # pylint: disable-msg=not-callable else: key = msg.key - if self._value_deserializer: - value = self._value_deserializer(msg.value) # pylint: disable-msg=not-callable + if self.config['value_deserializer']: + value = self.config['value_deserializer'](msg.value) # pylint: disable-msg=not-callable else: value = msg.value return key, value @@ -376,7 +376,7 @@ class Fetcher(object): partition_info = ( partition.partition, fetched, - self._max_partition_fetch_bytes + self.config['max_partition_fetch_bytes'] ) fetchable[node_id][partition.topic].append(partition_info) else: @@ -388,8 +388,8 @@ class Fetcher(object): for node_id, partition_data in six.iteritems(fetchable): requests[node_id] = FetchRequest( -1, # replica_id - self._fetch_max_wait_ms, - self._fetch_min_bytes, + self.config['fetch_max_wait_ms'], + self.config['fetch_min_bytes'], partition_data.items()) return requests diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 63a1b2e..b7093f3 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -1,5 +1,6 @@ from __future__ import absolute_import +import copy import logging import time @@ -18,33 +19,36 @@ log = logging.getLogger(__name__) class KafkaConsumer(object): """Consumer for Kafka 0.9""" - _bootstrap_servers = 'localhost' - _client_id = 'kafka-python-' + __version__ - _group_id = 'kafka-python-default-group' - _key_deserializer = None - _value_deserializer = None - _fetch_max_wait_ms = 500 - _fetch_min_bytes = 1024 - _max_partition_fetch_bytes = 1 * 1024 * 1024 - _request_timeout_ms = 40 * 1000 - _retry_backoff_ms = 100 - _reconnect_backoff_ms = 50 - _auto_offset_reset = 'latest' - _enable_auto_commit = True - _auto_commit_interval_ms = 5000 - _check_crcs = True - _metadata_max_age_ms = 5 * 60 * 1000 - _partition_assignment_strategy = (RoundRobinPartitionAssignor,) - _heartbeat_interval_ms = 3000 - _session_timeout_ms = 30000 - _send_buffer_bytes = 128 * 1024 - _receive_buffer_bytes = 32 * 1024 - _connections_max_idle_ms = 9 * 60 * 1000 # not implemented yet - #_metric_reporters = None - #_metrics_num_samples = 2 - #_metrics_sample_window_ms = 30000 - - def __init__(self, *topics, **kwargs): + DEFAULT_CONFIG = { + 'bootstrap_servers': 'localhost', + 'client_id': 'kafka-python-' + __version__, + 'group_id': 'kafka-python-default-group', + 'key_deserializer': None, + 'value_deserializer': None, + 'fetch_max_wait_ms': 500, + 'fetch_min_bytes': 1024, + 'max_partition_fetch_bytes': 1 * 1024 * 1024, + 'request_timeout_ms': 40 * 1000, + 'retry_backoff_ms': 100, + 'reconnect_backoff_ms': 50, + 'max_in_flight_requests_per_connection': 5, + 'auto_offset_reset': 'latest', + 'enable_auto_commit': True, + 'auto_commit_interval_ms': 5000, + 'check_crcs': True, + 'metadata_max_age_ms': 5 * 60 * 1000, + 'partition_assignment_strategy': (RoundRobinPartitionAssignor,), + 'heartbeat_interval_ms': 3000, + 'session_timeout_ms': 30000, + 'send_buffer_bytes': 128 * 1024, + 'receive_buffer_bytes': 32 * 1024, + 'connections_max_idle_ms': 9 * 60 * 1000, # not implemented yet + #'metric_reporters': None, + #'metrics_num_samples': 2, + #'metrics_sample_window_ms': 30000, + } + + def __init__(self, *topics, **configs): """A Kafka client that consumes records from a Kafka cluster. The consumer will transparently handle the failure of servers in the @@ -79,8 +83,8 @@ class KafkaConsumer(object): raw message value and returns a deserialized value. fetch_min_bytes (int): Minimum amount of data the server should return for a fetch request, otherwise wait up to - fetch_wait_max_ms for more data to accumulate. Default: 1024. - fetch_wait_max_ms (int): The maximum amount of time in milliseconds + fetch_max_wait_ms for more data to accumulate. Default: 1024. + fetch_max_wait_ms (int): The maximum amount of time in milliseconds the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch_min_bytes. Default: 500. @@ -97,8 +101,11 @@ class KafkaConsumer(object): retry_backoff_ms (int): Milliseconds to backoff when retrying on errors. Default: 100. reconnect_backoff_ms (int): The amount of time in milliseconds to - wait before attempting to reconnect to a given host. Defaults - to 50. + wait before attempting to reconnect to a given host. + Default: 50. + max_in_flight_requests_per_connection (int): Requests are pipelined + to kafka brokers up to this number of maximum requests per + broker connection. Default: 5. auto_offset_reset (str): A policy for resetting offsets on OffsetOutOfRange errors: 'earliest' will move to the oldest available message, 'latest' will move to the most recent. Any @@ -137,29 +144,19 @@ class KafkaConsumer(object): Configuration parameters are described in more detail at https://kafka.apache.org/090/configuration.html#newconsumerconfigs """ - for config in ('bootstrap_servers', 'client_id', 'group_id', - 'key_deserializer', 'value_deserializer', - 'fetch_max_wait_ms', 'fetch_min_bytes', - 'max_partition_fetch_bytes', 'request_timeout_ms', - 'retry_backoff_ms', 'reconnect_backoff_ms', - 'auto_offset_reset', 'enable_auto_commit', - 'auto_commit_interval_ms', 'check_crcs', - 'metadata_max_age_ms', 'partition_assignment_strategy', - 'heartbeat_interval_ms', 'session_timeout_ms', - 'send_buffer_bytes', 'receive_buffer_bytes'): - if config in kwargs: - setattr(self, '_' + config, kwargs[config]) - - self._client = KafkaClient(**kwargs) - self._subscription = SubscriptionState(self._auto_offset_reset) + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] + + self._client = KafkaClient(**self.config) + self._subscription = SubscriptionState(self.config['auto_offset_reset']) self._fetcher = Fetcher( - self._client, self._subscription, **kwargs) + self._client, self._subscription, **self.config) self._coordinator = ConsumerCoordinator( - self._client, self._group_id, self._subscription, - enable_auto_commit=self._enable_auto_commit, - auto_commit_interval_ms=self._auto_commit_interval_ms, - assignors=self._partition_assignment_strategy, - **kwargs) + self._client, self.config['group_id'], self._subscription, + assignors=self.config['partition_assignment_strategy'], + **self.config) self._closed = False #self.metrics = None @@ -213,11 +210,11 @@ class KafkaConsumer(object): #self.metrics.close() self._client.close() try: - self._key_deserializer.close() + self.config['key_deserializer'].close() except AttributeError: pass try: - self._value_deserializer.close() + self.config['value_deserializer'].close() except AttributeError: pass log.debug("The KafkaConsumer has closed.") diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 03302a3..ea5cb97 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -1,4 +1,5 @@ import abc +import copy import logging import time @@ -44,22 +45,24 @@ class AbstractCoordinator(object): _on_join_complete(). """ - _session_timeout_ms = 30000 - _heartbeat_interval_ms = 3000 - _retry_backoff_ms = 100 + DEFAULT_CONFIG = { + 'session_timeout_ms': 30000, + 'heartbeat_interval_ms': 3000, + 'retry_backoff_ms': 100, + } - def __init__(self, client, group_id, **kwargs): + def __init__(self, client, group_id, **configs): if not client: raise Errors.IllegalStateError('a client is required to use' ' Group Coordinator') if not group_id: raise Errors.IllegalStateError('a group_id is required to use' ' Group Coordinator') - for config in ('session_timeout_ms', - 'heartbeat_interval_ms', - 'retry_backoff_ms'): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) + + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] self._client = client self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID @@ -68,9 +71,7 @@ class AbstractCoordinator(object): self.coordinator_id = None self.rejoin_needed = True self.needs_join_prepare = True - self.heartbeat = Heartbeat( - session_timeout_ms=self._session_timeout_ms, - heartbeat_interval_ms=self._heartbeat_interval_ms) + self.heartbeat = Heartbeat(**self.config) self.heartbeat_task = HeartbeatTask(self) #self.sensors = GroupCoordinatorMetrics(metrics, metric_group_prefix, metric_tags) @@ -222,7 +223,7 @@ class AbstractCoordinator(object): continue elif not future.retriable(): raise exception # pylint: disable-msg=raising-bad-type - time.sleep(self._retry_backoff_ms / 1000.0) + time.sleep(self.config['retry_backoff_ms'] / 1000.0) def _perform_group_join(self): """Join the group and return the assignment for the next generation. @@ -242,7 +243,7 @@ class AbstractCoordinator(object): log.debug("(Re-)joining group %s", self.group_id) request = JoinGroupRequest( self.group_id, - self._session_timeout_ms, + self.config['session_timeout_ms'], self.member_id, self.protocol_type(), [(protocol, @@ -492,8 +493,7 @@ class AbstractCoordinator(object): def _send_heartbeat_request(self): """Send a heartbeat request""" request = HeartbeatRequest(self.group_id, self.generation, self.member_id) - log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, - request.member_id) + log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, request.member_id) #pylint: disable-msg=no-member future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_heartbeat_response, future) @@ -594,7 +594,7 @@ class HeartbeatTask(object): def _handle_heartbeat_failure(self, e): log.debug("Heartbeat failed; retrying") self._request_in_flight = False - etd = time.time() + self._coordinator._retry_backoff_ms / 1000.0 + etd = time.time() + self._coordinator.config['retry_backoff_ms'] / 1000.0 self._client.schedule(self, etd) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 211d1d0..dd3eea0 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -1,3 +1,4 @@ +import copy import collections import logging import time @@ -45,34 +46,36 @@ class ConsumerProtocol(object): class ConsumerCoordinator(AbstractCoordinator): """This class manages the coordination process with the consumer coordinator.""" - _enable_auto_commit = True - _auto_commit_interval_ms = 5000 - _default_offset_commit_callback = lambda offsets, error: True - _assignors = () - #_heartbeat_interval_ms = 3000 - #_session_timeout_ms = 30000 - #_retry_backoff_ms = 100 - - def __init__(self, client, group_id, subscription, **kwargs): + DEFAULT_CONFIG = { + 'enable_auto_commit': True, + 'auto_commit_interval_ms': 5000, + 'default_offset_commit_callback': lambda offsets, error: True, + 'assignors': (), + 'session_timeout_ms': 30000, + 'heartbeat_interval_ms': 3000, + 'retry_backoff_ms': 100, + } + + def __init__(self, client, group_id, subscription, **configs): """Initialize the coordination manager.""" - super(ConsumerCoordinator, self).__init__(client, group_id, **kwargs) - for config in ('enable_auto_commit', 'auto_commit_interval_ms', - 'default_offset_commit_callback', 'assignors'): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) + super(ConsumerCoordinator, self).__init__(client, group_id, **configs) + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] self._cluster = client.cluster self._subscription = subscription self._partitions_per_topic = {} self._auto_commit_task = None - if not self._assignors: + if not self.config['assignors']: raise Errors.IllegalStateError('Coordinator requires assignors') self._cluster.request_update() self._cluster.add_listener(self._handle_metadata_update) - if self._enable_auto_commit: - interval = self._auto_commit_interval_ms / 1000.0 + if self.config['enable_auto_commit']: + interval = self.config['auto_commit_interval_ms'] / 1000.0 self._auto_commit_task = AutoCommitTask(self, interval) # metrics=None, @@ -87,7 +90,7 @@ class ConsumerCoordinator(AbstractCoordinator): """Returns list of preferred (protocols, metadata)""" topics = self._subscription.subscription metadata_list = [] - for assignor in self._assignors: + for assignor in self.config['assignors']: metadata = assignor.metadata(topics) group_protocol = (assignor.name, metadata) metadata_list.append(group_protocol) @@ -126,7 +129,7 @@ class ConsumerCoordinator(AbstractCoordinator): return False def _lookup_assignor(self, name): - for assignor in self._assignors: + for assignor in self.config['assignors']: if assignor.name == name: return assignor return None @@ -152,7 +155,7 @@ class ConsumerCoordinator(AbstractCoordinator): assignor.on_assignment(assignment) # restart the autocommit task if needed - if self._enable_auto_commit: + if self.config['enable_auto_commit']: self._auto_commit_task.enable() assigned = set(self._subscription.assigned_partitions()) @@ -258,7 +261,7 @@ class ConsumerCoordinator(AbstractCoordinator): if not future.retriable(): raise future.exception # pylint: disable-msg=raising-bad-type - time.sleep(self._retry_backoff_ms / 1000.0) + time.sleep(self.config['retry_backoff_ms'] / 1000.0) def ensure_partition_assignment(self): """Ensure that we have a valid partition assignment from the coordinator.""" @@ -283,10 +286,11 @@ class ConsumerCoordinator(AbstractCoordinator): Returns: Future: indicating whether the commit was successful or not """ + if callback is None: + callback = self.config['default_offset_commit_callback'] self._subscription.needs_fetch_committed_offsets = True future = self._send_offset_commit_request(offsets) - cb = callback if callback else self._default_offset_commit_callback - future.add_both(cb, offsets) + future.add_both(callback, offsets) def commit_offsets_sync(self, offsets): """Commit specific offsets synchronously. @@ -314,10 +318,10 @@ class ConsumerCoordinator(AbstractCoordinator): if not future.retriable(): raise future.exception # pylint: disable-msg=raising-bad-type - time.sleep(self._retry_backoff_ms / 1000.0) + time.sleep(self.config['retry_backoff_ms'] / 1000.0) def _maybe_auto_commit_offsets_sync(self): - if self._enable_auto_commit: + if self.config['enable_auto_commit']: # disable periodic commits prior to committing synchronously. note that they will # be re-enabled after a rebalance completes self._auto_commit_task.disable() @@ -558,8 +562,8 @@ class AutoCommitTask(object): if self._coordinator.coordinator_unknown(): log.debug("Cannot auto-commit offsets because the coordinator is" " unknown, will retry after backoff") - next_at = time.time() + self._coordinator._retry_backoff_ms / 1000.0 - self._client.schedule(self, next_at) + backoff = self._coordinator.config['retry_backoff_ms'] / 1000.0 + self._client.schedule(self, time.time() + backoff) return self._request_in_flight = True diff --git a/kafka/coordinator/heartbeat.py b/kafka/coordinator/heartbeat.py index c153ddd..41ba025 100644 --- a/kafka/coordinator/heartbeat.py +++ b/kafka/coordinator/heartbeat.py @@ -1,23 +1,27 @@ +import copy import time import kafka.common as Errors class Heartbeat(object): - _heartbeat_interval_ms = 3000 - _session_timeout_ms = 30000 - - def __init__(self, **kwargs): - for config in ('heartbeat_interval_ms', 'session_timeout_ms'): - if config in kwargs: - setattr(self, '_' + config, kwargs.pop(config)) - - if self._heartbeat_interval_ms > self._session_timeout_ms: + DEFAULT_CONFIG = { + 'heartbeat_interval_ms': 3000, + 'session_timeout_ms': 30000, + } + + def __init__(self, **configs): + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] + + if self.config['heartbeat_interval_ms'] > self.config['session_timeout_ms']: raise Errors.IllegalArgumentError("Heartbeat interval must be set" " lower than the session timeout") - self.interval = self._heartbeat_interval_ms / 1000.0 - self.timeout = self._session_timeout_ms / 1000.0 + self.interval = self.config['heartbeat_interval_ms'] / 1000.0 + self.timeout = self.config['session_timeout_ms'] / 1000.0 self.last_send = 0 self.last_receive = 0 self.last_reset = time.time() -- cgit v1.2.1 From 88cf1b5e4551cd96322aa812fa482bf0f978060a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 11:48:08 -0800 Subject: Remove unused connection_delay method from KafkaClient --- kafka/client_async.py | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 6fb5fdd..386b0cb 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -162,25 +162,7 @@ class KafkaClient(object): log.warning("Node %s not found in current connection list; skipping", node_id) return - def connection_delay(self, node_id): - """ - Returns the number of milliseconds to wait, based on the connection - state, before attempting to send data. When disconnected, this respects - the reconnect backoff time. When connecting or connected, this handles - slow/stalled connections. - - @param node_id The id of the node to check - @return The number of milliseconds to wait. - """ - if node_id not in self._conns: - return 0 - conn = self._conns[node_id] - time_waited_ms = time.time() - (conn.last_attempt or 0) - if conn.state is ConnectionStates.DISCONNECTED: - return max(self.config['reconnect_backoff_ms'] - time_waited_ms, 0) - else: - return sys.maxint def connection_failed(self, node_id): """ -- cgit v1.2.1 From 9bc01657ed9402b502f7156ae95764029436eab3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 11:49:22 -0800 Subject: Resolve delayed task futures in KafkaClient.poll --- kafka/client_async.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 386b0cb..ca81214 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -243,11 +243,14 @@ class KafkaClient(object): metadata_timeout = self._maybe_refresh_metadata() # Send scheduled tasks - for task in self._delayed_tasks.pop_ready(): + for task, future in self._delayed_tasks.pop_ready(): try: - task() + result = task() except Exception as e: log.error("Task %s failed: %s", task, e) + future.failure(e) + else: + future.success(result) timeout = min(timeout_ms, metadata_timeout, self.config['request_timeout_ms']) @@ -450,7 +453,7 @@ class DelayedTaskQueue(object): else: task, future = maybe_task del self._task_map[task] - return task + return (task, future) def next_at(self): """Number of seconds until next task is ready""" -- cgit v1.2.1 From cfae9e3fa3432fad6bbd97c1d81f8ea4bc0ee363 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 11:50:42 -0800 Subject: Remove unnecessary calls in KafkaClient._poll - Dont process connections; outer poll() loop does this now - Only recv connections that select says are readable --- kafka/client_async.py | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index ca81214..eaa5ef0 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -264,22 +264,22 @@ class KafkaClient(object): def _poll(self, timeout): # select on reads across all connected sockets, blocking up to timeout - sockets = [conn._sock for conn in six.itervalues(self._conns) - if (conn.state is ConnectionStates.CONNECTED and - conn.in_flight_requests)] - if sockets: - select.select(sockets, [], [], timeout) + sockets = dict([(conn._sock, conn) + for conn in six.itervalues(self._conns) + if (conn.state is ConnectionStates.CONNECTED + and conn.in_flight_requests)]) + if not sockets: + return [] + + ready, _, _ = select.select(list(sockets.keys()), [], [], timeout) responses = [] # list, not iterator, because inline callbacks may add to self._conns - for conn in list(self._conns.values()): - if conn.state is ConnectionStates.CONNECTING: - conn.connect() - - if conn.in_flight_requests: - response = conn.recv() # This will run callbacks / errbacks - if response: - responses.append(response) + for sock in ready: + conn = sockets[sock] + response = conn.recv() # Note: conn.recv runs callbacks / errbacks + if response: + responses.append(response) return responses def in_flight_request_count(self, node_id=None): -- cgit v1.2.1 From 61161d8335bba3f1786f10a321a2a9e915adb286 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:18:02 -0800 Subject: Improve removed tasks handing in DelayedTaskQueue.pop_ready --- kafka/client_async.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index eaa5ef0..e2e68a8 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -465,8 +465,11 @@ class DelayedTaskQueue(object): def pop_ready(self): """Pop and return a list of all ready (task, future) tuples""" - self._drop_removed() ready_tasks = [] while self._tasks and self._tasks[0][0] < time.time(): - ready_tasks.append(self._pop_next()) + try: + task = self._pop_next() + except KeyError: + break + ready_tasks.append(task) return ready_tasks -- cgit v1.2.1 From f161cd11aa63715979c544ddcaeeb0fabe57b3dd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:21:00 -0800 Subject: Make group_id a Coordinator kwarg (not arg) for consistency --- kafka/consumer/group.py | 2 +- kafka/coordinator/abstract.py | 8 +++----- kafka/coordinator/consumer.py | 4 ++-- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index b7093f3..0814983 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -154,7 +154,7 @@ class KafkaConsumer(object): self._fetcher = Fetcher( self._client, self._subscription, **self.config) self._coordinator = ConsumerCoordinator( - self._client, self.config['group_id'], self._subscription, + self._client, self._subscription, assignors=self.config['partition_assignment_strategy'], **self.config) self._closed = False diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index ea5cb97..6790bb1 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -46,18 +46,16 @@ class AbstractCoordinator(object): """ DEFAULT_CONFIG = { + 'group_id': 'kafka-python-default-group', 'session_timeout_ms': 30000, 'heartbeat_interval_ms': 3000, 'retry_backoff_ms': 100, } - def __init__(self, client, group_id, **configs): + def __init__(self, client, **configs): if not client: raise Errors.IllegalStateError('a client is required to use' ' Group Coordinator') - if not group_id: - raise Errors.IllegalStateError('a group_id is required to use' - ' Group Coordinator') self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: @@ -67,7 +65,7 @@ class AbstractCoordinator(object): self._client = client self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID - self.group_id = group_id + self.group_id = self.config['group_id'] self.coordinator_id = None self.rejoin_needed = True self.needs_join_prepare = True diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index dd3eea0..3d5669e 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -47,6 +47,7 @@ class ConsumerProtocol(object): class ConsumerCoordinator(AbstractCoordinator): """This class manages the coordination process with the consumer coordinator.""" DEFAULT_CONFIG = { + 'group_id': 'kafka-python-default-group', 'enable_auto_commit': True, 'auto_commit_interval_ms': 5000, 'default_offset_commit_callback': lambda offsets, error: True, @@ -56,9 +57,8 @@ class ConsumerCoordinator(AbstractCoordinator): 'retry_backoff_ms': 100, } - def __init__(self, client, group_id, subscription, **configs): """Initialize the coordination manager.""" - super(ConsumerCoordinator, self).__init__(client, group_id, **configs) + super(ConsumerCoordinator, self).__init__(client, **configs) self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: -- cgit v1.2.1 From 8dcfa9654237d8f076b355d2e3647b9b109aa5c5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:21:50 -0800 Subject: Drop unused KafkaConsumer._ensure_not_closed method --- kafka/consumer/group.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 0814983..00955f8 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -300,10 +300,6 @@ class KafkaConsumer(object): committed = None return committed - def _ensure_not_closed(self): - if self._closed: - raise Errors.IllegalStateError("This consumer has already been closed.") - def topics(self): """Get all topic metadata topics the user is authorized to view. -- cgit v1.2.1 From 86c89cdaff0785040d43f5b6ff980bb046c782ef Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:23:42 -0800 Subject: Rename KafkaClient.connection_failed -> is_disconnected --- kafka/client_async.py | 2 +- kafka/coordinator/abstract.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index e2e68a8..06439fc 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -162,9 +162,9 @@ class KafkaClient(object): log.warning("Node %s not found in current connection list; skipping", node_id) return + def is_disconnected(self, node_id): - def connection_failed(self, node_id): """ Check if the connection of the node has failed, based on the connection state. Such connection failures are usually transient and can be resumed diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 6790bb1..89996c8 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -157,7 +157,7 @@ class AbstractCoordinator(object): if self.coordinator_id is None: return True - if self._client.connection_failed(self.coordinator_id): + if self._client.is_disconnected(self.coordinator_id): self.coordinator_dead() return True -- cgit v1.2.1 From b6a2ad9caa8d7b5b87d3808650376e7751d4e4da Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:45:24 -0800 Subject: BrokerConnection: wrap socket errors in retriable ConnectionError --- kafka/conn.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 8ce4a6f..3e49841 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -165,7 +165,7 @@ class BrokerConnection(object): self._sock.setblocking(False) except (AssertionError, socket.error) as e: log.exception("Error sending %s to %s", request, self) - self.close(error=e) + self.close(error=Errors.ConnectionError(e)) return future.failure(e) log.debug('%s Request %d: %s', self, correlation_id, request) @@ -230,7 +230,7 @@ class BrokerConnection(object): return None log.exception('%s: Error receiving 4-byte payload header -' ' closing socket', self) - self.close(error=e) + self.close(error=Errors.ConnectionError(e)) return None if self._rbuffer.tell() == 4: @@ -253,7 +253,7 @@ class BrokerConnection(object): if e.errno == errno.EWOULDBLOCK: return None log.exception('%s: Error in recv', self) - self.close(error=e) + self.close(error=Errors.ConnectionError(e)) return None staged_bytes = self._rbuffer.tell() -- cgit v1.2.1 From 1dd9e8bb05b6efc2888ac4cae8e7199b35dd633f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:46:42 -0800 Subject: Fix blacked_out typo in least_loaded_node --- kafka/client_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 06439fc..e8ab961 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -324,7 +324,7 @@ class KafkaClient(object): # if we found no connected node, return a disconnected one log.debug("No connected nodes found. Trying disconnected nodes.") for node_id in nodes: - if not self._conns[node_id].is_blacked_out(): + if not self._conns[node_id].blacked_out(): return node_id # if still no luck, look for a node not in self._conns yet -- cgit v1.2.1 From e093ffefaecb59c26f2e480214f72a03ba5a49fc Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 12:51:34 -0800 Subject: More Docstring Improvements --- kafka/client_async.py | 191 ++++++++++++++++++++++---------- kafka/consumer/fetcher.py | 133 +++++++++++++++------- kafka/consumer/group.py | 10 +- kafka/consumer/subscription_state.py | 113 +++++++++++++++---- kafka/coordinator/abstract.py | 74 +++++++++---- kafka/coordinator/assignors/abstract.py | 27 ++++- kafka/coordinator/consumer.py | 35 +++++- 7 files changed, 431 insertions(+), 152 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index e8ab961..87d616c 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -40,6 +40,33 @@ class KafkaClient(object): } def __init__(self, **configs): + """Initialize an asynchronous kafka client + + Keyword Arguments: + bootstrap_servers: 'host[:port]' string (or list of 'host[:port]' + strings) that the consumer should contact to bootstrap initial + cluster metadata. This does not have to be the full node list. + It just needs to have at least one broker that will respond to a + Metadata API Request. Default port is 9092. If no servers are + specified, will default to localhost:9092. + client_id (str): a name for this client. This string is passed in + each request to servers and can be used to identify specific + server-side log entries that correspond to this client. Also + submitted to GroupCoordinator for logging with respect to + consumer group administration. Default: 'kafka-python-{version}' + request_timeout_ms (int): Client request timeout in milliseconds. + Default: 40000. + reconnect_backoff_ms (int): The amount of time in milliseconds to + wait before attempting to reconnect to a given host. + Default: 50. + max_in_flight_requests_per_connection (int): Requests are pipelined + to kafka brokers up to this number of maximum requests per + broker connection. Default: 5. + send_buffer_bytes (int): The size of the TCP send buffer + (SO_SNDBUF) to use when sending data. Default: 131072 + receive_buffer_bytes (int): The size of the TCP receive buffer + (SO_RCVBUF) to use when reading data. Default: 32768 + """ self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: @@ -128,12 +155,13 @@ class KafkaClient(object): return state def ready(self, node_id): - """ - Begin connecting to the given node, return true if we are already - connected and ready to send to that node. + """Check whether a node is connected and ok to send more requests. - @param node_id The id of the node to check - @return True if we are ready to send to the given node + Arguments: + node_id (int): the id of the node to check + + Returns: + bool: True if we are ready to send to the given node """ if self.is_ready(node_id): return True @@ -151,7 +179,8 @@ class KafkaClient(object): def close(self, node_id=None): """Closes the connection to a particular node (if there is one). - @param node_id The id of the node + Arguments: + node_id (int): the id of the node to close """ if node_id is None: for conn in self._conns.values(): @@ -163,27 +192,34 @@ class KafkaClient(object): return def is_disconnected(self, node_id): + """Check whether the node connection has been disconnected failed. + A disconnected node has either been closed or has failed. Connection + failures are usually transient and can be resumed in the next ready() + call, but there are cases where transient failures need to be caught + and re-acted upon. - """ - Check if the connection of the node has failed, based on the connection - state. Such connection failures are usually transient and can be resumed - in the next ready(node) call, but there are cases where transient - failures need to be caught and re-acted upon. + Arguments: + node_id (int): the id of the node to check - @param node_id the id of the node to check - @return true iff the connection has failed and the node is disconnected + Returns: + bool: True iff the node exists and is disconnected """ if node_id not in self._conns: return False return self._conns[node_id].state is ConnectionStates.DISCONNECTED def is_ready(self, node_id): - """ - Check if the node with the given id is ready to send more requests. + """Check whether a node is ready to send more requests. + + In addition to connection-level checks, this method also is used to + block additional requests from being sent during a metadata refresh. + + Arguments: + node_id (int): id of the node to check - @param node_id The id of the node - @return true if the node is ready + Returns: + bool: True if the node is ready and metadata is not refreshing """ # if we need to update our metadata now declare all requests unready to # make metadata requests first priority @@ -199,12 +235,17 @@ class KafkaClient(object): return conn.connected() and conn.can_send_more() def send(self, node_id, request): - """ - Send the given request. Requests can only be sent out to ready nodes. + """Send a request to a specific node. + + Arguments: + node_id (int): destination node + request (Struct): request object (not-encoded) - @param node destination node - @param request The request - @param now The current timestamp + Raises: + IllegalStateError: if node_id is not ready + + Returns: + Future: resolves to Response struct """ if not self._can_send_request(node_id): raise Errors.IllegalStateError("Attempt to send a request to node %s which is not ready." % node_id) @@ -217,15 +258,20 @@ class KafkaClient(object): return self._conns[node_id].send(request, expect_response=expect_response) def poll(self, timeout_ms=None, future=None): - """Do actual reads and writes to sockets. - - @param timeout_ms The maximum amount of time to wait (in ms) for - responses if there are none available immediately. - Must be non-negative. The actual timeout will be the - minimum of timeout, request timeout and metadata - timeout. If unspecified, default to request_timeout_ms - @param future Optionally block until the provided future completes. - @return The list of responses received. + """Try to read and write to sockets. + + This method will also attempt to complete node connections, refresh + stale metadata, and run previously-scheduled tasks. + + Arguments: + timeout_ms (int, optional): maximum amount of time to wait (in ms) + for at least one response. Must be non-negative. The actual + timeout will be the minimum of timeout, request timeout and + metadata timeout. Default: request_timeout_ms + future (Future, optional): if provided, blocks until future.is_done + + Returns: + list: responses received (can be empty) """ if timeout_ms is None: timeout_ms = self.config['request_timeout_ms'] @@ -283,7 +329,15 @@ class KafkaClient(object): return responses def in_flight_request_count(self, node_id=None): - """Get the number of in-flight requests""" + """Get the number of in-flight requests for a node or all nodes. + + Arguments: + node_id (int, optional): a specific node to check. If unspecified, + return the total for all nodes + + Returns: + int: pending in-flight requests for the node, or all nodes if None + """ if node_id is not None: if node_id not in self._conns: return 0 @@ -292,16 +346,17 @@ class KafkaClient(object): return sum([len(conn.in_flight_requests) for conn in self._conns.values()]) def least_loaded_node(self): - """ - Choose the node with the fewest outstanding requests which is at least - eligible for connection. This method will prefer a node with an - existing connection, but will potentially choose a node for which we - don't yet have a connection if all existing connections are in use. - This method will never choose a node for which there is no existing - connection and from which we have disconnected within the reconnect - backoff period. - - @return The node_id with the fewest in-flight requests. + """Choose the node with fewest outstanding requests, with fallbacks. + + This method will prefer a node with an existing connection, but will + potentially choose a node for which we don't yet have a connection if + all existing connections are in use. This method will never choose a + node that was disconnected within the reconnect backoff period. + If all else fails, the method will attempt to bootstrap again using the + bootstrap_servers list. + + Returns: + node_id or None if no suitable node was found """ nodes = list(self._conns.keys()) random.shuffle(nodes) @@ -339,10 +394,13 @@ class KafkaClient(object): return None def set_topics(self, topics): - """ - Set specific topics to track for metadata + """Set specific topics to track for metadata. + + Arguments: + topics (list of str): topics to check for metadata - Returns a future that will complete after metadata request/response + Returns: + Future: resolves after metadata request/response """ if set(topics).difference(self._topics): future = self.cluster.request_update() @@ -353,7 +411,11 @@ class KafkaClient(object): # request metadata update on disconnect and timedout def _maybe_refresh_metadata(self): - """Send a metadata request if needed""" + """Send a metadata request if needed. + + Returns: + int: milliseconds until next refresh + """ ttl = self.cluster.ttl() if ttl > 0: return ttl @@ -383,26 +445,30 @@ class KafkaClient(object): return 0 def schedule(self, task, at): - """ - Schedule a new task to be executed at the given time. + """Schedule a new task to be executed at the given time. This is "best-effort" scheduling and should only be used for coarse synchronization. A task cannot be scheduled for multiple times simultaneously; any previously scheduled instance of the same task will be cancelled. - @param task The task to be scheduled -- function or implement __call__ - @param at Epoch seconds when it should run (see time.time()) - @returns Future + Arguments: + task (callable): task to be scheduled + at (float or int): epoch seconds when task should run + + Returns: + Future: resolves to result of task call, or exception if raised """ return self._delayed_tasks.add(task, at) def unschedule(self, task): - """ - Unschedule a task. This will remove all instances of the task from the task queue. + """Unschedule a task. + + This will remove all instances of the task from the task queue. This is a no-op if the task is not scheduled. - @param task The task to be unscheduled. + Arguments: + task (callable): task to be unscheduled """ self._delayed_tasks.remove(task) @@ -415,10 +481,14 @@ class DelayedTaskQueue(object): self._counter = itertools.count() # unique sequence count def add(self, task, at): - """Add a task to run at a later time + """Add a task to run at a later time. + + Arguments: + task: can be anything, but generally a callable + at (float or int): epoch seconds to schedule task - task: anything - at: seconds from epoch to schedule task (see time.time()) + Returns: + Future: a future that will be returned with the task when ready """ if task in self._task_map: self.remove(task) @@ -430,9 +500,10 @@ class DelayedTaskQueue(object): return future def remove(self, task): - """Remove a previously scheduled task + """Remove a previously scheduled task. - Raises KeyError if task is not found + Raises: + KeyError: if task is not found """ entry = self._task_map.pop(task) task, future = entry[-1] @@ -456,7 +527,7 @@ class DelayedTaskQueue(object): return (task, future) def next_at(self): - """Number of seconds until next task is ready""" + """Number of seconds until next task is ready.""" self._drop_removed() if not self._tasks: return sys.maxint diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 39e1244..a4be7ae 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -39,6 +39,33 @@ class Fetcher(object): } def __init__(self, client, subscriptions, **configs): + """Initialize a Kafka Message Fetcher. + + Keyword Arguments: + key_deserializer (callable): Any callable that takes a + raw message key and returns a deserialized key. + value_deserializer (callable, optional): Any callable that takes a + raw message value and returns a deserialized value. + fetch_min_bytes (int): Minimum amount of data the server should + return for a fetch request, otherwise wait up to + fetch_max_wait_ms for more data to accumulate. Default: 1024. + fetch_max_wait_ms (int): The maximum amount of time in milliseconds + the server will block before answering the fetch request if + there isn't sufficient data to immediately satisfy the + requirement given by fetch_min_bytes. Default: 500. + max_partition_fetch_bytes (int): The maximum amount of data + per-partition the server will return. The maximum total memory + used for a request = #partitions * max_partition_fetch_bytes. + This size must be at least as large as the maximum message size + the server allows or else it is possible for the producer to + send messages larger than the consumer can fetch. If that + happens, the consumer can get stuck trying to fetch a large + message on a certain partition. Default: 1048576. + check_crcs (bool): Automatically check the CRC32 of the records + consumed. This ensures no on-the-wire or on-disk corruption to + the messages occurred. This check adds some overhead, so it may + be disabled in cases seeking extreme performance. Default: True + """ #metrics=None, #metric_group_prefix='consumer', self.config = copy.copy(self.DEFAULT_CONFIG) @@ -56,7 +83,11 @@ class Fetcher(object): #self.sensors = FetchManagerMetrics(metrics, metric_group_prefix) def init_fetches(self): - """Send FetchRequests asynchronously for all assigned partitions""" + """Send FetchRequests asynchronously for all assigned partitions. + + Returns: + List of Futures: each future resolves to a FetchResponse + """ futures = [] for node_id, request in six.iteritems(self._create_fetch_requests()): if self._client.ready(node_id): @@ -70,8 +101,11 @@ class Fetcher(object): def update_fetch_positions(self, partitions): """Update the fetch positions for the provided partitions. - @param partitions: iterable of TopicPartitions - @raises NoOffsetForPartitionError If no offset is stored for a given + Arguments: + partitions (list of TopicPartitions): partitions to update + + Raises: + NoOffsetForPartitionError: if no offset is stored for a given partition and no reset policy is available """ # reset the fetch position to the committed position @@ -104,8 +138,11 @@ class Fetcher(object): def _reset_offset(self, partition): """Reset offsets for the given partition using the offset reset strategy. - @param partition The given partition that needs reset offset - @raises NoOffsetForPartitionError If no offset reset strategy is defined + Arguments: + partition (TopicPartition): the partition that needs reset offset + + Raises: + NoOffsetForPartitionError: if no offset reset strategy is defined """ timestamp = self._subscriptions.assignment[partition].reset_strategy if timestamp is OffsetResetStrategy.EARLIEST: @@ -129,11 +166,14 @@ class Fetcher(object): Blocks until offset is obtained, or a non-retriable exception is raised - @param partition The partition that needs fetching offset. - @param timestamp The timestamp for fetching offset. - @raises exceptions - @return The offset of the message that is published before the given - timestamp + Arguments: + partition The partition that needs fetching offset. + timestamp (int): timestamp for fetching offset. -1 for the latest + available, -2 for the earliest available. Otherwise timestamp + is treated as epoch seconds. + + Returns: + int: message offset """ while True: future = self._send_offset_request(partition, timestamp) @@ -150,10 +190,12 @@ class Fetcher(object): self._client.poll(future=refresh_future) def _raise_if_offset_out_of_range(self): - """ - If any partition from previous FetchResponse contains - OffsetOutOfRangeError and the default_reset_policy is None, - raise OffsetOutOfRangeError + """Check FetchResponses for offset out of range. + + Raises: + OffsetOutOfRangeError: if any partition from previous FetchResponse + contains OffsetOutOfRangeError and the default_reset_policy is + None """ current_out_of_range_partitions = {} @@ -174,11 +216,10 @@ class Fetcher(object): raise Errors.OffsetOutOfRangeError(current_out_of_range_partitions) def _raise_if_unauthorized_topics(self): - """ - If any topic from previous FetchResponse contains an Authorization - error, raise an exception + """Check FetchResponses for topic authorization failures. - @raise TopicAuthorizationFailedError + Raises: + TopicAuthorizationFailedError """ if self._unauthorized_topics: topics = set(self._unauthorized_topics) @@ -186,12 +227,10 @@ class Fetcher(object): raise Errors.TopicAuthorizationFailedError(topics) def _raise_if_record_too_large(self): - """ - If any partition from previous FetchResponse gets a RecordTooLarge - error, raise RecordTooLargeError + """Check FetchResponses for messages larger than the max per partition. - @raise RecordTooLargeError If there is a message larger than fetch size - and hence cannot be ever returned + Raises: + RecordTooLargeError: if there is a message larger than fetch size """ copied_record_too_large_partitions = dict(self._record_too_large_partitions) self._record_too_large_partitions.clear() @@ -207,12 +246,21 @@ class Fetcher(object): self.config['max_partition_fetch_bytes']) def fetched_records(self): - """Returns previously fetched records and updates consumed offsets + """Returns previously fetched records and updates consumed offsets. NOTE: returning empty records guarantees the consumed position are NOT updated. - @return {TopicPartition: deque([messages])} - @raises OffsetOutOfRangeError if no subscription offset_reset_strategy + Raises: + OffsetOutOfRangeError: if no subscription offset_reset_strategy + InvalidMessageError: if message crc validation fails (check_crcs + must be set to True) + RecordTooLargeError: if a message is larger than the currently + configured max_partition_fetch_bytes + TopicAuthorizationError: if consumer is not authorized to fetch + messages from the topic + + Returns: + dict: {TopicPartition: deque([messages])} """ if self._subscriptions.needs_partition_assignment: return {} @@ -280,12 +328,14 @@ class Fetcher(object): return key, value def _send_offset_request(self, partition, timestamp): - """ - Fetch a single offset before the given timestamp for the partition. + """Fetch a single offset before the given timestamp for the partition. - @param partition The TopicPartition that needs fetching offset. - @param timestamp The timestamp for fetching offset. - @return A future which can be polled to obtain the corresponding offset. + Arguments: + partition (TopicPartition): partition that needs fetching offset + timestamp (int): timestamp for fetching offset + + Returns: + Future: resolves to the corresponding offset """ node_id = self._client.cluster.leader_for_partition(partition) if node_id is None: @@ -315,11 +365,13 @@ class Fetcher(object): def _handle_offset_response(self, partition, future, response): """Callback for the response of the list offset call above. - @param partition The partition that was fetched - @param future the future to update based on response - @param response The OffsetResponse from the server + Arguments: + partition (TopicPartition): The partition that was fetched + future (Future): the future to update based on response + response (OffsetResponse): response from the server - @raises IllegalStateError if response does not match partition + Raises: + IllegalStateError: if response does not match partition """ topic, partition_info = response.topics[0] if len(response.topics) != 1 or len(partition_info) != 1: @@ -351,10 +403,13 @@ class Fetcher(object): future.failure(error_type(partition)) def _create_fetch_requests(self): - """ - Create fetch requests for all assigned partitions, grouped by node - Except where no leader, node has requests in flight, or we have - not returned all previously fetched records to consumer + """Create fetch requests for all assigned partitions, grouped by node. + + FetchRequests skipped if no leader, node has requests in flight, or we + have not returned all previously fetched records to consumer + + Returns: + dict: {node_id: [FetchRequest,...]} """ # create the fetch info as a dict of lists of partition info tuples # which can be passed to FetchRequest() via .items() diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 00955f8..14485d2 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -114,6 +114,10 @@ class KafkaConsumer(object): periodically committed in the background. Default: True. auto_commit_interval_ms (int): milliseconds between automatic offset commits, if enable_auto_commit is True. Default: 5000. + default_offset_commit_callback (callable): called as + callback(offsets, response) response will be either an Exception + or a OffsetCommitResponse struct. This callback can be used to + trigger custom actions when a commit request completes. check_crcs (bool): Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may @@ -438,13 +442,17 @@ class KafkaConsumer(object): self._subscription.resume(partition) def seek(self, partition, offset): - """Manually specify the fetch offset for a TopicPartition + """Manually specify the fetch offset for a TopicPartition. Overrides the fetch offsets that the consumer will use on the next poll(). If this API is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets. + + Arguments: + partition (TopicPartition): partition for seek operation + offset (int): message offset in partition """ if offset < 0: raise Errors.IllegalStateError("seek offset must not be a negative number") diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 38d4571..fa36bc2 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -42,10 +42,10 @@ class SubscriptionState(object): def __init__(self, offset_reset_strategy='earliest'): """Initialize a SubscriptionState instance - offset_reset_strategy: 'earliest' or 'latest', otherwise - exception will be raised when fetching an offset - that is no longer available. - Defaults to earliest. + Keyword Arguments: + offset_reset_strategy: 'earliest' or 'latest', otherwise + exception will be raised when fetching an offset that is no + longer available. Default: 'earliest' """ try: offset_reset_strategy = getattr(OffsetResetStrategy, @@ -67,14 +67,39 @@ class SubscriptionState(object): self.needs_fetch_committed_offsets = True def subscribe(self, topics=(), pattern=None, listener=None): - """Subscribe to a list of topics, or a topic regex pattern + """Subscribe to a list of topics, or a topic regex pattern. - Partitions will be assigned via a group coordinator - (incompatible with assign_from_user) + Partitions will be dynamically assigned via a group coordinator. + Topic subscriptions are not incremental: this list will replace the + current assignment (if there is one). - Optionally include listener callback, which must be a - ConsumerRebalanceListener and will be called before and - after each rebalance operation. + This method is incompatible with assign_from_user() + + Arguments: + topics (list): List of topics for subscription. + pattern (str): Pattern to match available topics. You must provide + either topics or pattern, but not both. + listener (ConsumerRebalanceListener): Optionally include listener + callback, which will be called before and after each rebalance + operation. + + As part of group management, the consumer will keep track of the + list of consumers that belong to a particular group and will + trigger a rebalance operation if one of the following events + trigger: + + * Number of partitions change for any of the subscribed topics + * Topic is created or deleted + * An existing member of the consumer group dies + * A new member is added to the consumer group + + When any of these events are triggered, the provided listener + will be invoked first to indicate that the consumer's assignment + has been revoked, and then again when the new assignment has + been received. Note that this listener will immediately override + any listener set in a previous call to subscribe. It is + guaranteed, however, that the partitions revoked/assigned + through this interface are from topics subscribed in this call. """ if self._user_assignment or (topics and pattern): raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) @@ -93,6 +118,14 @@ class SubscriptionState(object): self.listener = listener def change_subscription(self, topics): + """Change the topic subscription. + + Arguments: + topics (list of str): topics for subscription + + Raises: + IllegalStateErrror: if assign_from_user has been used already + """ if self._user_assignment: raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) @@ -117,7 +150,8 @@ class SubscriptionState(object): This is used by the group leader to ensure that it receives metadata updates for all topics that any member of the group is subscribed to. - @param topics list of topics to add to the group subscription + Arguments: + topics (list of str): topics to add to the group subscription """ if self._user_assignment: raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) @@ -128,12 +162,22 @@ class SubscriptionState(object): self.needs_partition_assignment = True def assign_from_user(self, partitions): - """ - Change the assignment to the specified partitions provided by the user, - note this is different from assign_from_subscribed() - whose input partitions are provided from the subscribed topics. + """Manually assign a list of TopicPartitions to this consumer. + + This interface does not allow for incremental assignment and will + replace the previous assignment (if there was one). - @param partitions: list (or iterable) of TopicPartition() + Manual topic assignment through this method does not use the consumer's + group management functionality. As such, there will be no rebalance + operation triggered when group membership or cluster and topic metadata + change. Note that it is not possible to use both manual partition + assignment with assign() and group assignment with subscribe(). + + Arguments: + partitions (list of TopicPartition): assignment for this instance. + + Raises: + IllegalStateError: if consumer has already called subscribe() """ if self.subscription is not None: raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) @@ -175,6 +219,7 @@ class SubscriptionState(object): log.info("Updated partition assignment: %s", assignments) def unsubscribe(self): + """Clear all topic subscriptions and partition assignments""" self.subscription = None self._user_assignment.clear() self.assignment.clear() @@ -191,17 +236,32 @@ class SubscriptionState(object): that would require rebalancing (the leader fetches metadata for all topics in the group so that it can do partition assignment). - @return set of topics + Returns: + set: topics """ return self._group_subscription def seek(self, partition, offset): + """Manually specify the fetch offset for a TopicPartition. + + Overrides the fetch offsets that the consumer will use on the next + poll(). If this API is invoked for the same partition more than once, + the latest offset will be used on the next poll(). Note that you may + lose data if this API is arbitrarily used in the middle of consumption, + to reset the fetch offsets. + + Arguments: + partition (TopicPartition): partition for seek operation + offset (int): message offset in partition + """ self.assignment[partition].seek(offset) def assigned_partitions(self): + """Return set of TopicPartitions in current assignment.""" return set(self.assignment.keys()) def fetchable_partitions(self): + """Return set of TopicPartitions that should be Fetched.""" fetchable = set() for partition, state in six.iteritems(self.assignment): if state.is_fetchable(): @@ -209,6 +269,7 @@ class SubscriptionState(object): return fetchable def partitions_auto_assigned(self): + """Return True unless user supplied partitions manually.""" return self.subscription is not None def all_consumed_offsets(self): @@ -220,11 +281,18 @@ class SubscriptionState(object): return all_consumed def need_offset_reset(self, partition, offset_reset_strategy=None): + """Mark partition for offset reset using specified or default strategy. + + Arguments: + partition (TopicPartition): partition to mark + offset_reset_strategy (OffsetResetStrategy, optional) + """ if offset_reset_strategy is None: offset_reset_strategy = self._default_offset_reset_strategy self.assignment[partition].await_reset(offset_reset_strategy) def has_default_offset_reset_policy(self): + """Return True if default offset reset policy is Earliest or Latest""" return self._default_offset_reset_strategy != OffsetResetStrategy.NONE def is_offset_reset_needed(self, partition): @@ -372,8 +440,9 @@ class ConsumerRebalanceListener(object): NOTE: This method is only called before rebalances. It is not called prior to KafkaConsumer.close() - @param partitions The list of partitions that were assigned to the - consumer on the last rebalance + Arguments: + revoked (list of TopicPartition): the partitions that were assigned + to the consumer on the last rebalance """ pass @@ -389,8 +458,8 @@ class ConsumerRebalanceListener(object): their on_partitions_revoked() callback before any instance executes its on_partitions_assigned() callback. - @param partitions The list of partitions that are now assigned to the - consumer (may include partitions previously assigned - to the consumer) + Arguments: + assigned (list of TopicPartition): the partitions assigned to the + consumer (may include partitions that were previously assigned) """ pass diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 89996c8..7c16034 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -53,6 +53,25 @@ class AbstractCoordinator(object): } def __init__(self, client, **configs): + """ + Keyword Arguments: + group_id (str): name of the consumer group to join for dynamic + partition assignment (if enabled), and to use for fetching and + committing offsets. Default: 'kafka-python-default-group' + session_timeout_ms (int): The timeout used to detect failures when + using Kafka's group managementment facilities. Default: 30000 + heartbeat_interval_ms (int): The expected time in milliseconds + between heartbeats to the consumer coordinator when using + Kafka's group management feature. Heartbeats are used to ensure + that the consumer's session stays active and to facilitate + rebalancing when new consumers join or leave the group. The + value must be set lower than session_timeout_ms, but typically + should be set no higher than 1/3 of that value. It can be + adjusted even lower to control the expected time for normal + rebalances. Default: 3000 + retry_backoff_ms (int): Milliseconds to backoff when retrying on + errors. Default: 100. + """ if not client: raise Errors.IllegalStateError('a client is required to use' ' Group Coordinator') @@ -79,7 +98,8 @@ class AbstractCoordinator(object): Unique identifier for the class of protocols implements (e.g. "consumer" or "connect"). - @return str protocol type name + Returns: + str: protocol type name """ pass @@ -96,7 +116,8 @@ class AbstractCoordinator(object): Note: metadata must be type bytes or support an encode() method - @return [(protocol, metadata), ...] + Returns: + list: [(protocol, metadata), ...] """ pass @@ -107,9 +128,10 @@ class AbstractCoordinator(object): This is typically used to perform any cleanup from the previous generation (such as committing offsets for the consumer) - @param generation The previous generation or -1 if there was none - @param member_id The identifier of this member in the previous group - or '' if there was none + Arguments: + generation (int): The previous generation or -1 if there was none + member_id (str): The identifier of this member in the previous group + or '' if there was none """ pass @@ -120,14 +142,16 @@ class AbstractCoordinator(object): This is used by the leader to push state to all the members of the group (e.g. to push partition assignments in the case of the new consumer) - @param leader_id: The id of the leader (which is this member) - @param protocol: the chosen group protocol (assignment strategy) - @param members: [(member_id, metadata_bytes)] from JoinGroupResponse. - metadata_bytes are associated with the chosen group - protocol, and the Coordinator subclass is responsible - for decoding metadata_bytes based on that protocol. + Arguments: + leader_id (str): The id of the leader (which is this member) + protocol (str): the chosen group protocol (assignment strategy) + members (list): [(member_id, metadata_bytes)] from + JoinGroupResponse. metadata_bytes are associated with the chosen + group protocol, and the Coordinator subclass is responsible for + decoding metadata_bytes based on that protocol. - @return dict of {member_id: assignment}; assignment must either be bytes + Returns: + dict: {member_id: assignment}; assignment must either be bytes or have an encode() method to convert to bytes """ pass @@ -137,22 +161,23 @@ class AbstractCoordinator(object): member_assignment_bytes): """Invoked when a group member has successfully joined a group. - @param generation The generation that was joined - @param member_id The identifier for the local member in the group - @param protocol The protocol selected by the coordinator - @param member_assignment_bytes The protocol-encoded assignment - propagated from the group leader. The Coordinator instance is - responsible for decoding based on the chosen protocol. + Arguments: + generation (int): the generation that was joined + member_id (str): the identifier for the local member in the group + protocol (str): the protocol selected by the coordinator + member_assignment_bytes (bytes): the protocol-encoded assignment + propagated from the group leader. The Coordinator instance is + responsible for decoding based on the chosen protocol. """ pass def coordinator_unknown(self): - """ - Check if we know who the coordinator is and we have an active connection + """Check if we know who the coordinator is and have an active connection Side-effect: reset coordinator_id to None if connection failed - @return True if the coordinator is unknown + Returns: + bool: True if the coordinator is unknown """ if self.coordinator_id is None: return True @@ -186,9 +211,10 @@ class AbstractCoordinator(object): raise future.exception # pylint: disable-msg=raising-bad-type def need_rejoin(self): - """ - Check whether the group should be rejoined (e.g. if metadata changes) - @return True if it should, False otherwise + """Check whether the group should be rejoined (e.g. if metadata changes) + + Returns: + bool: True if it should, False otherwise """ return self.rejoin_needed diff --git a/kafka/coordinator/assignors/abstract.py b/kafka/coordinator/assignors/abstract.py index ed09a6e..773280a 100644 --- a/kafka/coordinator/assignors/abstract.py +++ b/kafka/coordinator/assignors/abstract.py @@ -19,17 +19,36 @@ class AbstractPartitionAssignor(object): def assign(self, cluster, members): """Perform group assignment given cluster metadata and member subscriptions - @param cluster: cluster metadata - @param members: {member_id: subscription} - @return {member_id: MemberAssignment} + Arguments: + cluster (ClusterMetadata): metadata for use in assignment + members (dict of {member_id: MemberMetadata}): decoded metadata for + each member in the group. + + Returns: + dict: {member_id: MemberAssignment} """ pass @abc.abstractmethod def metadata(self, topics): - """return ProtocolMetadata to be submitted via JoinGroupRequest""" + """Generate ProtocolMetadata to be submitted via JoinGroupRequest. + + Arguments: + topics (set): a member's subscribed topics + + Returns: + MemberMetadata struct + """ pass @abc.abstractmethod def on_assignment(self, assignment): + """Callback that runs on each assignment. + + This method can be used to update internal state, if any, of the + partition assignor. + + Arguments: + assignment (MemberAssignment): the member's assignment + """ pass diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 3d5669e..d5436c4 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -50,14 +50,45 @@ class ConsumerCoordinator(AbstractCoordinator): 'group_id': 'kafka-python-default-group', 'enable_auto_commit': True, 'auto_commit_interval_ms': 5000, - 'default_offset_commit_callback': lambda offsets, error: True, + 'default_offset_commit_callback': lambda offsets, response: True, 'assignors': (), 'session_timeout_ms': 30000, 'heartbeat_interval_ms': 3000, 'retry_backoff_ms': 100, } - """Initialize the coordination manager.""" + def __init__(self, client, subscription, **configs): + """Initialize the coordination manager. + + Keyword Arguments: + group_id (str): name of the consumer group to join for dynamic + partition assignment (if enabled), and to use for fetching and + committing offsets. Default: 'kafka-python-default-group' + enable_auto_commit (bool): If true the consumer's offset will be + periodically committed in the background. Default: True. + auto_commit_interval_ms (int): milliseconds between automatic + offset commits, if enable_auto_commit is True. Default: 5000. + default_offset_commit_callback (callable): called as + callback(offsets, response) response will be either an Exception + or a OffsetCommitResponse struct. This callback can be used to + trigger custom actions when a commit request completes. + assignors (list): List of objects to use to distribute partition + ownership amongst consumer instances when group management is + used. Default: [RoundRobinPartitionAssignor] + heartbeat_interval_ms (int): The expected time in milliseconds + between heartbeats to the consumer coordinator when using + Kafka's group management feature. Heartbeats are used to ensure + that the consumer's session stays active and to facilitate + rebalancing when new consumers join or leave the group. The + value must be set lower than session_timeout_ms, but typically + should be set no higher than 1/3 of that value. It can be + adjusted even lower to control the expected time for normal + rebalances. Default: 3000 + session_timeout_ms (int): The timeout used to detect failures when + using Kafka's group managementment facilities. Default: 30000 + retry_backoff_ms (int): Milliseconds to backoff when retrying on + errors. Default: 100. + """ super(ConsumerCoordinator, self).__init__(client, **configs) self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: -- cgit v1.2.1 From 61c6e1840800eb2108ce2d1dbd78e67a0a5bff4b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 13:05:11 -0800 Subject: Start test fixtures in new session to avoid propagating signals (i.e., Ctrl-C) --- test/service.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test/service.py b/test/service.py index b986a71..ea29c33 100644 --- a/test/service.py +++ b/test/service.py @@ -1,4 +1,5 @@ import logging +import os import re import select import subprocess @@ -52,6 +53,7 @@ class SpawnedService(threading.Thread): self.child = subprocess.Popen( self.args, + preexec_fn=os.setsid, # to avoid propagating signals env=self.env, bufsize=1, stdout=subprocess.PIPE, -- cgit v1.2.1 From 59c051314890a0a6713e6fdb28d74bc3dc053aa9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 13:17:16 -0800 Subject: Catch GroupCoordinatorNotAvailableError in GroupCoordinatorResponse handler --- kafka/coordinator/abstract.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index 7c16034..ca5d38d 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -470,6 +470,9 @@ class AbstractCoordinator(object): if self.generation > 0: self.heartbeat_task.reset() future.success(self.coordinator_id) + elif error_type is Errors.GroupCoordinatorNotAvailableError: + log.debug("Group Coordinator Not Available; retry") + future.failure(error_type()) elif error_type is Errors.GroupAuthorizationFailedError: error = error_type(self.group_id) log.error("Group Coordinator Request failed: %s", error) -- cgit v1.2.1 From 422050f952344e4796725d88db55a983bae4e1ee Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 16:16:13 -0800 Subject: Prefer assert or more-specific error to IllegalState / IllegalArgument --- kafka/client_async.py | 14 +++++++------- kafka/conn.py | 11 +++-------- kafka/consumer/fetcher.py | 20 +++++++------------- kafka/consumer/group.py | 10 ++++------ kafka/consumer/subscription_state.py | 9 +++------ kafka/coordinator/abstract.py | 4 ---- kafka/coordinator/consumer.py | 19 +++++-------------- kafka/coordinator/heartbeat.py | 6 +++--- kafka/future.py | 12 ++++-------- 9 files changed, 36 insertions(+), 69 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 87d616c..d71c9a4 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -131,10 +131,9 @@ class KafkaClient(object): return conn.state is ConnectionStates.DISCONNECTED and not conn.blacked_out() def _initiate_connect(self, node_id): - """Initiate a connection to the given node""" + """Initiate a connection to the given node (must be in metadata)""" broker = self.cluster.broker_metadata(node_id) - if not broker: - raise Errors.IllegalArgumentError('Broker %s not found in current cluster metadata', node_id) + assert broker, 'Broker id %s not in current metadata' % node_id if node_id not in self._conns: log.debug("Initiating connection to node %s at %s:%s", @@ -144,8 +143,7 @@ class KafkaClient(object): return self._finish_connect(node_id) def _finish_connect(self, node_id): - if node_id not in self._conns: - raise Errors.IllegalArgumentError('Node %s not found in connections', node_id) + assert node_id in self._conns, '%s is not in current conns' % node_id state = self._conns[node_id].connect() if state is ConnectionStates.CONNECTING: self._connecting.add(node_id) @@ -242,13 +240,15 @@ class KafkaClient(object): request (Struct): request object (not-encoded) Raises: - IllegalStateError: if node_id is not ready + NodeNotReadyError: if node_id is not ready Returns: Future: resolves to Response struct """ if not self._can_send_request(node_id): - raise Errors.IllegalStateError("Attempt to send a request to node %s which is not ready." % node_id) + raise Errors.NodeNotReadyError("Attempt to send a request to node" + " which is not ready (node id %s)." + % node_id) # Every request gets a response, except one special case: expect_response = True diff --git a/kafka/conn.py b/kafka/conn.py index 3e49841..a1767ef 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -190,9 +190,7 @@ class BrokerConnection(object): Return response if available """ - if self._processing: - raise Errors.IllegalStateError('Recursive connection processing' - ' not supported') + assert not self._processing, 'Recursion not supported' if not self.connected(): log.warning('%s cannot recv: socket not connected', self) # If requests are pending, we should close the socket and @@ -272,11 +270,8 @@ class BrokerConnection(object): return response def _process_response(self, read_buffer): - if self._processing: - raise Errors.IllegalStateError('Recursive connection processing' - ' not supported') - else: - self._processing = True + assert not self._processing, 'Recursion not supported' + self._processing = True ifr = self.in_flight_requests.popleft() # verify send/recv correlation ids match diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index a4be7ae..c133a31 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -371,23 +371,19 @@ class Fetcher(object): response (OffsetResponse): response from the server Raises: - IllegalStateError: if response does not match partition + AssertionError: if response does not match partition """ topic, partition_info = response.topics[0] - if len(response.topics) != 1 or len(partition_info) != 1: - raise Errors.IllegalStateError("OffsetResponse should only be for" - " a single topic-partition") + assert len(response.topics) == 1 and len(partition_info) == 1, ( + 'OffsetResponse should only be for a single topic-partition') part, error_code, offsets = partition_info[0] - if topic != partition.topic or part != partition.partition: - raise Errors.IllegalStateError("OffsetResponse partition does not" - " match OffsetRequest partition") + assert topic == partition.topic and part == partition.partition, ( + 'OffsetResponse partition does not match OffsetRequest partition') error_type = Errors.for_code(error_code) if error_type is Errors.NoError: - if len(offsets) != 1: - raise Errors.IllegalStateError("OffsetResponse should only" - " return a single offset") + assert len(offsets) == 1, 'Expected OffsetResponse with one offset' offset = offsets[0] log.debug("Fetched offset %d for partition %s", offset, partition) future.success(offset) @@ -519,9 +515,7 @@ class Fetcher(object): elif error_type is Errors.UnknownError: log.warn("Unknown error fetching data for topic-partition %s", tp) else: - raise Errors.IllegalStateError("Unexpected error code %s" - " while fetching data" - % error_code) + raise error_type('Unexpected error while fetching data') """TOOD - metrics self.sensors.bytesFetched.record(totalBytes) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 14485d2..90d9d37 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -345,8 +345,7 @@ class KafkaConsumer(object): dict: topic to deque of records since the last fetch for the subscribed list of topics and partitions """ - if timeout_ms < 0: - raise Errors.IllegalArgumentError("Timeout must not be negative") + assert timeout_ms >= 0, 'Timeout must not be negative' # poll for new data until the timeout expires start = time.time() @@ -408,8 +407,8 @@ class KafkaConsumer(object): Arguments: partition (TopicPartition): partition to check """ - if not self._subscription.is_assigned(partition): - raise Errors.IllegalStateError("You can only check the position for partitions assigned to this consumer.") + assert self._subscription.is_assigned(partition) + offset = self._subscription.assignment[partition].consumed if offset is None: self._update_fetch_positions(partition) @@ -454,8 +453,7 @@ class KafkaConsumer(object): partition (TopicPartition): partition for seek operation offset (int): message offset in partition """ - if offset < 0: - raise Errors.IllegalStateError("seek offset must not be a negative number") + assert offset >= 0 log.debug("Seeking to offset %s for partition %s", offset, partition) self._subscription.assignment[partition].seek(offset) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index fa36bc2..c60f192 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -103,8 +103,7 @@ class SubscriptionState(object): """ if self._user_assignment or (topics and pattern): raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) - if not (topics or pattern): - raise IllegalStateError('Must provide topics or a pattern') + assert topics or pattern, 'Must provide topics or pattern' if pattern: log.info('Subscribing to pattern: /%s/', pattern) @@ -341,8 +340,7 @@ class TopicPartitionState(object): self._fetched = None # current fetch position def _set_fetched(self, offset): - if not self.has_valid_position: - raise IllegalStateError("Cannot update fetch position without valid consumed/fetched positions") + assert self.has_valid_position, 'Valid consumed/fetch position required' self._fetched = offset def _get_fetched(self): @@ -351,8 +349,7 @@ class TopicPartitionState(object): fetched = property(_get_fetched, _set_fetched, None, "current fetch position") def _set_consumed(self, offset): - if not self.has_valid_position: - raise IllegalStateError("Cannot update consumed position without valid consumed/fetched positions") + assert self.has_valid_position, 'Valid consumed/fetch position required' self._consumed = offset def _get_consumed(self): diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py index ca5d38d..032ae31 100644 --- a/kafka/coordinator/abstract.py +++ b/kafka/coordinator/abstract.py @@ -72,10 +72,6 @@ class AbstractCoordinator(object): retry_backoff_ms (int): Milliseconds to backoff when retrying on errors. Default: 100. """ - if not client: - raise Errors.IllegalStateError('a client is required to use' - ' Group Coordinator') - self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index d5436c4..7bc10cd 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -99,8 +99,7 @@ class ConsumerCoordinator(AbstractCoordinator): self._subscription = subscription self._partitions_per_topic = {} self._auto_commit_task = None - if not self.config['assignors']: - raise Errors.IllegalStateError('Coordinator requires assignors') + assert self.config['assignors'], 'Coordinator require assignors' self._cluster.request_update() self._cluster.add_listener(self._handle_metadata_update) @@ -168,10 +167,7 @@ class ConsumerCoordinator(AbstractCoordinator): def _on_join_complete(self, generation, member_id, protocol, member_assignment_bytes): assignor = self._lookup_assignor(protocol) - if not assignor: - raise Errors.IllegalStateError("Coordinator selected invalid" - " assignment protocol: %s" - % protocol) + assert assignor, 'invalid assignment protocol: %s' % protocol assignment = ConsumerProtocol.ASSIGNMENT.decode(member_assignment_bytes) @@ -202,10 +198,7 @@ class ConsumerCoordinator(AbstractCoordinator): def _perform_assignment(self, leader_id, assignment_strategy, members): assignor = self._lookup_assignor(assignment_strategy) - if not assignor: - raise Errors.IllegalStateError("Coordinator selected invalid" - " assignment protocol: %s" - % assignment_strategy) + assert assignor, 'Invalid assignment protocol: %s' % assignment_strategy member_metadata = {} all_subscribed_topics = set() for member_id, metadata_bytes in members: @@ -581,10 +574,8 @@ class AutoCommitTask(object): pass def _reschedule(self, at): - if self._enabled: - self._client.schedule(self, at) - else: - raise Errors.IllegalStateError('AutoCommitTask not enabled') + assert self._enabled, 'AutoCommitTask not enabled' + self._client.schedule(self, at) def __call__(self): if not self._enabled: diff --git a/kafka/coordinator/heartbeat.py b/kafka/coordinator/heartbeat.py index 41ba025..9a28f5e 100644 --- a/kafka/coordinator/heartbeat.py +++ b/kafka/coordinator/heartbeat.py @@ -16,9 +16,9 @@ class Heartbeat(object): if key in configs: self.config[key] = configs[key] - if self.config['heartbeat_interval_ms'] > self.config['session_timeout_ms']: - raise Errors.IllegalArgumentError("Heartbeat interval must be set" - " lower than the session timeout") + assert (self.config['heartbeat_interval_ms'] + <= self.config['session_timeout_ms'], + 'Heartbeat interval must be lower than the session timeout') self.interval = self.config['heartbeat_interval_ms'] / 1000.0 self.timeout = self.config['session_timeout_ms'] / 1000.0 diff --git a/kafka/future.py b/kafka/future.py index 1f22cb7..958e85f 100644 --- a/kafka/future.py +++ b/kafka/future.py @@ -27,10 +27,7 @@ class Future(object): return False def success(self, value): - if self.is_done: - raise Errors.IllegalStateError('Invalid attempt to complete a' - ' request future which is already' - ' complete') + assert not self.is_done, 'Future is already complete' self.value = value self.is_done = True for f in self._callbacks: @@ -41,11 +38,10 @@ class Future(object): return self def failure(self, e): - if self.is_done: - raise Errors.IllegalStateError('Invalid attempt to complete a' - ' request future which is already' - ' complete') + assert not self.is_done, 'Future is already complete' self.exception = e if type(e) is not type else e() + assert isinstance(self.exception, BaseException), ( + 'future failed without an exception') self.is_done = True for f in self._errbacks: try: -- cgit v1.2.1 From 93b8afed014f354dd6d348d97dfa2b159c17c5da Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 16:17:25 -0800 Subject: Rename AbstractCoordinator -> BaseCoordinator, log as kafka.coordinator --- kafka/coordinator/abstract.py | 677 ------------------------------------------ kafka/coordinator/base.py | 677 ++++++++++++++++++++++++++++++++++++++++++ kafka/coordinator/consumer.py | 4 +- 3 files changed, 679 insertions(+), 679 deletions(-) delete mode 100644 kafka/coordinator/abstract.py create mode 100644 kafka/coordinator/base.py diff --git a/kafka/coordinator/abstract.py b/kafka/coordinator/abstract.py deleted file mode 100644 index 032ae31..0000000 --- a/kafka/coordinator/abstract.py +++ /dev/null @@ -1,677 +0,0 @@ -import abc -import copy -import logging -import time - -import six - -import kafka.common as Errors -from kafka.future import Future -from kafka.protocol.commit import (GroupCoordinatorRequest, - OffsetCommitRequest_v2 as OffsetCommitRequest) -from kafka.protocol.group import (HeartbeatRequest, JoinGroupRequest, - LeaveGroupRequest, SyncGroupRequest) -from .heartbeat import Heartbeat - -log = logging.getLogger(__name__) - - -class AbstractCoordinator(object): - """ - AbstractCoordinator implements group management for a single group member - by interacting with a designated Kafka broker (the coordinator). Group - semantics are provided by extending this class. See ConsumerCoordinator - for example usage. - - From a high level, Kafka's group management protocol consists of the - following sequence of actions: - - 1. Group Registration: Group members register with the coordinator providing - their own metadata (such as the set of topics they are interested in). - - 2. Group/Leader Selection: The coordinator select the members of the group - and chooses one member as the leader. - - 3. State Assignment: The leader collects the metadata from all the members - of the group and assigns state. - - 4. Group Stabilization: Each member receives the state assigned by the - leader and begins processing. - - To leverage this protocol, an implementation must define the format of - metadata provided by each member for group registration in group_protocols() - and the format of the state assignment provided by the leader in - _perform_assignment() and which becomes available to members in - _on_join_complete(). - """ - - DEFAULT_CONFIG = { - 'group_id': 'kafka-python-default-group', - 'session_timeout_ms': 30000, - 'heartbeat_interval_ms': 3000, - 'retry_backoff_ms': 100, - } - - def __init__(self, client, **configs): - """ - Keyword Arguments: - group_id (str): name of the consumer group to join for dynamic - partition assignment (if enabled), and to use for fetching and - committing offsets. Default: 'kafka-python-default-group' - session_timeout_ms (int): The timeout used to detect failures when - using Kafka's group managementment facilities. Default: 30000 - heartbeat_interval_ms (int): The expected time in milliseconds - between heartbeats to the consumer coordinator when using - Kafka's group management feature. Heartbeats are used to ensure - that the consumer's session stays active and to facilitate - rebalancing when new consumers join or leave the group. The - value must be set lower than session_timeout_ms, but typically - should be set no higher than 1/3 of that value. It can be - adjusted even lower to control the expected time for normal - rebalances. Default: 3000 - retry_backoff_ms (int): Milliseconds to backoff when retrying on - errors. Default: 100. - """ - self.config = copy.copy(self.DEFAULT_CONFIG) - for key in self.config: - if key in configs: - self.config[key] = configs[key] - - self._client = client - self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID - self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID - self.group_id = self.config['group_id'] - self.coordinator_id = None - self.rejoin_needed = True - self.needs_join_prepare = True - self.heartbeat = Heartbeat(**self.config) - self.heartbeat_task = HeartbeatTask(self) - #self.sensors = GroupCoordinatorMetrics(metrics, metric_group_prefix, metric_tags) - - @abc.abstractmethod - def protocol_type(self): - """ - Unique identifier for the class of protocols implements - (e.g. "consumer" or "connect"). - - Returns: - str: protocol type name - """ - pass - - @abc.abstractmethod - def group_protocols(self): - """Return the list of supported group protocols and metadata. - - This list is submitted by each group member via a JoinGroupRequest. - The order of the protocols in the list indicates the preference of the - protocol (the first entry is the most preferred). The coordinator takes - this preference into account when selecting the generation protocol - (generally more preferred protocols will be selected as long as all - members support them and there is no disagreement on the preference). - - Note: metadata must be type bytes or support an encode() method - - Returns: - list: [(protocol, metadata), ...] - """ - pass - - @abc.abstractmethod - def _on_join_prepare(self, generation, member_id): - """Invoked prior to each group join or rejoin. - - This is typically used to perform any cleanup from the previous - generation (such as committing offsets for the consumer) - - Arguments: - generation (int): The previous generation or -1 if there was none - member_id (str): The identifier of this member in the previous group - or '' if there was none - """ - pass - - @abc.abstractmethod - def _perform_assignment(self, leader_id, protocol, members): - """Perform assignment for the group. - - This is used by the leader to push state to all the members of the group - (e.g. to push partition assignments in the case of the new consumer) - - Arguments: - leader_id (str): The id of the leader (which is this member) - protocol (str): the chosen group protocol (assignment strategy) - members (list): [(member_id, metadata_bytes)] from - JoinGroupResponse. metadata_bytes are associated with the chosen - group protocol, and the Coordinator subclass is responsible for - decoding metadata_bytes based on that protocol. - - Returns: - dict: {member_id: assignment}; assignment must either be bytes - or have an encode() method to convert to bytes - """ - pass - - @abc.abstractmethod - def _on_join_complete(self, generation, member_id, protocol, - member_assignment_bytes): - """Invoked when a group member has successfully joined a group. - - Arguments: - generation (int): the generation that was joined - member_id (str): the identifier for the local member in the group - protocol (str): the protocol selected by the coordinator - member_assignment_bytes (bytes): the protocol-encoded assignment - propagated from the group leader. The Coordinator instance is - responsible for decoding based on the chosen protocol. - """ - pass - - def coordinator_unknown(self): - """Check if we know who the coordinator is and have an active connection - - Side-effect: reset coordinator_id to None if connection failed - - Returns: - bool: True if the coordinator is unknown - """ - if self.coordinator_id is None: - return True - - if self._client.is_disconnected(self.coordinator_id): - self.coordinator_dead() - return True - - return not self._client.ready(self.coordinator_id) - - def ensure_coordinator_known(self): - """Block until the coordinator for this group is known - (and we have an active connection -- java client uses unsent queue). - """ - while self.coordinator_unknown(): - - # Dont look for a new coordinator node if we are just waiting - # for connection to finish - if self.coordinator_id is not None: - self._client.poll() - continue - - future = self._send_group_metadata_request() - self._client.poll(future=future) - - if future.failed(): - if future.retriable(): - metadata_update = self._client.cluster.request_update() - self._client.poll(future=metadata_update) - else: - raise future.exception # pylint: disable-msg=raising-bad-type - - def need_rejoin(self): - """Check whether the group should be rejoined (e.g. if metadata changes) - - Returns: - bool: True if it should, False otherwise - """ - return self.rejoin_needed - - def ensure_active_group(self): - """Ensure that the group is active (i.e. joined and synced)""" - if not self.need_rejoin(): - return - - if self.needs_join_prepare: - self._on_join_prepare(self.generation, self.member_id) - self.needs_join_prepare = False - - while self.need_rejoin(): - self.ensure_coordinator_known() - - future = self._perform_group_join() - self._client.poll(future=future) - - if future.succeeded(): - member_assignment_bytes = future.value - self._on_join_complete(self.generation, self.member_id, - self.protocol, member_assignment_bytes) - self.needs_join_prepare = True - self.heartbeat_task.reset() - else: - exception = future.exception - if isinstance(exception, (Errors.UnknownMemberIdError, - Errors.RebalanceInProgressError, - Errors.IllegalGenerationError)): - continue - elif not future.retriable(): - raise exception # pylint: disable-msg=raising-bad-type - time.sleep(self.config['retry_backoff_ms'] / 1000.0) - - def _perform_group_join(self): - """Join the group and return the assignment for the next generation. - - This function handles both JoinGroup and SyncGroup, delegating to - _perform_assignment() if elected leader by the coordinator. - - Returns: - Future: resolves to the encoded-bytes assignment returned from the - group leader - """ - if self.coordinator_unknown(): - e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) - return Future().failure(e) - - # send a join group request to the coordinator - log.debug("(Re-)joining group %s", self.group_id) - request = JoinGroupRequest( - self.group_id, - self.config['session_timeout_ms'], - self.member_id, - self.protocol_type(), - [(protocol, - metadata if isinstance(metadata, bytes) else metadata.encode()) - for protocol, metadata in self.group_protocols()]) - - # create the request for the coordinator - log.debug("Issuing request (%s) to coordinator %s", request, self.coordinator_id) - future = Future() - _f = self._client.send(self.coordinator_id, request) - _f.add_callback(self._handle_join_group_response, future) - _f.add_errback(self._failed_request, self.coordinator_id, - request, future) - return future - - def _failed_request(self, node_id, request, future, error): - log.error('Error sending %s to node %s [%s] -- marking coordinator dead', - request.__class__.__name__, node_id, error) - self.coordinator_dead() - future.failure(error) - - def _handle_join_group_response(self, future, response): - error_type = Errors.for_code(response.error_code) - if error_type is Errors.NoError: - self.member_id = response.member_id - self.generation = response.generation_id - self.rejoin_needed = False - self.protocol = response.group_protocol - log.info("Joined group '%s' (generation %s) with member_id %s", - self.group_id, self.generation, self.member_id) - #self.sensors.join_latency.record(response.requestLatencyMs()) - if response.leader_id == response.member_id: - log.info("Elected group leader -- performing partition" - " assignments using %s", self.protocol) - self._on_join_leader(response).chain(future) - else: - self._on_join_follower().chain(future) - - elif error_type is Errors.GroupLoadInProgressError: - log.debug("Attempt to join group %s rejected since coordinator is" - " loading the group.", self.group_id) - # backoff and retry - future.failure(error_type(response)) - elif error_type is Errors.UnknownMemberIdError: - # reset the member id and retry immediately - error = error_type(self.member_id) - self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID - log.info("Attempt to join group %s failed due to unknown member id," - " resetting and retrying.", self.group_id) - future.failure(error) - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError): - # re-discover the coordinator and retry with backoff - self.coordinator_dead() - log.info("Attempt to join group %s failed due to obsolete " - "coordinator information, retrying.", self.group_id) - future.failure(error_type()) - elif error_type in (Errors.InconsistentGroupProtocolError, - Errors.InvalidSessionTimeoutError, - Errors.InvalidGroupIdError): - # log the error and re-throw the exception - error = error_type(response) - log.error("Attempt to join group %s failed due to: %s", - self.group_id, error) - future.failure(error) - elif error_type is Errors.GroupAuthorizationFailedError: - future.failure(error_type(self.group_id)) - else: - # unexpected error, throw the exception - error = error_type() - log.error("Unexpected error in join group response: %s", error) - future.failure(error) - - def _on_join_follower(self): - # send follower's sync group with an empty assignment - request = SyncGroupRequest( - self.group_id, - self.generation, - self.member_id, - {}) - log.debug("Issuing follower SyncGroup (%s) to coordinator %s", - request, self.coordinator_id) - return self._send_sync_group_request(request) - - def _on_join_leader(self, response): - """ - Perform leader synchronization and send back the assignment - for the group via SyncGroupRequest - - Arguments: - response (JoinResponse): broker response to parse - - Returns: - Future: resolves to member assignment encoded-bytes - """ - try: - group_assignment = self._perform_assignment(response.leader_id, - response.group_protocol, - response.members) - except Exception as e: - return Future().failure(e) - - request = SyncGroupRequest( - self.group_id, - self.generation, - self.member_id, - [(member_id, - assignment if isinstance(assignment, bytes) else assignment.encode()) - for member_id, assignment in six.iteritems(group_assignment)]) - - log.debug("Issuing leader SyncGroup (%s) to coordinator %s", - request, self.coordinator_id) - return self._send_sync_group_request(request) - - def _send_sync_group_request(self, request): - if self.coordinator_unknown(): - return Future().failure(Errors.GroupCoordinatorNotAvailableError()) - future = Future() - _f = self._client.send(self.coordinator_id, request) - _f.add_callback(self._handle_sync_group_response, future) - _f.add_errback(self._failed_request, self.coordinator_id, - request, future) - return future - - def _handle_sync_group_response(self, future, response): - error_type = Errors.for_code(response.error_code) - if error_type is Errors.NoError: - log.debug("Received successful sync group response for group %s: %s", - self.group_id, response) - #self.sensors.syncLatency.record(response.requestLatencyMs()) - future.success(response.member_assignment) - return - - # Always rejoin on error - self.rejoin_needed = True - if error_type is Errors.GroupAuthorizationFailedError: - future.failure(error_type(self.group_id)) - elif error_type is Errors.RebalanceInProgressError: - log.info("SyncGroup for group %s failed due to coordinator" - " rebalance, rejoining the group", self.group_id) - future.failure(error_type(self.group_id)) - elif error_type in (Errors.UnknownMemberIdError, - Errors.IllegalGenerationError): - error = error_type() - log.info("SyncGroup for group %s failed due to %s," - " rejoining the group", self.group_id, error) - self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID - future.failure(error) - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError): - error = error_type() - log.info("SyncGroup for group %s failed due to %s, will find new" - " coordinator and rejoin", self.group_id, error) - self.coordinator_dead() - future.failure(error) - else: - error = error_type() - log.error("Unexpected error from SyncGroup: %s", error) - future.failure(error) - - def _send_group_metadata_request(self): - """Discover the current coordinator for the group. - - Returns: - Future: resolves to the node id of the coordinator - """ - node_id = self._client.least_loaded_node() - if node_id is None or not self._client.ready(node_id): - return Future().failure(Errors.NoBrokersAvailable()) - - log.debug("Issuing group metadata request to broker %s", node_id) - request = GroupCoordinatorRequest(self.group_id) - future = Future() - _f = self._client.send(node_id, request) - _f.add_callback(self._handle_group_coordinator_response, future) - _f.add_errback(self._failed_request, node_id, request, future) - return future - - def _handle_group_coordinator_response(self, future, response): - log.debug("Group metadata response %s", response) - if not self.coordinator_unknown(): - # We already found the coordinator, so ignore the request - log.debug("Coordinator already known -- ignoring metadata response") - future.success(self.coordinator_id) - return - - error_type = Errors.for_code(response.error_code) - if error_type is Errors.NoError: - ok = self._client.cluster.add_group_coordinator(self.group_id, response) - if not ok: - # This could happen if coordinator metadata is different - # than broker metadata - future.failure(Errors.IllegalStateError()) - return - - self.coordinator_id = response.coordinator_id - self._client.ready(self.coordinator_id) - - # start sending heartbeats only if we have a valid generation - if self.generation > 0: - self.heartbeat_task.reset() - future.success(self.coordinator_id) - elif error_type is Errors.GroupCoordinatorNotAvailableError: - log.debug("Group Coordinator Not Available; retry") - future.failure(error_type()) - elif error_type is Errors.GroupAuthorizationFailedError: - error = error_type(self.group_id) - log.error("Group Coordinator Request failed: %s", error) - future.failure(error) - else: - error = error_type() - log.error("Unrecognized failure in Group Coordinator Request: %s", - error) - future.failure(error) - - def coordinator_dead(self, error=None): - """Mark the current coordinator as dead.""" - if self.coordinator_id is not None: - log.info("Marking the coordinator dead (node %s): %s.", - self.coordinator_id, error) - self.coordinator_id = None - - def close(self): - """Close the coordinator, leave the current group - and reset local generation/memberId.""" - try: - self._client.unschedule(self.heartbeat_task) - except KeyError: - pass - if not self.coordinator_unknown() and self.generation > 0: - # this is a minimal effort attempt to leave the group. we do not - # attempt any resending if the request fails or times out. - request = LeaveGroupRequest(self.group_id, self.member_id) - future = self._client.send(self.coordinator_id, request) - future.add_callback(self._handle_leave_group_response) - future.add_errback(log.error, "LeaveGroup request failed: %s") - self._client.poll(future=future) - - self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID - self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID - self.rejoin_needed = True - - def _handle_leave_group_response(self, response): - error_type = Errors.for_code(response.error_code) - if error_type is Errors.NoError: - log.info("LeaveGroup request succeeded") - else: - log.error("LeaveGroup request failed: %s", error_type()) - - def _send_heartbeat_request(self): - """Send a heartbeat request""" - request = HeartbeatRequest(self.group_id, self.generation, self.member_id) - log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, request.member_id) #pylint: disable-msg=no-member - future = Future() - _f = self._client.send(self.coordinator_id, request) - _f.add_callback(self._handle_heartbeat_response, future) - _f.add_errback(self._failed_request, self.coordinator_id, - request, future) - return future - - def _handle_heartbeat_response(self, future, response): - #self.sensors.heartbeat_latency.record(response.requestLatencyMs()) - error_type = Errors.for_code(response.error_code) - if error_type is Errors.NoError: - log.debug("Received successful heartbeat response.") - future.success(None) - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError): - log.info("Heartbeat failed: coordinator is either not started or" - " not valid; will refresh metadata and retry") - self.coordinator_dead() - future.failure(error_type()) - elif error_type is Errors.RebalanceInProgressError: - log.info("Heartbeat failed: group is rebalancing; re-joining group") - self.rejoin_needed = True - future.failure(error_type()) - elif error_type is Errors.IllegalGenerationError: - log.info("Heartbeat failed: local generation id is not current;" - " re-joining group") - self.rejoin_needed = True - future.failure(error_type()) - elif error_type is Errors.UnknownMemberIdError: - log.info("Heartbeat failed: local member_id was not recognized;" - " resetting and re-joining group") - self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID - self.rejoin_needed = True - future.failure(error_type) - elif error_type is Errors.GroupAuthorizationFailedError: - error = error_type(self.group_id) - log.error("Heartbeat failed: authorization error: %s", error) - future.failure(error) - else: - error = error_type() - log.error("Heartbeat failed: Unhandled error: %s", error) - future.failure(error) - - -class HeartbeatTask(object): - def __init__(self, coordinator): - self._coordinator = coordinator - self._heartbeat = coordinator.heartbeat - self._client = coordinator._client - self._request_in_flight = False - - def reset(self): - # start or restart the heartbeat task to be executed at the next chance - self._heartbeat.reset_session_timeout() - try: - self._client.unschedule(self) - except KeyError: - pass - if not self._request_in_flight: - self._client.schedule(self, time.time()) - - def __call__(self): - if (self._coordinator.generation < 0 or - self._coordinator.need_rejoin() or - self._coordinator.coordinator_unknown()): - # no need to send the heartbeat we're not using auto-assignment - # or if we are awaiting a rebalance - log.debug("Skipping heartbeat: no auto-assignment" - " or waiting on rebalance") - return - - if self._heartbeat.session_expired(): - # we haven't received a successful heartbeat in one session interval - # so mark the coordinator dead - log.error("Heartbeat session expired - marking coordinator dead") - self._coordinator.coordinator_dead() - return - - if not self._heartbeat.should_heartbeat(): - # we don't need to heartbeat now, so reschedule for when we do - ttl = self._heartbeat.ttl() - log.debug("Heartbeat task unneeded now, retrying in %s", ttl) - self._client.schedule(self, time.time() + ttl) - else: - self._heartbeat.sent_heartbeat() - self._request_in_flight = True - future = self._coordinator._send_heartbeat_request() - future.add_callback(self._handle_heartbeat_success) - future.add_errback(self._handle_heartbeat_failure) - - def _handle_heartbeat_success(self, v): - log.debug("Received successful heartbeat") - self._request_in_flight = False - self._heartbeat.received_heartbeat() - ttl = self._heartbeat.ttl() - self._client.schedule(self, time.time() + ttl) - - def _handle_heartbeat_failure(self, e): - log.debug("Heartbeat failed; retrying") - self._request_in_flight = False - etd = time.time() + self._coordinator.config['retry_backoff_ms'] / 1000.0 - self._client.schedule(self, etd) - - -class GroupCoordinatorMetrics(object): - def __init__(self, metrics, prefix, tags=None): - self.metrics = metrics - self.group_name = prefix + "-coordinator-metrics" - - self.heartbeat_latency = metrics.sensor("heartbeat-latency") - self.heartbeat_latency.add(metrics.metricName( - "heartbeat-response-time-max", self.group_name, - "The max time taken to receive a response to a heartbeat request", - tags), metrics.Max()) - self.heartbeat_latency.add(metrics.metricName( - "heartbeat-rate", self.group_name, - "The average number of heartbeats per second", - tags), metrics.Rate(metrics.Count())) - - self.join_latency = metrics.sensor("join-latency") - self.join_latency.add(metrics.metricName( - "join-time-avg", self.group_name, - "The average time taken for a group rejoin", - tags), metrics.Avg()) - self.join_latency.add(metrics.metricName( - "join-time-max", self.group_name, - "The max time taken for a group rejoin", - tags), metrics.Avg()) - self.join_latency.add(metrics.metricName( - "join-rate", self.group_name, - "The number of group joins per second", - tags), metrics.Rate(metrics.Count())) - - self.sync_latency = metrics.sensor("sync-latency") - self.sync_latency.add(metrics.metricName( - "sync-time-avg", self.group_name, - "The average time taken for a group sync", - tags), metrics.Avg()) - self.sync_latency.add(metrics.MetricName( - "sync-time-max", self.group_name, - "The max time taken for a group sync", - tags), metrics.Avg()) - self.sync_latency.add(metrics.metricName( - "sync-rate", self.group_name, - "The number of group syncs per second", - tags), metrics.Rate(metrics.Count())) - - """ - lastHeartbeat = Measurable( - measure=lambda _, value: value - heartbeat.last_heartbeat_send() - ) - metrics.addMetric(metrics.metricName( - "last-heartbeat-seconds-ago", self.group_name, - "The number of seconds since the last controller heartbeat", - tags), lastHeartbeat) - """ - - diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py new file mode 100644 index 0000000..4f16bb0 --- /dev/null +++ b/kafka/coordinator/base.py @@ -0,0 +1,677 @@ +import abc +import copy +import logging +import time + +import six + +import kafka.common as Errors +from kafka.future import Future +from kafka.protocol.commit import (GroupCoordinatorRequest, + OffsetCommitRequest_v2 as OffsetCommitRequest) +from kafka.protocol.group import (HeartbeatRequest, JoinGroupRequest, + LeaveGroupRequest, SyncGroupRequest) +from .heartbeat import Heartbeat + +log = logging.getLogger('kafka.coordinator') + + +class BaseCoordinator(object): + """ + BaseCoordinator implements group management for a single group member + by interacting with a designated Kafka broker (the coordinator). Group + semantics are provided by extending this class. See ConsumerCoordinator + for example usage. + + From a high level, Kafka's group management protocol consists of the + following sequence of actions: + + 1. Group Registration: Group members register with the coordinator providing + their own metadata (such as the set of topics they are interested in). + + 2. Group/Leader Selection: The coordinator select the members of the group + and chooses one member as the leader. + + 3. State Assignment: The leader collects the metadata from all the members + of the group and assigns state. + + 4. Group Stabilization: Each member receives the state assigned by the + leader and begins processing. + + To leverage this protocol, an implementation must define the format of + metadata provided by each member for group registration in group_protocols() + and the format of the state assignment provided by the leader in + _perform_assignment() and which becomes available to members in + _on_join_complete(). + """ + + DEFAULT_CONFIG = { + 'group_id': 'kafka-python-default-group', + 'session_timeout_ms': 30000, + 'heartbeat_interval_ms': 3000, + 'retry_backoff_ms': 100, + } + + def __init__(self, client, **configs): + """ + Keyword Arguments: + group_id (str): name of the consumer group to join for dynamic + partition assignment (if enabled), and to use for fetching and + committing offsets. Default: 'kafka-python-default-group' + session_timeout_ms (int): The timeout used to detect failures when + using Kafka's group managementment facilities. Default: 30000 + heartbeat_interval_ms (int): The expected time in milliseconds + between heartbeats to the consumer coordinator when using + Kafka's group management feature. Heartbeats are used to ensure + that the consumer's session stays active and to facilitate + rebalancing when new consumers join or leave the group. The + value must be set lower than session_timeout_ms, but typically + should be set no higher than 1/3 of that value. It can be + adjusted even lower to control the expected time for normal + rebalances. Default: 3000 + retry_backoff_ms (int): Milliseconds to backoff when retrying on + errors. Default: 100. + """ + self.config = copy.copy(self.DEFAULT_CONFIG) + for key in self.config: + if key in configs: + self.config[key] = configs[key] + + self._client = client + self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + self.group_id = self.config['group_id'] + self.coordinator_id = None + self.rejoin_needed = True + self.needs_join_prepare = True + self.heartbeat = Heartbeat(**self.config) + self.heartbeat_task = HeartbeatTask(self) + #self.sensors = GroupCoordinatorMetrics(metrics, metric_group_prefix, metric_tags) + + @abc.abstractmethod + def protocol_type(self): + """ + Unique identifier for the class of protocols implements + (e.g. "consumer" or "connect"). + + Returns: + str: protocol type name + """ + pass + + @abc.abstractmethod + def group_protocols(self): + """Return the list of supported group protocols and metadata. + + This list is submitted by each group member via a JoinGroupRequest. + The order of the protocols in the list indicates the preference of the + protocol (the first entry is the most preferred). The coordinator takes + this preference into account when selecting the generation protocol + (generally more preferred protocols will be selected as long as all + members support them and there is no disagreement on the preference). + + Note: metadata must be type bytes or support an encode() method + + Returns: + list: [(protocol, metadata), ...] + """ + pass + + @abc.abstractmethod + def _on_join_prepare(self, generation, member_id): + """Invoked prior to each group join or rejoin. + + This is typically used to perform any cleanup from the previous + generation (such as committing offsets for the consumer) + + Arguments: + generation (int): The previous generation or -1 if there was none + member_id (str): The identifier of this member in the previous group + or '' if there was none + """ + pass + + @abc.abstractmethod + def _perform_assignment(self, leader_id, protocol, members): + """Perform assignment for the group. + + This is used by the leader to push state to all the members of the group + (e.g. to push partition assignments in the case of the new consumer) + + Arguments: + leader_id (str): The id of the leader (which is this member) + protocol (str): the chosen group protocol (assignment strategy) + members (list): [(member_id, metadata_bytes)] from + JoinGroupResponse. metadata_bytes are associated with the chosen + group protocol, and the Coordinator subclass is responsible for + decoding metadata_bytes based on that protocol. + + Returns: + dict: {member_id: assignment}; assignment must either be bytes + or have an encode() method to convert to bytes + """ + pass + + @abc.abstractmethod + def _on_join_complete(self, generation, member_id, protocol, + member_assignment_bytes): + """Invoked when a group member has successfully joined a group. + + Arguments: + generation (int): the generation that was joined + member_id (str): the identifier for the local member in the group + protocol (str): the protocol selected by the coordinator + member_assignment_bytes (bytes): the protocol-encoded assignment + propagated from the group leader. The Coordinator instance is + responsible for decoding based on the chosen protocol. + """ + pass + + def coordinator_unknown(self): + """Check if we know who the coordinator is and have an active connection + + Side-effect: reset coordinator_id to None if connection failed + + Returns: + bool: True if the coordinator is unknown + """ + if self.coordinator_id is None: + return True + + if self._client.is_disconnected(self.coordinator_id): + self.coordinator_dead() + return True + + return not self._client.ready(self.coordinator_id) + + def ensure_coordinator_known(self): + """Block until the coordinator for this group is known + (and we have an active connection -- java client uses unsent queue). + """ + while self.coordinator_unknown(): + + # Dont look for a new coordinator node if we are just waiting + # for connection to finish + if self.coordinator_id is not None: + self._client.poll() + continue + + future = self._send_group_metadata_request() + self._client.poll(future=future) + + if future.failed(): + if future.retriable(): + metadata_update = self._client.cluster.request_update() + self._client.poll(future=metadata_update) + else: + raise future.exception # pylint: disable-msg=raising-bad-type + + def need_rejoin(self): + """Check whether the group should be rejoined (e.g. if metadata changes) + + Returns: + bool: True if it should, False otherwise + """ + return self.rejoin_needed + + def ensure_active_group(self): + """Ensure that the group is active (i.e. joined and synced)""" + if not self.need_rejoin(): + return + + if self.needs_join_prepare: + self._on_join_prepare(self.generation, self.member_id) + self.needs_join_prepare = False + + while self.need_rejoin(): + self.ensure_coordinator_known() + + future = self._perform_group_join() + self._client.poll(future=future) + + if future.succeeded(): + member_assignment_bytes = future.value + self._on_join_complete(self.generation, self.member_id, + self.protocol, member_assignment_bytes) + self.needs_join_prepare = True + self.heartbeat_task.reset() + else: + exception = future.exception + if isinstance(exception, (Errors.UnknownMemberIdError, + Errors.RebalanceInProgressError, + Errors.IllegalGenerationError)): + continue + elif not future.retriable(): + raise exception # pylint: disable-msg=raising-bad-type + time.sleep(self.config['retry_backoff_ms'] / 1000.0) + + def _perform_group_join(self): + """Join the group and return the assignment for the next generation. + + This function handles both JoinGroup and SyncGroup, delegating to + _perform_assignment() if elected leader by the coordinator. + + Returns: + Future: resolves to the encoded-bytes assignment returned from the + group leader + """ + if self.coordinator_unknown(): + e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) + return Future().failure(e) + + # send a join group request to the coordinator + log.debug("(Re-)joining group %s", self.group_id) + request = JoinGroupRequest( + self.group_id, + self.config['session_timeout_ms'], + self.member_id, + self.protocol_type(), + [(protocol, + metadata if isinstance(metadata, bytes) else metadata.encode()) + for protocol, metadata in self.group_protocols()]) + + # create the request for the coordinator + log.debug("Issuing request (%s) to coordinator %s", request, self.coordinator_id) + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_join_group_response, future) + _f.add_errback(self._failed_request, self.coordinator_id, + request, future) + return future + + def _failed_request(self, node_id, request, future, error): + log.error('Error sending %s to node %s [%s] -- marking coordinator dead', + request.__class__.__name__, node_id, error) + self.coordinator_dead() + future.failure(error) + + def _handle_join_group_response(self, future, response): + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + self.member_id = response.member_id + self.generation = response.generation_id + self.rejoin_needed = False + self.protocol = response.group_protocol + log.info("Joined group '%s' (generation %s) with member_id %s", + self.group_id, self.generation, self.member_id) + #self.sensors.join_latency.record(response.requestLatencyMs()) + if response.leader_id == response.member_id: + log.info("Elected group leader -- performing partition" + " assignments using %s", self.protocol) + self._on_join_leader(response).chain(future) + else: + self._on_join_follower().chain(future) + + elif error_type is Errors.GroupLoadInProgressError: + log.debug("Attempt to join group %s rejected since coordinator is" + " loading the group.", self.group_id) + # backoff and retry + future.failure(error_type(response)) + elif error_type is Errors.UnknownMemberIdError: + # reset the member id and retry immediately + error = error_type(self.member_id) + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + log.info("Attempt to join group %s failed due to unknown member id," + " resetting and retrying.", self.group_id) + future.failure(error) + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError): + # re-discover the coordinator and retry with backoff + self.coordinator_dead() + log.info("Attempt to join group %s failed due to obsolete " + "coordinator information, retrying.", self.group_id) + future.failure(error_type()) + elif error_type in (Errors.InconsistentGroupProtocolError, + Errors.InvalidSessionTimeoutError, + Errors.InvalidGroupIdError): + # log the error and re-throw the exception + error = error_type(response) + log.error("Attempt to join group %s failed due to: %s", + self.group_id, error) + future.failure(error) + elif error_type is Errors.GroupAuthorizationFailedError: + future.failure(error_type(self.group_id)) + else: + # unexpected error, throw the exception + error = error_type() + log.error("Unexpected error in join group response: %s", error) + future.failure(error) + + def _on_join_follower(self): + # send follower's sync group with an empty assignment + request = SyncGroupRequest( + self.group_id, + self.generation, + self.member_id, + {}) + log.debug("Issuing follower SyncGroup (%s) to coordinator %s", + request, self.coordinator_id) + return self._send_sync_group_request(request) + + def _on_join_leader(self, response): + """ + Perform leader synchronization and send back the assignment + for the group via SyncGroupRequest + + Arguments: + response (JoinResponse): broker response to parse + + Returns: + Future: resolves to member assignment encoded-bytes + """ + try: + group_assignment = self._perform_assignment(response.leader_id, + response.group_protocol, + response.members) + except Exception as e: + return Future().failure(e) + + request = SyncGroupRequest( + self.group_id, + self.generation, + self.member_id, + [(member_id, + assignment if isinstance(assignment, bytes) else assignment.encode()) + for member_id, assignment in six.iteritems(group_assignment)]) + + log.debug("Issuing leader SyncGroup (%s) to coordinator %s", + request, self.coordinator_id) + return self._send_sync_group_request(request) + + def _send_sync_group_request(self, request): + if self.coordinator_unknown(): + return Future().failure(Errors.GroupCoordinatorNotAvailableError()) + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_sync_group_response, future) + _f.add_errback(self._failed_request, self.coordinator_id, + request, future) + return future + + def _handle_sync_group_response(self, future, response): + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.debug("Received successful sync group response for group %s: %s", + self.group_id, response) + #self.sensors.syncLatency.record(response.requestLatencyMs()) + future.success(response.member_assignment) + return + + # Always rejoin on error + self.rejoin_needed = True + if error_type is Errors.GroupAuthorizationFailedError: + future.failure(error_type(self.group_id)) + elif error_type is Errors.RebalanceInProgressError: + log.info("SyncGroup for group %s failed due to coordinator" + " rebalance, rejoining the group", self.group_id) + future.failure(error_type(self.group_id)) + elif error_type in (Errors.UnknownMemberIdError, + Errors.IllegalGenerationError): + error = error_type() + log.info("SyncGroup for group %s failed due to %s," + " rejoining the group", self.group_id, error) + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + future.failure(error) + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError): + error = error_type() + log.info("SyncGroup for group %s failed due to %s, will find new" + " coordinator and rejoin", self.group_id, error) + self.coordinator_dead() + future.failure(error) + else: + error = error_type() + log.error("Unexpected error from SyncGroup: %s", error) + future.failure(error) + + def _send_group_metadata_request(self): + """Discover the current coordinator for the group. + + Returns: + Future: resolves to the node id of the coordinator + """ + node_id = self._client.least_loaded_node() + if node_id is None or not self._client.ready(node_id): + return Future().failure(Errors.NoBrokersAvailable()) + + log.debug("Issuing group metadata request to broker %s", node_id) + request = GroupCoordinatorRequest(self.group_id) + future = Future() + _f = self._client.send(node_id, request) + _f.add_callback(self._handle_group_coordinator_response, future) + _f.add_errback(self._failed_request, node_id, request, future) + return future + + def _handle_group_coordinator_response(self, future, response): + log.debug("Group metadata response %s", response) + if not self.coordinator_unknown(): + # We already found the coordinator, so ignore the request + log.debug("Coordinator already known -- ignoring metadata response") + future.success(self.coordinator_id) + return + + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + ok = self._client.cluster.add_group_coordinator(self.group_id, response) + if not ok: + # This could happen if coordinator metadata is different + # than broker metadata + future.failure(Errors.IllegalStateError()) + return + + self.coordinator_id = response.coordinator_id + self._client.ready(self.coordinator_id) + + # start sending heartbeats only if we have a valid generation + if self.generation > 0: + self.heartbeat_task.reset() + future.success(self.coordinator_id) + elif error_type is Errors.GroupCoordinatorNotAvailableError: + log.debug("Group Coordinator Not Available; retry") + future.failure(error_type()) + elif error_type is Errors.GroupAuthorizationFailedError: + error = error_type(self.group_id) + log.error("Group Coordinator Request failed: %s", error) + future.failure(error) + else: + error = error_type() + log.error("Unrecognized failure in Group Coordinator Request: %s", + error) + future.failure(error) + + def coordinator_dead(self, error=None): + """Mark the current coordinator as dead.""" + if self.coordinator_id is not None: + log.info("Marking the coordinator dead (node %s): %s.", + self.coordinator_id, error) + self.coordinator_id = None + + def close(self): + """Close the coordinator, leave the current group + and reset local generation/memberId.""" + try: + self._client.unschedule(self.heartbeat_task) + except KeyError: + pass + if not self.coordinator_unknown() and self.generation > 0: + # this is a minimal effort attempt to leave the group. we do not + # attempt any resending if the request fails or times out. + request = LeaveGroupRequest(self.group_id, self.member_id) + future = self._client.send(self.coordinator_id, request) + future.add_callback(self._handle_leave_group_response) + future.add_errback(log.error, "LeaveGroup request failed: %s") + self._client.poll(future=future) + + self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + self.rejoin_needed = True + + def _handle_leave_group_response(self, response): + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.info("LeaveGroup request succeeded") + else: + log.error("LeaveGroup request failed: %s", error_type()) + + def _send_heartbeat_request(self): + """Send a heartbeat request""" + request = HeartbeatRequest(self.group_id, self.generation, self.member_id) + log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, request.member_id) #pylint: disable-msg=no-member + future = Future() + _f = self._client.send(self.coordinator_id, request) + _f.add_callback(self._handle_heartbeat_response, future) + _f.add_errback(self._failed_request, self.coordinator_id, + request, future) + return future + + def _handle_heartbeat_response(self, future, response): + #self.sensors.heartbeat_latency.record(response.requestLatencyMs()) + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + log.debug("Received successful heartbeat response.") + future.success(None) + elif error_type in (Errors.GroupCoordinatorNotAvailableError, + Errors.NotCoordinatorForGroupError): + log.info("Heartbeat failed: coordinator is either not started or" + " not valid; will refresh metadata and retry") + self.coordinator_dead() + future.failure(error_type()) + elif error_type is Errors.RebalanceInProgressError: + log.info("Heartbeat failed: group is rebalancing; re-joining group") + self.rejoin_needed = True + future.failure(error_type()) + elif error_type is Errors.IllegalGenerationError: + log.info("Heartbeat failed: local generation id is not current;" + " re-joining group") + self.rejoin_needed = True + future.failure(error_type()) + elif error_type is Errors.UnknownMemberIdError: + log.info("Heartbeat failed: local member_id was not recognized;" + " resetting and re-joining group") + self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID + self.rejoin_needed = True + future.failure(error_type) + elif error_type is Errors.GroupAuthorizationFailedError: + error = error_type(self.group_id) + log.error("Heartbeat failed: authorization error: %s", error) + future.failure(error) + else: + error = error_type() + log.error("Heartbeat failed: Unhandled error: %s", error) + future.failure(error) + + +class HeartbeatTask(object): + def __init__(self, coordinator): + self._coordinator = coordinator + self._heartbeat = coordinator.heartbeat + self._client = coordinator._client + self._request_in_flight = False + + def reset(self): + # start or restart the heartbeat task to be executed at the next chance + self._heartbeat.reset_session_timeout() + try: + self._client.unschedule(self) + except KeyError: + pass + if not self._request_in_flight: + self._client.schedule(self, time.time()) + + def __call__(self): + if (self._coordinator.generation < 0 or + self._coordinator.need_rejoin() or + self._coordinator.coordinator_unknown()): + # no need to send the heartbeat we're not using auto-assignment + # or if we are awaiting a rebalance + log.debug("Skipping heartbeat: no auto-assignment" + " or waiting on rebalance") + return + + if self._heartbeat.session_expired(): + # we haven't received a successful heartbeat in one session interval + # so mark the coordinator dead + log.error("Heartbeat session expired - marking coordinator dead") + self._coordinator.coordinator_dead() + return + + if not self._heartbeat.should_heartbeat(): + # we don't need to heartbeat now, so reschedule for when we do + ttl = self._heartbeat.ttl() + log.debug("Heartbeat task unneeded now, retrying in %s", ttl) + self._client.schedule(self, time.time() + ttl) + else: + self._heartbeat.sent_heartbeat() + self._request_in_flight = True + future = self._coordinator._send_heartbeat_request() + future.add_callback(self._handle_heartbeat_success) + future.add_errback(self._handle_heartbeat_failure) + + def _handle_heartbeat_success(self, v): + log.debug("Received successful heartbeat") + self._request_in_flight = False + self._heartbeat.received_heartbeat() + ttl = self._heartbeat.ttl() + self._client.schedule(self, time.time() + ttl) + + def _handle_heartbeat_failure(self, e): + log.debug("Heartbeat failed; retrying") + self._request_in_flight = False + etd = time.time() + self._coordinator.config['retry_backoff_ms'] / 1000.0 + self._client.schedule(self, etd) + + +class GroupCoordinatorMetrics(object): + def __init__(self, metrics, prefix, tags=None): + self.metrics = metrics + self.group_name = prefix + "-coordinator-metrics" + + self.heartbeat_latency = metrics.sensor("heartbeat-latency") + self.heartbeat_latency.add(metrics.metricName( + "heartbeat-response-time-max", self.group_name, + "The max time taken to receive a response to a heartbeat request", + tags), metrics.Max()) + self.heartbeat_latency.add(metrics.metricName( + "heartbeat-rate", self.group_name, + "The average number of heartbeats per second", + tags), metrics.Rate(metrics.Count())) + + self.join_latency = metrics.sensor("join-latency") + self.join_latency.add(metrics.metricName( + "join-time-avg", self.group_name, + "The average time taken for a group rejoin", + tags), metrics.Avg()) + self.join_latency.add(metrics.metricName( + "join-time-max", self.group_name, + "The max time taken for a group rejoin", + tags), metrics.Avg()) + self.join_latency.add(metrics.metricName( + "join-rate", self.group_name, + "The number of group joins per second", + tags), metrics.Rate(metrics.Count())) + + self.sync_latency = metrics.sensor("sync-latency") + self.sync_latency.add(metrics.metricName( + "sync-time-avg", self.group_name, + "The average time taken for a group sync", + tags), metrics.Avg()) + self.sync_latency.add(metrics.MetricName( + "sync-time-max", self.group_name, + "The max time taken for a group sync", + tags), metrics.Avg()) + self.sync_latency.add(metrics.metricName( + "sync-rate", self.group_name, + "The number of group syncs per second", + tags), metrics.Rate(metrics.Count())) + + """ + lastHeartbeat = Measurable( + measure=lambda _, value: value - heartbeat.last_heartbeat_send() + ) + metrics.addMetric(metrics.metricName( + "last-heartbeat-seconds-ago", self.group_name, + "The number of seconds since the last controller heartbeat", + tags), lastHeartbeat) + """ + + diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 7bc10cd..99d62f2 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -5,7 +5,7 @@ import time import six -from .abstract import AbstractCoordinator +from .base import BaseCoordinator import kafka.common as Errors from kafka.common import OffsetAndMetadata, TopicPartition from kafka.future import Future @@ -44,7 +44,7 @@ class ConsumerProtocol(object): ASSIGNMENT = ConsumerProtocolMemberAssignment -class ConsumerCoordinator(AbstractCoordinator): +class ConsumerCoordinator(BaseCoordinator): """This class manages the coordination process with the consumer coordinator.""" DEFAULT_CONFIG = { 'group_id': 'kafka-python-default-group', -- cgit v1.2.1 From 14de82535a66e2bfadddb76e7cb2b842be63b0fe Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 16:26:46 -0800 Subject: Support simple message iteration in Fetcher and new KafkaConsumer --- kafka/consumer/fetcher.py | 49 +++++++++++++++++++++++++++++++++++++++++++++++ kafka/consumer/group.py | 22 +++++++++++++++++++++ 2 files changed, 71 insertions(+) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index c133a31..8a48575 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -316,6 +316,55 @@ class Fetcher(object): tp, fetch_offset) return dict(drained) + def __iter__(self): + """Iterate over fetched_records""" + if self._subscriptions.needs_partition_assignment: + raise StopIteration('Subscription needs partition assignment') + + self._raise_if_offset_out_of_range() + self._raise_if_unauthorized_topics() + self._raise_if_record_too_large() + + while self._records: + (fetch_offset, tp, messages) = self._records.popleft() + + if not self._subscriptions.is_assigned(tp): + # this can happen when a rebalance happened before + # fetched records are returned + log.warning("Not returning fetched records for partition %s" + " since it is no longer assigned", tp) + continue + + # note that the consumed position should always be available + # as long as the partition is still assigned + consumed = self._subscriptions.assignment[tp].consumed + if not self._subscriptions.is_fetchable(tp): + # this can happen when a partition consumption paused before + # fetched records are returned + log.warning("Not returning fetched records for assigned partition" + " %s since it is no longer fetchable", tp) + + # we also need to reset the fetch positions to pretend we did + # not fetch this partition in the previous request at all + self._subscriptions.assignment[tp].fetched = consumed + + elif fetch_offset == consumed: + # TODO: handle compressed messages + for offset, size, msg in messages: + if msg.attributes: + raise Errors.KafkaError('Compressed messages not supported yet') + elif self.config['check_crcs'] and not msg.validate_crc(): + raise Errors.InvalidMessageError(msg) + + self._subscriptions.assignment[tp].consumed = offset + 1 + key, value = self._deserialize(msg) + yield ConsumerRecord(tp.topic, tp.partition, offset, key, value) + else: + # these records aren't next in line based on the last consumed + # position, ignore them they must be from an obsolete request + log.warning("Ignoring fetched records for %s at offset %s", + tp, fetch_offset) + def _deserialize(self, msg): if self.config['key_deserializer']: key = self.config['key_deserializer'](msg.key) # pylint: disable-msg=not-callable diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 90d9d37..bde283c 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -4,6 +4,8 @@ import copy import logging import time +import six + import kafka.common as Errors from kafka.client_async import KafkaClient @@ -565,3 +567,23 @@ class KafkaConsumer(object): # then do any offset lookups in case some positions are not known self._fetcher.update_fetch_positions(partitions) + + def __iter__(self): + while True: + # records = self._poll_once(self.config['request_timeout_ms']) + self._coordinator.ensure_coordinator_known() + + # ensure we have partitions assigned if we expect to + if self._subscription.partitions_auto_assigned(): + self._coordinator.ensure_active_group() + + # fetch positions if we have partitions we're subscribed to that we + # don't know the offset for + if not self._subscription.has_all_fetch_positions(): + self._update_fetch_positions(self._subscription.missing_fetch_positions()) + + # init any new fetches (won't resend pending fetches) + self._fetcher.init_fetches() + self._client.poll(self.config['request_timeout_ms'] / 1000.0) + for msg in self._fetcher: + yield msg -- cgit v1.2.1 From 650b134f0b41bdc4c9b9c20a26803ff2355b47cb Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Dec 2015 16:28:49 -0800 Subject: Towards version 1.0 --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 9272695..712bd6a 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '0.9.5' +__version__ = '0.99.0-dev' -- cgit v1.2.1 From bc2688e69c46fe6890c2e44990720698475c85ed Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 31 Dec 2015 11:18:49 -0800 Subject: Support message decompression --- kafka/consumer/fetcher.py | 34 +++++++++++++++------------------- kafka/protocol/message.py | 27 +++++++++++++++++++++++++-- 2 files changed, 40 insertions(+), 21 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 8a48575..fc03d7a 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -299,15 +299,7 @@ class Fetcher(object): " and update consumed position to %s", tp, next_offset) self._subscriptions.assignment[tp].consumed = next_offset - # TODO: handle compressed messages - for offset, size, msg in messages: - if msg.attributes: - raise Errors.KafkaError('Compressed messages not supported yet') - elif self.config['check_crcs'] and not msg.validate_crc(): - raise Errors.InvalidMessageError(msg) - - key, value = self._deserialize(msg) - record = ConsumerRecord(tp.topic, tp.partition, offset, key, value) + for record in self._unpack_message_set(tp, messages): drained[tp].append(record) else: # these records aren't next in line based on the last consumed @@ -316,6 +308,17 @@ class Fetcher(object): tp, fetch_offset) return dict(drained) + def _unpack_message_set(self, tp, messages): + for offset, size, msg in messages: + if self.config['check_crcs'] and not msg.validate_crc(): + raise Errors.InvalidMessageError(msg) + elif msg.is_compressed(): + for record in self._unpack_message_set(tp, msg.decompress()): + yield record + else: + key, value = self._deserialize(msg) + yield ConsumerRecord(tp.topic, tp.partition, offset, key, value) + def __iter__(self): """Iterate over fetched_records""" if self._subscriptions.needs_partition_assignment: @@ -349,16 +352,9 @@ class Fetcher(object): self._subscriptions.assignment[tp].fetched = consumed elif fetch_offset == consumed: - # TODO: handle compressed messages - for offset, size, msg in messages: - if msg.attributes: - raise Errors.KafkaError('Compressed messages not supported yet') - elif self.config['check_crcs'] and not msg.validate_crc(): - raise Errors.InvalidMessageError(msg) - - self._subscriptions.assignment[tp].consumed = offset + 1 - key, value = self._deserialize(msg) - yield ConsumerRecord(tp.topic, tp.partition, offset, key, value) + for msg in self._unpack_message_set(tp, messages): + self._subscriptions.assignment[tp].consumed = msg.offset + 1 + yield msg else: # these records aren't next in line based on the last consumed # position, ignore them they must be from an obsolete request diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index f6cbb33..f893912 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -1,5 +1,6 @@ import io +from ..codec import gzip_decode, snappy_decode from . import pickle from .struct import Struct from .types import ( @@ -16,6 +17,9 @@ class Message(Struct): ('key', Bytes), ('value', Bytes) ) + CODEC_MASK = 0x03 + CODEC_GZIP = 0x01 + CODEC_SNAPPY = 0x02 def __init__(self, value, key=None, magic=0, attributes=0, crc=0): self.crc = crc @@ -49,6 +53,19 @@ class Message(Struct): return True return False + def is_compressed(self): + return self.attributes & self.CODEC_MASK != 0 + + def decompress(self): + codec = self.attributes & self.CODEC_MASK + assert codec in (self.CODEC_GZIP, self.CODEC_SNAPPY) + if codec == self.CODEC_GZIP: + raw_bytes = gzip_decode(self.value) + else: + raw_bytes = snappy_decode(self.value) + + return MessageSet.decode(raw_bytes, bytes_to_read=len(raw_bytes)) + class PartialMessage(bytes): def __repr__(self): @@ -81,8 +98,14 @@ class MessageSet(AbstractType): return Int32.encode(len(encoded)) + encoded @classmethod - def decode(cls, data): - bytes_to_read = Int32.decode(data) + def decode(cls, data, bytes_to_read=None): + """Compressed messages should pass in bytes_to_read (via message size) + otherwise, we decode from data as Int32 + """ + if isinstance(data, bytes): + data = io.BytesIO(data) + if bytes_to_read is None: + bytes_to_read = Int32.decode(data) items = [] # We need at least 8 + 4 + 14 bytes to read offset + message size + message -- cgit v1.2.1 From 61ccbc5f7bd1527096c4609f2e881e6a1075e579 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 31 Dec 2015 11:20:01 -0800 Subject: Fixup heartbeat config assertion --- kafka/coordinator/heartbeat.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/heartbeat.py b/kafka/coordinator/heartbeat.py index 9a28f5e..4ddcf09 100644 --- a/kafka/coordinator/heartbeat.py +++ b/kafka/coordinator/heartbeat.py @@ -17,7 +17,7 @@ class Heartbeat(object): self.config[key] = configs[key] assert (self.config['heartbeat_interval_ms'] - <= self.config['session_timeout_ms'], + <= self.config['session_timeout_ms']), ( 'Heartbeat interval must be lower than the session timeout') self.interval = self.config['heartbeat_interval_ms'] / 1000.0 -- cgit v1.2.1 From 89e22a0e457ac4f6ddbf237ff32e5a278c2c02ed Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 31 Dec 2015 11:21:16 -0800 Subject: Improve request pipelining in consumer iterator --- kafka/consumer/fetcher.py | 38 +++++++++++++++++++++++++------------- kafka/consumer/group.py | 3 +++ 2 files changed, 28 insertions(+), 13 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index fc03d7a..5e15424 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -197,6 +197,9 @@ class Fetcher(object): contains OffsetOutOfRangeError and the default_reset_policy is None """ + if not self._offset_out_of_range_partitions: + return + current_out_of_range_partitions = {} # filter only the fetchable partitions @@ -232,18 +235,20 @@ class Fetcher(object): Raises: RecordTooLargeError: if there is a message larger than fetch size """ + if not self._record_too_large_partitions: + return + copied_record_too_large_partitions = dict(self._record_too_large_partitions) self._record_too_large_partitions.clear() - if copied_record_too_large_partitions: - raise RecordTooLargeError( - "There are some messages at [Partition=Offset]: %s " - " whose size is larger than the fetch size %s" - " and hence cannot be ever returned." - " Increase the fetch size, or decrease the maximum message" - " size the broker will allow.", - copied_record_too_large_partitions, - self.config['max_partition_fetch_bytes']) + raise RecordTooLargeError( + "There are some messages at [Partition=Offset]: %s " + " whose size is larger than the fetch size %s" + " and hence cannot be ever returned." + " Increase the fetch size, or decrease the maximum message" + " size the broker will allow.", + copied_record_too_large_partitions, + self.config['max_partition_fetch_bytes']) def fetched_records(self): """Returns previously fetched records and updates consumed offsets. @@ -324,11 +329,13 @@ class Fetcher(object): if self._subscriptions.needs_partition_assignment: raise StopIteration('Subscription needs partition assignment') - self._raise_if_offset_out_of_range() - self._raise_if_unauthorized_topics() - self._raise_if_record_too_large() - while self._records: + + # Check on each iteration since this is a generator + self._raise_if_offset_out_of_range() + self._raise_if_unauthorized_topics() + self._raise_if_record_too_large() + (fetch_offset, tp, messages) = self._records.popleft() if not self._subscriptions.is_assigned(tp): @@ -361,6 +368,11 @@ class Fetcher(object): log.warning("Ignoring fetched records for %s at offset %s", tp, fetch_offset) + # Send any additional FetchRequests that we can now + # this will likely fetch each partition individually, rather than + # fetch multiple partitions in bulk when they are on the same broker + self.init_fetches() + def _deserialize(self, msg): if self.config['key_deserializer']: key = self.config['key_deserializer'](msg.key) # pylint: disable-msg=not-callable diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index bde283c..67e352a 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -585,5 +585,8 @@ class KafkaConsumer(object): # init any new fetches (won't resend pending fetches) self._fetcher.init_fetches() self._client.poll(self.config['request_timeout_ms'] / 1000.0) + timeout = time.time() + self.config['heartbeat_interval_ms'] / 1000.0 for msg in self._fetcher: yield msg + if time.time() > timeout: + break -- cgit v1.2.1 From 9258bb4ba5e66565f74a691640c5c11f3b58bbd8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 31 Dec 2015 11:21:44 -0800 Subject: Remove some unused bits from kafka.consumer.group --- kafka/consumer/group.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 67e352a..5278214 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -4,10 +4,6 @@ import copy import logging import time -import six - -import kafka.common as Errors - from kafka.client_async import KafkaClient from kafka.consumer.fetcher import Fetcher from kafka.consumer.subscription_state import SubscriptionState @@ -570,7 +566,6 @@ class KafkaConsumer(object): def __iter__(self): while True: - # records = self._poll_once(self.config['request_timeout_ms']) self._coordinator.ensure_coordinator_known() # ensure we have partitions assigned if we expect to -- cgit v1.2.1 From 9fe904e9bbc64a8bcd1fa5876a76ca93b544cdfe Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 12:11:08 -0800 Subject: Struct __eq__ --- kafka/protocol/struct.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index d340abf..0a96c02 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -44,6 +44,13 @@ class Struct(AbstractType): key_vals.append('%s=%s' % (name, field.repr(self.__dict__[name]))) return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' + def __eq__(self, other): + if self.SCHEMA != other.SCHEMA: + return False + for attr in self.SCHEMA.names: + if self.__dict__[attr] != other.__dict__[attr]: + return False + return True """ class MetaStruct(type): -- cgit v1.2.1 From b1e0aef468aa602c30bc827af2afe74a1558bb6c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 12:11:32 -0800 Subject: Skeleton tests for async kafka client --- test/test_client_async.py | 103 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 103 insertions(+) create mode 100644 test/test_client_async.py diff --git a/test/test_client_async.py b/test/test_client_async.py new file mode 100644 index 0000000..5f0ccb0 --- /dev/null +++ b/test/test_client_async.py @@ -0,0 +1,103 @@ + +from mock import patch +from . import unittest + +from kafka.client_async import KafkaClient +from kafka.common import BrokerMetadata +from kafka.conn import ConnectionStates +from kafka.future import Future +from kafka.protocol.metadata import MetadataResponse, MetadataRequest + + +class TestAsyncKafkaClient(unittest.TestCase): + + def test_init(self): + with patch.object(KafkaClient, '_bootstrap') as bootstrap: + + KafkaClient() + bootstrap.assert_called_with([('localhost', 9092)]) + + other_test_cases = [ + ('foobar:1234', [('foobar', 1234)]), + ('fizzbuzz', [('fizzbuzz', 9092)]), + ('foo:12,bar:34', [('foo', 12), ('bar', 34)]), + (['fizz:56', 'buzz'], [('fizz', 56), ('buzz', 9092)]) + ] + for arg, test in other_test_cases: + KafkaClient(bootstrap_servers=arg) + # host order is randomized internally, so resort before testing + (hosts,), _ = bootstrap.call_args + assert sorted(hosts) == sorted(test) + + @patch('kafka.client_async.BrokerConnection') + def test_bootstrap(self, conn): + conn.return_value = conn + conn.state = ConnectionStates.CONNECTED + conn.send.return_value = Future().success(MetadataResponse( + [(0, 'foo', 12), (1, 'bar', 34)], [])) + + cli = KafkaClient() + conn.assert_called_once_with('localhost', 9092, **cli.config) + conn.connect.assert_called_with() + conn.send.assert_called_once_with(MetadataRequest([])) + assert cli._bootstrap_fails == 0 + assert cli.cluster.brokers() == set([BrokerMetadata(0, 'foo', 12), + BrokerMetadata(1, 'bar', 34)]) + + conn.state = ConnectionStates.DISCONNECTED + cli = KafkaClient() + conn.connect.assert_called_with() + conn.close.assert_called_with() + assert cli._bootstrap_fails == 1 + + def test_can_connect(self): + pass + + def test_initiate_connect(self): + pass + + def test_finish_connect(self): + pass + + def test_ready(self): + pass + + def test_close(self): + pass + + def test_is_disconnected(self): + pass + + def test_is_ready(self): + pass + + def test_can_send_request(self): + pass + + def test_send(self): + pass + + def test_poll(self): + pass + + def test__poll(self): + pass + + def test_in_flight_request_count(self): + pass + + def test_least_loaded_node(self): + pass + + def test_set_topics(self): + pass + + def test_maybe_refresh_metadata(self): + pass + + def test_schedule(self): + pass + + def test_unschedule(self): + pass + -- cgit v1.2.1 From cb325af5ffc3f711842e103130a87d5b073a9ff5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 15:03:36 -0800 Subject: Use find_packages() for setup.py --- setup.py | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/setup.py b/setup.py index 5fc2417..8a2a08a 100644 --- a/setup.py +++ b/setup.py @@ -1,6 +1,6 @@ import sys import os -from setuptools import setup, Command +from setuptools import setup, Command, find_packages # Pull version from source without importing # since we can't import something we haven't built yet :) @@ -37,14 +37,7 @@ setup( tests_require=test_require, cmdclass={"test": Tox}, - - packages=[ - "kafka", - "kafka.consumer", - "kafka.partitioner", - "kafka.producer", - ], - + packages=find_packages(exclude=['test']), author="Dana Powers", author_email="dana.powers@gmail.com", url="https://github.com/dpkp/kafka-python", -- cgit v1.2.1 From 1a6ff2615cebde0dab49151b4b8a15fc97edb4e6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 15:04:51 -0800 Subject: Use pytest instead of nose for running tests via tox - use pytest-sugar for pretty output - add linting to each python tox environment - drop lint as separate tox target - replace travis_selector.sh with shell magic --- .gitignore | 3 ++- .travis.yml | 2 +- tox.ini | 43 +++++++++---------------------------------- travis_selector.sh | 18 ------------------ 4 files changed, 12 insertions(+), 54 deletions(-) delete mode 100755 travis_selector.sh diff --git a/.gitignore b/.gitignore index 30d663d..3e7c09a 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ dist MANIFEST env servers/*/kafka-bin -.coverage +.coverage* .noseids docs/_build +.cache* diff --git a/.travis.yml b/.travis.yml index 1f0baa6..64d019c 100644 --- a/.travis.yml +++ b/.travis.yml @@ -48,7 +48,7 @@ deploy: # branch: master script: - - if [ -n "$UNIT_AND_LINT_ONLY" ]; then tox -e lint,`./travis_selector.sh $TRAVIS_PYTHON_VERSION`; else tox -e `./travis_selector.sh $TRAVIS_PYTHON_VERSION`; fi + - tox -e ${TRAVIS_PYTHON_VERSION/./} after_success: - coveralls diff --git a/tox.ini b/tox.ini index 1ee1e16..b00d531 100644 --- a/tox.ini +++ b/tox.ini @@ -1,48 +1,23 @@ [tox] -envlist = lint, py26, py27, pypy, py33, py34, py35, docs +envlist = py{26,27,py,33,34,35}, docs [testenv] deps = - nose - nose-timer - coverage + pytest + pytest-cov + pytest-catchlog + pytest-pylint + pytest-sugar mock python-snappy + py{26,27}: six + py26: unittest2 commands = - nosetests {posargs:-v -x --with-id --id-file={envdir}/.noseids --with-timer --timer-top-n 10 --with-coverage --cover-erase --cover-package kafka} + py.test {posargs:--pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --durations=10 --cov=kafka --doctest-modules kafka test} setenv = - NOSE_LOGFORMAT = %(asctime)s - %(thread)d - %(name)s - %(levelname)s - %(message)s PROJECT_ROOT = {toxinidir} passenv = KAFKA_VERSION -[testenv:py26] -deps = - six - unittest2 - nose - nose-timer - coverage - mock - python-snappy - -[testenv:py27] -deps = - six - unittest2 - nose - nose-timer - coverage - mock - python-snappy - -[testenv:lint] -basepython = python2.7 -deps = - unittest2 - mock - pylint -commands = pylint --rcfile=pylint.rc {posargs: -E kafka test} - [testenv:docs] deps = sphinxcontrib-napoleon diff --git a/travis_selector.sh b/travis_selector.sh deleted file mode 100755 index 5700cbc..0000000 --- a/travis_selector.sh +++ /dev/null @@ -1,18 +0,0 @@ -#!/bin/bash -# This works with the .travis.yml file to select a python version for testing - -if [ $1 == "pypy" ]; then - echo "pypy" -elif [ $1 == "3.5" ]; then - echo "py35" -elif [ $1 == "3.4" ]; then - echo "py34" -elif [ $1 == "3.3" ]; then - echo "py33" -elif [ $1 == "2.7" ]; then - echo "py27" -elif [ $1 == "2.6" ]; then - echo "py26" -else - echo $1 -fi; -- cgit v1.2.1 From b957de1fd965d1deb43a81c80647bb29b3528c27 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 22:48:24 -0800 Subject: sys.maxint not supported in python3 --- kafka/client_async.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index d71c9a4..190384e 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -4,7 +4,6 @@ import itertools import logging import random import select -import sys import time import six @@ -360,7 +359,7 @@ class KafkaClient(object): """ nodes = list(self._conns.keys()) random.shuffle(nodes) - inflight = sys.maxint + inflight = float('inf') found = None for node_id in nodes: conn = self._conns[node_id] @@ -421,7 +420,7 @@ class KafkaClient(object): return ttl if self._metadata_refresh_in_progress: - return sys.maxint + return 9999999999 node_id = self.least_loaded_node() @@ -530,7 +529,7 @@ class DelayedTaskQueue(object): """Number of seconds until next task is ready.""" self._drop_removed() if not self._tasks: - return sys.maxint + return 9999999999 else: return max(self._tasks[0][0] - time.time(), 0) -- cgit v1.2.1 From cb1b52f7056591d3964c7e4ba12c8fb21085b135 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 22:49:34 -0800 Subject: Use python3 compatible next() in roundrobin assignor --- kafka/coordinator/assignors/roundrobin.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/coordinator/assignors/roundrobin.py b/kafka/coordinator/assignors/roundrobin.py index 2927f3e..55b73e1 100644 --- a/kafka/coordinator/assignors/roundrobin.py +++ b/kafka/coordinator/assignors/roundrobin.py @@ -36,14 +36,14 @@ class RoundRobinPartitionAssignor(AbstractPartitionAssignor): member_iter = itertools.cycle(sorted(member_metadata.keys())) for partition in all_topic_partitions: - member_id = member_iter.next() + member_id = next(member_iter) # Because we constructed all_topic_partitions from the set of # member subscribed topics, we should be safe assuming that # each topic in all_topic_partitions is in at least one member # subscription; otherwise this could yield an infinite loop while partition.topic not in member_metadata[member_id].subscription: - member_id = member_iter.next() + member_id = next(member_iter) assignment[member_id][partition.topic].append(partition.partition) protocol_assignment = {} -- cgit v1.2.1 From eab50649297033a7c0883fb30b7f6e0ade77b603 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 22:57:08 -0800 Subject: Use log exception for metadata failure in async producer --- kafka/producer/base.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 4972cd4..d733172 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -196,8 +196,8 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, log.warn('Async producer forcing metadata refresh metadata before retrying') try: client.load_metadata_for_topics() - except Exception as e: - log.error("Async producer couldn't reload topic metadata. Error: `%s`", e.message) + except Exception: + log.exception("Async producer couldn't reload topic metadata.") # Apply retry limit, dropping messages that are over request_tries = dict( -- cgit v1.2.1 From b6e9b7f74dd1f48b5331600fc4c80406c35c6993 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 22:59:55 -0800 Subject: Disable pylint errors for py2/py3 compatibility workarounds --- kafka/consumer/base.py | 3 ++- kafka/consumer/simple.py | 2 +- kafka/producer/base.py | 7 ++++--- kafka/protocol/pickle.py | 4 ++-- test/__init__.py | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index 4ac8c66..a90038f 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -197,7 +197,8 @@ class Consumer(object): # ValueError on list.remove() if the exithandler no longer # exists is fine here try: - atexit._exithandlers.remove((self._cleanup_func, (self,), {})) + atexit._exithandlers.remove( # pylint: disable=no-member + (self._cleanup_func, (self,), {})) except ValueError: pass diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 9c2812b..946e9c7 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -3,7 +3,7 @@ from __future__ import absolute_import try: from itertools import zip_longest as izip_longest, repeat # pylint: disable=E0611 except ImportError: - from itertools import izip_longest as izip_longest, repeat # python 2 + from itertools import izip_longest as izip_longest, repeat # pylint: disable=E0611 import logging try: import queue # python 3 diff --git a/kafka/producer/base.py b/kafka/producer/base.py index d733172..4f5edbc 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -5,9 +5,9 @@ import logging import time try: - from queue import Empty, Full, Queue + from queue import Empty, Full, Queue # pylint: disable=import-error except ImportError: - from Queue import Empty, Full, Queue + from Queue import Empty, Full, Queue # pylint: disable=import-error from collections import defaultdict from threading import Thread, Event @@ -444,7 +444,8 @@ class Producer(object): # ValueError on list.remove() if the exithandler no longer exists # but that is fine here try: - atexit._exithandlers.remove((self._cleanup_func, (self,), {})) + atexit._exithandlers.remove( # pylint: disable=no-member + (self._cleanup_func, (self,), {})) except ValueError: pass diff --git a/kafka/protocol/pickle.py b/kafka/protocol/pickle.py index 2265efd..b7e5264 100644 --- a/kafka/protocol/pickle.py +++ b/kafka/protocol/pickle.py @@ -1,9 +1,9 @@ from __future__ import absolute_import try: - import copyreg + import copyreg # pylint: disable=import-error except ImportError: - import copy_reg as copyreg # python2 + import copy_reg as copyreg # pylint: disable=import-error import types diff --git a/test/__init__.py b/test/__init__.py index c4d1e80..da1069f 100644 --- a/test/__init__.py +++ b/test/__init__.py @@ -1,6 +1,6 @@ import sys if sys.version_info < (2, 7): - import unittest2 as unittest + import unittest2 as unittest # pylint: disable=import-error else: import unittest -- cgit v1.2.1 From 77350fe0ca8bd9440c978b07a670b7a4188abcbd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 1 Jan 2016 23:24:43 -0800 Subject: Add default sphinx docs for kafka.coordinator and kafka.protocol --- docs/apidoc/kafka.coordinator.assignors.rst | 30 +++++++ docs/apidoc/kafka.coordinator.rst | 45 ++++++++++ docs/apidoc/kafka.protocol.rst | 126 ++++++++++++++++++++++++++++ 3 files changed, 201 insertions(+) create mode 100644 docs/apidoc/kafka.coordinator.assignors.rst create mode 100644 docs/apidoc/kafka.coordinator.rst create mode 100644 docs/apidoc/kafka.protocol.rst diff --git a/docs/apidoc/kafka.coordinator.assignors.rst b/docs/apidoc/kafka.coordinator.assignors.rst new file mode 100644 index 0000000..87b9f84 --- /dev/null +++ b/docs/apidoc/kafka.coordinator.assignors.rst @@ -0,0 +1,30 @@ +kafka.coordinator.assignors package +=================================== + +Submodules +---------- + +kafka.coordinator.assignors.abstract module +------------------------------------------- + +.. automodule:: kafka.coordinator.assignors.abstract + :members: + :undoc-members: + :show-inheritance: + +kafka.coordinator.assignors.roundrobin module +--------------------------------------------- + +.. automodule:: kafka.coordinator.assignors.roundrobin + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka.coordinator.assignors + :members: + :undoc-members: + :show-inheritance: diff --git a/docs/apidoc/kafka.coordinator.rst b/docs/apidoc/kafka.coordinator.rst new file mode 100644 index 0000000..e15f638 --- /dev/null +++ b/docs/apidoc/kafka.coordinator.rst @@ -0,0 +1,45 @@ +kafka.coordinator package +========================= + +Subpackages +----------- + +.. toctree:: + + kafka.coordinator.assignors + +Submodules +---------- + +kafka.coordinator.base module +----------------------------- + +.. automodule:: kafka.coordinator.base + :members: + :undoc-members: + :show-inheritance: + +kafka.coordinator.consumer module +--------------------------------- + +.. automodule:: kafka.coordinator.consumer + :members: + :undoc-members: + :show-inheritance: + +kafka.coordinator.heartbeat module +---------------------------------- + +.. automodule:: kafka.coordinator.heartbeat + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka.coordinator + :members: + :undoc-members: + :show-inheritance: diff --git a/docs/apidoc/kafka.protocol.rst b/docs/apidoc/kafka.protocol.rst new file mode 100644 index 0000000..4e69aaf --- /dev/null +++ b/docs/apidoc/kafka.protocol.rst @@ -0,0 +1,126 @@ +kafka.protocol package +====================== + +Submodules +---------- + +kafka.protocol.abstract module +------------------------------ + +.. automodule:: kafka.protocol.abstract + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.admin module +--------------------------- + +.. automodule:: kafka.protocol.admin + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.api module +------------------------- + +.. automodule:: kafka.protocol.api + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.commit module +---------------------------- + +.. automodule:: kafka.protocol.commit + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.fetch module +--------------------------- + +.. automodule:: kafka.protocol.fetch + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.group module +--------------------------- + +.. automodule:: kafka.protocol.group + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.legacy module +---------------------------- + +.. automodule:: kafka.protocol.legacy + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.message module +----------------------------- + +.. automodule:: kafka.protocol.message + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.metadata module +------------------------------ + +.. automodule:: kafka.protocol.metadata + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.offset module +---------------------------- + +.. automodule:: kafka.protocol.offset + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.pickle module +---------------------------- + +.. automodule:: kafka.protocol.pickle + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.produce module +----------------------------- + +.. automodule:: kafka.protocol.produce + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.struct module +---------------------------- + +.. automodule:: kafka.protocol.struct + :members: + :undoc-members: + :show-inheritance: + +kafka.protocol.types module +--------------------------- + +.. automodule:: kafka.protocol.types + :members: + :undoc-members: + :show-inheritance: + + +Module contents +--------------- + +.. automodule:: kafka.protocol + :members: + :undoc-members: + :show-inheritance: -- cgit v1.2.1 From 976970f89acfdb3582feed613722158004b0ff3e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 2 Jan 2016 13:12:29 -0800 Subject: Fixup b6a2ad9: Fail with ConnectionErrors in BrokerConnection.send --- kafka/conn.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index a1767ef..0e02382 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -165,8 +165,9 @@ class BrokerConnection(object): self._sock.setblocking(False) except (AssertionError, socket.error) as e: log.exception("Error sending %s to %s", request, self) - self.close(error=Errors.ConnectionError(e)) - return future.failure(e) + error = Errors.ConnectionError(e) + self.close(error=error) + return future.failure(error) log.debug('%s Request %d: %s', self, correlation_id, request) if expect_response: -- cgit v1.2.1 From 24a4c2a7c5a1265899316aca86a1149496d6564e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 2 Jan 2016 13:18:10 -0800 Subject: Improve iterator interface - Support single message consumption via next(consumer) in py2/py3 - batch message methods (Fetcher.fetched_records / KafkaConsumer.poll) are incompatible with iterators -- message generator state keeps messages internally after they are popped from _records, but before subscription_state is updated. --- kafka/consumer/fetcher.py | 28 ++++++++++++++++++++++------ kafka/consumer/group.py | 28 ++++++++++++++++++++++------ 2 files changed, 44 insertions(+), 12 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 5e15424..ddf9d6f 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -28,7 +28,7 @@ class RecordTooLargeError(Errors.KafkaError): pass -class Fetcher(object): +class Fetcher(six.Iterator): DEFAULT_CONFIG = { 'key_deserializer': None, 'value_deserializer': None, @@ -79,6 +79,7 @@ class Fetcher(object): self._unauthorized_topics = set() self._offset_out_of_range_partitions = dict() # {topic_partition: offset} self._record_too_large_partitions = dict() # {topic_partition: offset} + self._iterator = None #self.sensors = FetchManagerMetrics(metrics, metric_group_prefix) @@ -253,7 +254,7 @@ class Fetcher(object): def fetched_records(self): """Returns previously fetched records and updates consumed offsets. - NOTE: returning empty records guarantees the consumed position are NOT updated. + Incompatible with iterator interface - use one or the other, not both. Raises: OffsetOutOfRangeError: if no subscription offset_reset_strategy @@ -263,10 +264,13 @@ class Fetcher(object): configured max_partition_fetch_bytes TopicAuthorizationError: if consumer is not authorized to fetch messages from the topic + AssertionError: if used with iterator (incompatible) Returns: dict: {TopicPartition: deque([messages])} """ + assert self._iterator is None, ( + 'fetched_records is incompatible with message iterator') if self._subscriptions.needs_partition_assignment: return {} @@ -324,7 +328,7 @@ class Fetcher(object): key, value = self._deserialize(msg) yield ConsumerRecord(tp.topic, tp.partition, offset, key, value) - def __iter__(self): + def _message_generator(self): """Iterate over fetched_records""" if self._subscriptions.needs_partition_assignment: raise StopIteration('Subscription needs partition assignment') @@ -342,7 +346,7 @@ class Fetcher(object): # this can happen when a rebalance happened before # fetched records are returned log.warning("Not returning fetched records for partition %s" - " since it is no longer assigned", tp) + " since it is no longer assigned", tp) continue # note that the consumed position should always be available @@ -352,7 +356,7 @@ class Fetcher(object): # this can happen when a partition consumption paused before # fetched records are returned log.warning("Not returning fetched records for assigned partition" - " %s since it is no longer fetchable", tp) + " %s since it is no longer fetchable", tp) # we also need to reset the fetch positions to pretend we did # not fetch this partition in the previous request at all @@ -366,13 +370,25 @@ class Fetcher(object): # these records aren't next in line based on the last consumed # position, ignore them they must be from an obsolete request log.warning("Ignoring fetched records for %s at offset %s", - tp, fetch_offset) + tp, fetch_offset) # Send any additional FetchRequests that we can now # this will likely fetch each partition individually, rather than # fetch multiple partitions in bulk when they are on the same broker self.init_fetches() + def __iter__(self): + return self + + def __next__(self): + if not self._iterator: + self._iterator = self._message_generator() + try: + return next(self._iterator) + except StopIteration: + self._iterator = None + raise + def _deserialize(self, msg): if self.config['key_deserializer']: key = self.config['key_deserializer'](msg.key) # pylint: disable-msg=not-callable diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 5278214..cea2e1c 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -4,6 +4,8 @@ import copy import logging import time +import six + from kafka.client_async import KafkaClient from kafka.consumer.fetcher import Fetcher from kafka.consumer.subscription_state import SubscriptionState @@ -15,7 +17,7 @@ from kafka.version import __version__ log = logging.getLogger(__name__) -class KafkaConsumer(object): +class KafkaConsumer(six.Iterator): """Consumer for Kafka 0.9""" DEFAULT_CONFIG = { 'bootstrap_servers': 'localhost', @@ -160,6 +162,7 @@ class KafkaConsumer(object): assignors=self.config['partition_assignment_strategy'], **self.config) self._closed = False + self._iterator = None #self.metrics = None if topics: @@ -324,16 +327,16 @@ class KafkaConsumer(object): return self._client.cluster.partitions_for_topic(topic) def poll(self, timeout_ms=0): - """ - Fetch data for the topics or partitions specified using one of the - subscribe/assign APIs. It is an error to not have subscribed to any - topics or partitions before polling for data. + """Fetch data from assigned topics / partitions. + Records are fetched and returned in batches by topic-partition. On each poll, consumer will try to use the last consumed offset as the starting offset and fetch sequentially. The last consumed offset can be manually set through seek(partition, offset) or automatically set as the last committed offset for the subscribed list of partitions. + Incompatible with iterator interface -- use one or the other, not both. + Arguments: timeout_ms (int, optional): milliseconds to spend waiting in poll if data is not available. If 0, returns immediately with any @@ -344,6 +347,7 @@ class KafkaConsumer(object): subscribed list of topics and partitions """ assert timeout_ms >= 0, 'Timeout must not be negative' + assert self._iterator is None, 'Incompatible with iterator interface' # poll for new data until the timeout expires start = time.time() @@ -564,7 +568,7 @@ class KafkaConsumer(object): # then do any offset lookups in case some positions are not known self._fetcher.update_fetch_positions(partitions) - def __iter__(self): + def _message_generator(self): while True: self._coordinator.ensure_coordinator_known() @@ -585,3 +589,15 @@ class KafkaConsumer(object): yield msg if time.time() > timeout: break + + def __iter__(self): + return self + + def __next__(self): + if not self._iterator: + self._iterator = self._message_generator() + try: + return next(self._iterator) + except StopIteration: + self._iterator = None + raise -- cgit v1.2.1 From 00a1e6e81935b3a32d339eeb3f9be316d42642c4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 2 Jan 2016 13:21:16 -0800 Subject: Batched message methods now return dict of lists --- kafka/consumer/fetcher.py | 4 ++-- kafka/consumer/group.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index ddf9d6f..8f25561 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -267,14 +267,14 @@ class Fetcher(six.Iterator): AssertionError: if used with iterator (incompatible) Returns: - dict: {TopicPartition: deque([messages])} + dict: {TopicPartition: [messages]} """ assert self._iterator is None, ( 'fetched_records is incompatible with message iterator') if self._subscriptions.needs_partition_assignment: return {} - drained = collections.defaultdict(collections.deque) + drained = collections.defaultdict(list) self._raise_if_offset_out_of_range() self._raise_if_unauthorized_topics() self._raise_if_record_too_large() diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index cea2e1c..33563b8 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -343,7 +343,7 @@ class KafkaConsumer(six.Iterator): records that are available now. Must not be negative. Default: 0 Returns: - dict: topic to deque of records since the last fetch for the + dict: topic to list of records since the last fetch for the subscribed list of topics and partitions """ assert timeout_ms >= 0, 'Timeout must not be negative' @@ -377,7 +377,7 @@ class KafkaConsumer(six.Iterator): timeout_ms (int): The maximum time in milliseconds to block Returns: - dict: map of topic to deque of records (may be empty) + dict: map of topic to list of records (may be empty) """ # TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) self._coordinator.ensure_coordinator_known() -- cgit v1.2.1 From 2b289224f9dcfa47059717a136230ca1f5b07b6d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 2 Jan 2016 13:22:33 -0800 Subject: Raise exception in KafkaConsumer on unrecognized kwargs --- kafka/consumer/group.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 33563b8..b8b5bde 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -151,7 +151,10 @@ class KafkaConsumer(six.Iterator): self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: - self.config[key] = configs[key] + self.config[key] = configs.pop(key) + + # Only check for extra config keys in top-level class + assert not configs, 'Unrecognized configs: %s' % configs self._client = KafkaClient(**self.config) self._subscription = SubscriptionState(self.config['auto_offset_reset']) -- cgit v1.2.1 From 995ea64b8fe13f9a223e67b646ac6a1430b07cb8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 15:43:43 -0800 Subject: Fix timeout bug in BrokerConnection.connect() --- kafka/conn.py | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 0e02382..e13913f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -98,14 +98,15 @@ class BrokerConnection(object): self.close() # error=TimeoutError ? self.last_failure = time.time() - ret = self._sock.connect_ex((self.host, self.port)) - if not ret or ret is errno.EISCONN: - self.state = ConnectionStates.CONNECTED - elif ret is not errno.EALREADY: - log.error('Connect attempt to %s returned error %s.' - ' Disconnecting.', self, ret) - self.close() - self.last_failure = time.time() + else: + ret = self._sock.connect_ex((self.host, self.port)) + if not ret or ret is errno.EISCONN: + self.state = ConnectionStates.CONNECTED + elif ret is not errno.EALREADY: + log.error('Connect attempt to %s returned error %s.' + ' Disconnecting.', self, ret) + self.close() + self.last_failure = time.time() return self.state def blacked_out(self): -- cgit v1.2.1 From 4dc1fbab30b7cbff13b8f12424aa4cac512995d8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 15:45:18 -0800 Subject: Fix bug in _initiate_connect preventing reconnect to 'bootstrap' --- kafka/client_async.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 190384e..8c61288 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -131,10 +131,10 @@ class KafkaClient(object): def _initiate_connect(self, node_id): """Initiate a connection to the given node (must be in metadata)""" - broker = self.cluster.broker_metadata(node_id) - assert broker, 'Broker id %s not in current metadata' % node_id - if node_id not in self._conns: + broker = self.cluster.broker_metadata(node_id) + assert broker, 'Broker id %s not in current metadata' % node_id + log.debug("Initiating connection to node %s at %s:%s", node_id, broker.host, broker.port) self._conns[node_id] = BrokerConnection(broker.host, broker.port, -- cgit v1.2.1 From 30fefa9b4f6922b97536b5641ec696dcc8257601 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 15:46:37 -0800 Subject: Add KafkaClient.check_version() to guess broker version --- kafka/client_async.py | 52 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/kafka/client_async.py b/kafka/client_async.py index 8c61288..8a92159 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -471,6 +471,58 @@ class KafkaClient(object): """ self._delayed_tasks.remove(task) + def check_version(self, node_id=None): + """Attempt to guess the broker version""" + if node_id is None: + node_id = self.least_loaded_node() + + def connect(): + timeout = time.time() + 10 + # brokers < 0.9 do not return any broker metadata if there are no topics + # so we're left with a single bootstrap connection + while not self.ready(node_id): + if time.time() >= timeout: + raise Errors.NodeNotReadyError(node_id) + time.sleep(0.025) + + # kafka kills the connection when it doesnt recognize an API request + # so we can send a test request and then follow immediately with a + # vanilla MetadataRequest. If the server did not recognize the first + # request, both will be failed with a ConnectionError that wraps + # socket.error (32 or 54) + import socket + from .protocol.admin import ListGroupsRequest + from .protocol.commit import ( + OffsetFetchRequest_v0, GroupCoordinatorRequest) + from .protocol.metadata import MetadataRequest + + test_cases = [ + ('0.9', ListGroupsRequest()), + ('0.8.2', GroupCoordinatorRequest('kafka-python-default-group')), + ('0.8.1', OffsetFetchRequest_v0('kafka-python-default-group', [])), + ('0.8.0', MetadataRequest([])), + ] + + + for version, request in test_cases: + connect() + f = self.send(node_id, request) + time.sleep(0.5) + self.send(node_id, MetadataRequest([])) + self.poll(future=f) + + assert f.is_done + + if f.succeeded(): + log.info('Broker version identifed as %s', version) + return version + + assert isinstance(f.exception.message, socket.error) + assert f.exception.message.errno in (32, 54) + log.info("Broker is not v%s -- it did not recognize %s", + version, request.__class__.__name__) + continue + class DelayedTaskQueue(object): # see https://docs.python.org/2/library/heapq.html -- cgit v1.2.1 From fae1a227b1eb67fda2264d81c36cdbe39b49e057 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:00:16 -0800 Subject: Add api_version config to KafkaConsumer; disable features inline by version --- kafka/consumer/group.py | 47 ++++++++++++++++++++++++++++++++++++----------- 1 file changed, 36 insertions(+), 11 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index b8b5bde..a9a4ac0 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -42,6 +42,7 @@ class KafkaConsumer(six.Iterator): 'session_timeout_ms': 30000, 'send_buffer_bytes': 128 * 1024, 'receive_buffer_bytes': 32 * 1024, + 'api_version': 'auto', 'connections_max_idle_ms': 9 * 60 * 1000, # not implemented yet #'metric_reporters': None, #'metrics_num_samples': 2, @@ -144,6 +145,12 @@ class KafkaConsumer(six.Iterator): (SO_SNDBUF) to use when sending data. Default: 131072 receive_buffer_bytes (int): The size of the TCP receive buffer (SO_RCVBUF) to use when reading data. Default: 32768 + api_version (str): specify which kafka API version to use. + 0.9 enables full group coordination features; 0.8.2 enables + kafka-storage offset commits; 0.8.1 enables zookeeper-storage + offset commits; 0.8.0 is what is left. If set to 'auto', will + attempt to infer the broker version by probing various APIs. + Default: auto Configuration parameters are described in more detail at https://kafka.apache.org/090/configuration.html#newconsumerconfigs @@ -157,6 +164,16 @@ class KafkaConsumer(six.Iterator): assert not configs, 'Unrecognized configs: %s' % configs self._client = KafkaClient(**self.config) + + # Check Broker Version if not set explicitly + if self.config['api_version'] == 'auto': + self.config['api_version'] = self._client.check_version() + assert self.config['api_version'] in ('0.9', '0.8.2', '0.8.1', '0.8.0') + + # Convert api_version config to tuple for easy comparisons + self.config['api_version'] = tuple( + map(int, self.config['api_version'].split('.'))) + self._subscription = SubscriptionState(self.config['auto_offset_reset']) self._fetcher = Fetcher( self._client, self._subscription, **self.config) @@ -250,6 +267,7 @@ class KafkaConsumer(six.Iterator): Returns: kafka.future.Future """ + assert self.config['api_version'] >= (0, 8, 1) if offsets is None: offsets = self._subscription.all_consumed_offsets() log.debug("Committing offsets: %s", offsets) @@ -275,6 +293,7 @@ class KafkaConsumer(six.Iterator): to commit with the configured group_id. Defaults to current consumed offsets for all subscribed partitions. """ + assert self.config['api_version'] >= (0, 8, 1) if offsets is None: offsets = self._subscription.all_consumed_offsets() self._coordinator.commit_offsets_sync(offsets) @@ -295,6 +314,7 @@ class KafkaConsumer(six.Iterator): Returns: The last committed offset, or None if there was no prior commit. """ + assert self.config['api_version'] >= (0, 8, 1) if self._subscription.is_assigned(partition): committed = self._subscription.assignment[partition].committed if committed is None: @@ -382,12 +402,14 @@ class KafkaConsumer(six.Iterator): Returns: dict: map of topic to list of records (may be empty) """ - # TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) - self._coordinator.ensure_coordinator_known() + if self.config['api_version'] >= (0, 8, 2): + # TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) + self._coordinator.ensure_coordinator_known() - # ensure we have partitions assigned if we expect to - if self._subscription.partitions_auto_assigned(): - self._coordinator.ensure_active_group() + if self.config['api_version'] >= (0, 9): + # ensure we have partitions assigned if we expect to + if self._subscription.partitions_auto_assigned(): + self._coordinator.ensure_active_group() # fetch positions if we have partitions we're subscribed to that we # don't know the offset for @@ -565,19 +587,22 @@ class KafkaConsumer(six.Iterator): NoOffsetForPartitionError: If no offset is stored for a given partition and no offset reset policy is defined """ - # refresh commits for all assigned partitions - self._coordinator.refresh_committed_offsets_if_needed() + if self.config['api_version'] >= (0, 8, 1): + # refresh commits for all assigned partitions + self._coordinator.refresh_committed_offsets_if_needed() # then do any offset lookups in case some positions are not known self._fetcher.update_fetch_positions(partitions) def _message_generator(self): while True: - self._coordinator.ensure_coordinator_known() + if self.config['api_version'] >= (0, 8, 2): + self._coordinator.ensure_coordinator_known() - # ensure we have partitions assigned if we expect to - if self._subscription.partitions_auto_assigned(): - self._coordinator.ensure_active_group() + if self.config['api_version'] >= (0, 9): + # ensure we have partitions assigned if we expect to + if self._subscription.partitions_auto_assigned(): + self._coordinator.ensure_active_group() # fetch positions if we have partitions we're subscribed to that we # don't know the offset for -- cgit v1.2.1 From 5c45ec13f3e59d9c398f2d3035c762ca13589885 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:03:30 -0800 Subject: Check api_version in ConsumerCoordinator - Full group support in 0.9 - Kafka-storage offsets w/ GroupCoordinator in 0.8.2 - Zookeeper-storage offsets in 0.8.1 - Assign all partitions locally if < 0.9 --- kafka/coordinator/consumer.py | 126 +++++++++++++++++++++++++++++++----------- 1 file changed, 94 insertions(+), 32 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 99d62f2..673cbaf 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -9,7 +9,9 @@ from .base import BaseCoordinator import kafka.common as Errors from kafka.common import OffsetAndMetadata, TopicPartition from kafka.future import Future -from kafka.protocol.commit import OffsetCommitRequest_v2, OffsetFetchRequest_v1 +from kafka.protocol.commit import ( + OffsetCommitRequest_v2, OffsetCommitRequest_v1, OffsetCommitRequest_v0, + OffsetFetchRequest_v0, OffsetFetchRequest_v1) from kafka.protocol.struct import Struct from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String @@ -55,6 +57,7 @@ class ConsumerCoordinator(BaseCoordinator): 'session_timeout_ms': 30000, 'heartbeat_interval_ms': 3000, 'retry_backoff_ms': 100, + 'api_version': (0, 9), } def __init__(self, client, subscription, **configs): @@ -99,14 +102,16 @@ class ConsumerCoordinator(BaseCoordinator): self._subscription = subscription self._partitions_per_topic = {} self._auto_commit_task = None - assert self.config['assignors'], 'Coordinator require assignors' + if self.config['api_version'] >= (0, 9): + assert self.config['assignors'], 'Coordinator require assignors' self._cluster.request_update() self._cluster.add_listener(self._handle_metadata_update) - if self.config['enable_auto_commit']: - interval = self.config['auto_commit_interval_ms'] / 1000.0 - self._auto_commit_task = AutoCommitTask(self, interval) + if self.config['api_version'] >= (0, 8, 1): + if self.config['enable_auto_commit']: + interval = self.config['auto_commit_interval_ms'] / 1000.0 + self._auto_commit_task = AutoCommitTask(self, interval) # metrics=None, # metric_group_prefix=None, @@ -143,7 +148,17 @@ class ConsumerCoordinator(BaseCoordinator): # check if there are any changes to the metadata which should trigger a rebalance if self._subscription_metadata_changed(): - self._subscription.mark_for_reassignment() + if self.config['api_version'] >= (0, 9): + self._subscription.mark_for_reassignment() + + # If we haven't got group coordinator support, + # just assign all partitions locally + else: + self._subscription.assign_from_subscribed([ + TopicPartition(topic, partition) + for topic in self._subscription.subscription + for partition in self._partitions_per_topic[topic] + ]) def _subscription_metadata_changed(self): if not self._subscription.partitions_auto_assigned(): @@ -273,7 +288,8 @@ class ConsumerCoordinator(BaseCoordinator): dict: {TopicPartition: OffsetAndMetadata} """ while True: - self.ensure_coordinator_known() + if self.config['api_version'] >= (0, 8, 2): + self.ensure_coordinator_known() # contact coordinator to fetch committed offsets future = self._send_offset_fetch_request(partitions) @@ -331,7 +347,8 @@ class ConsumerCoordinator(BaseCoordinator): return while True: - self.ensure_coordinator_known() + if self.config['api_version'] >= (0, 8, 2): + self.ensure_coordinator_known() future = self._send_offset_commit_request(offsets) self._client.poll(future=future) @@ -345,6 +362,8 @@ class ConsumerCoordinator(BaseCoordinator): time.sleep(self.config['retry_backoff_ms'] / 1000.0) def _maybe_auto_commit_offsets_sync(self): + if self.config['api_version'] < (0, 8, 1): + return if self.config['enable_auto_commit']: # disable periodic commits prior to committing synchronously. note that they will # be re-enabled after a rebalance completes @@ -379,8 +398,12 @@ class ConsumerCoordinator(BaseCoordinator): Returns: Future: indicating whether the commit was successful or not """ - if self.coordinator_unknown(): - return Future().failure(Errors.GroupCoordinatorNotAvailableError) + if self.config['api_version'] >= (0, 8, 2): + if self.coordinator_unknown(): + return Future().failure(Errors.GroupCoordinatorNotAvailableError) + node_id = self.coordinator_id + else: + node_id = self._client.least_loaded_node() if not offsets: return Future().failure(None) @@ -390,25 +413,49 @@ class ConsumerCoordinator(BaseCoordinator): for tp, offset in six.iteritems(offsets): offset_data[tp.topic][tp.partition] = offset - request = OffsetCommitRequest_v2( - self.group_id, - self.generation, - self.member_id, - OffsetCommitRequest_v2.DEFAULT_RETENTION_TIME, - [( - topic, [( - partition, - offset.offset, - offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] - ) + if self.config['api_version'] >= (0, 9): + request = OffsetCommitRequest_v2( + self.group_id, + self.generation, + self.member_id, + OffsetCommitRequest_v2.DEFAULT_RETENTION_TIME, + [( + topic, [( + partition, + offset.offset, + offset.metadata + ) for partition, offset in six.iteritems(partitions)] + ) for topic, partitions in six.iteritems(offset_data)] + ) + elif self.config['api_version'] >= (0, 8, 2): + request = OffsetCommitRequest_v1( + self.group_id, -1, '', + [( + topic, [( + partition, + offset.offset, + -1, + offset.metadata + ) for partition, offset in six.iteritems(partitions)] + ) for topic, partitions in six.iteritems(offset_data)] + ) + elif self.config['api_version'] >= (0, 8, 1): + request = OffsetCommitRequest_v0( + self.group_id, + [( + topic, [( + partition, + offset.offset, + offset.metadata + ) for partition, offset in six.iteritems(partitions)] + ) for topic, partitions in six.iteritems(offset_data)] + ) log.debug("Sending offset-commit request with %s to %s", - offsets, self.coordinator_id) + offsets, node_id) future = Future() - _f = self._client.send(self.coordinator_id, request) + _f = self._client.send(node_id, request) _f.add_callback(self._handle_offset_commit_response, offsets, future) _f.add_errback(self._failed_request, future) return future @@ -495,22 +542,33 @@ class ConsumerCoordinator(BaseCoordinator): Returns: Future: resolves to dict of offsets: {TopicPartition: int} """ - if self.coordinator_unknown(): - return Future().failure(Errors.GroupCoordinatorNotAvailableError) + if self.config['api_version'] >= (0, 8, 2): + if self.coordinator_unknown(): + return Future().failure(Errors.GroupCoordinatorNotAvailableError) + node_id = self.coordinator_id + else: + node_id = self._client.least_loaded_node() log.debug("Fetching committed offsets for partitions: %s", partitions) # construct the request topic_partitions = collections.defaultdict(set) for tp in partitions: topic_partitions[tp.topic].add(tp.partition) - request = OffsetFetchRequest_v1( - self.group_id, - list(topic_partitions.items()) - ) + + if self.config['api_version'] >= (0, 8, 2): + request = OffsetFetchRequest_v1( + self.group_id, + list(topic_partitions.items()) + ) + else: + request = OffsetFetchRequest_v0( + self.group_id, + list(topic_partitions.items()) + ) # send the request with a callback future = Future() - _f = self._client.send(self.coordinator_id, request) + _f = self._client.send(node_id, request) _f.add_callback(self._handle_offset_fetch_response, future) _f.add_errback(self._failed_request, future) return future @@ -536,6 +594,10 @@ class ConsumerCoordinator(BaseCoordinator): # need to re-join group self._subscription.mark_for_reassignment() future.failure(error) + elif error_type is Errors.UnknownTopicOrPartitionError: + log.warning("OffsetFetchRequest -- unknown topic %s", + topic) + continue else: log.error("Unknown error fetching offsets for %s: %s", tp, error) -- cgit v1.2.1 From 9b07bfb5298f961b965ee4a295b0bceb52803852 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:06:35 -0800 Subject: Check for 0.8.2 GroupCoordinator quirk in BrokerConnection --- kafka/conn.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index e13913f..d713b56 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -16,6 +16,7 @@ import kafka.common as Errors from kafka.common import ConnectionError from kafka.future import Future from kafka.protocol.api import RequestHeader +from kafka.protocol.commit import GroupCoordinatorResponse from kafka.protocol.types import Int32 from kafka.version import __version__ @@ -44,6 +45,7 @@ class BrokerConnection(object): 'max_in_flight_requests_per_connection': 5, 'receive_buffer_bytes': 32768, 'send_buffer_bytes': 131072, + 'api_version': (0, 8, 2), # default to most restrictive } def __init__(self, host, port, **configs): @@ -278,7 +280,17 @@ class BrokerConnection(object): # verify send/recv correlation ids match recv_correlation_id = Int32.decode(read_buffer) - if ifr.correlation_id != recv_correlation_id: + + # 0.8.2 quirk + if (self.config['api_version'] == (0, 8, 2) and + ifr.response_type is GroupCoordinatorResponse and + recv_correlation_id == 0): + raise Errors.KafkaError( + 'Kafka 0.8.2 quirk -- try creating a topic first') + + elif ifr.correlation_id != recv_correlation_id: + + error = Errors.CorrelationIdError( 'Correlation ids do not match: sent %d, recv %d' % (ifr.correlation_id, recv_correlation_id)) -- cgit v1.2.1 From 79aa0f04892ce4f5b0e27a80654e3689ac9d7e32 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:07:24 -0800 Subject: Support consumer_timeout_ms in new KafkaConsumer --- kafka/consumer/group.py | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index a9a4ac0..6a5084d 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -42,6 +42,7 @@ class KafkaConsumer(six.Iterator): 'session_timeout_ms': 30000, 'send_buffer_bytes': 128 * 1024, 'receive_buffer_bytes': 32 * 1024, + 'consumer_timeout_ms': -1, 'api_version': 'auto', 'connections_max_idle_ms': 9 * 60 * 1000, # not implemented yet #'metric_reporters': None, @@ -145,6 +146,9 @@ class KafkaConsumer(six.Iterator): (SO_SNDBUF) to use when sending data. Default: 131072 receive_buffer_bytes (int): The size of the TCP receive buffer (SO_RCVBUF) to use when reading data. Default: 32768 + consumer_timeout_ms (int): number of millisecond to throw a timeout + exception to the consumer if no message is available for + consumption. Default: -1 (dont throw exception) api_version (str): specify which kafka API version to use. 0.9 enables full group coordination features; 0.8.2 enables kafka-storage offset commits; 0.8.1 enables zookeeper-storage @@ -183,6 +187,7 @@ class KafkaConsumer(six.Iterator): **self.config) self._closed = False self._iterator = None + self._consumer_timeout = float('inf') #self.metrics = None if topics: @@ -595,7 +600,7 @@ class KafkaConsumer(six.Iterator): self._fetcher.update_fetch_positions(partitions) def _message_generator(self): - while True: + while time.time() < self._consumer_timeout: if self.config['api_version'] >= (0, 8, 2): self._coordinator.ensure_coordinator_known() @@ -612,7 +617,11 @@ class KafkaConsumer(six.Iterator): # init any new fetches (won't resend pending fetches) self._fetcher.init_fetches() self._client.poll(self.config['request_timeout_ms'] / 1000.0) - timeout = time.time() + self.config['heartbeat_interval_ms'] / 1000.0 + timeout = self._consumer_timeout + if self.config['api_version'] >= (0, 9): + heartbeat_timeout = time.time() + ( + self.config['heartbeat_interval_ms'] / 1000.0) + timeout = min(heartbeat_timeout, timeout) for msg in self._fetcher: yield msg if time.time() > timeout: @@ -624,6 +633,12 @@ class KafkaConsumer(six.Iterator): def __next__(self): if not self._iterator: self._iterator = self._message_generator() + + # consumer_timeout_ms can be used to stop iteration early + if self.config['consumer_timeout_ms'] >= 0: + self._consumer_timeout = time.time() + ( + self.config['consumer_timeout_ms'] / 1000.0) + try: return next(self._iterator) except StopIteration: -- cgit v1.2.1 From 2b43ae1609295e33f35aa4f5d1b61d60889b420c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:08:11 -0800 Subject: Return empty dict from KafkaConsumer.poll if no records --- kafka/consumer/group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 6a5084d..44666ad 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -394,7 +394,7 @@ class KafkaConsumer(six.Iterator): remaining = timeout_ms - elapsed_ms if remaining <= 0: - break + return {} def _poll_once(self, timeout_ms): """ -- cgit v1.2.1 From 16c56235d2fa1a9b63a38bd01aeb28be9aaa2405 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:08:56 -0800 Subject: Keep support for old largest/smallest auto_offset_reset options --- kafka/consumer/group.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 44666ad..fe94d86 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -167,6 +167,13 @@ class KafkaConsumer(six.Iterator): # Only check for extra config keys in top-level class assert not configs, 'Unrecognized configs: %s' % configs + deprecated = {'smallest': 'earliest', 'largest': 'latest' } + if self.config['auto_offset_reset'] in deprecated: + new_config = deprecated[self.config['auto_offset_reset']] + log.warning('use auto_offset_reset=%s (%s is deprecated)', + new_config, self.config['auto_offset_reset']) + self.config['auto_offset_reset'] = new_config + self._client = KafkaClient(**self.config) # Check Broker Version if not set explicitly -- cgit v1.2.1 From 71af307f4e350a0e3ef21a6c986ebebb6717dc15 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:09:52 -0800 Subject: Disable pylint error on __iter__ (next is provided via six.Iterator) --- kafka/consumer/fetcher.py | 2 +- kafka/consumer/group.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 8f25561..1593018 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -377,7 +377,7 @@ class Fetcher(six.Iterator): # fetch multiple partitions in bulk when they are on the same broker self.init_fetches() - def __iter__(self): + def __iter__(self): # pylint: disable=non-iterator-returned return self def __next__(self): diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index fe94d86..0293b4c 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -634,7 +634,7 @@ class KafkaConsumer(six.Iterator): if time.time() > timeout: break - def __iter__(self): + def __iter__(self): # pylint: disable=non-iterator-returned return self def __next__(self): -- cgit v1.2.1 From abdbc6ca2785a5646ee9dccf4f5ccf700da5f648 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:10:40 -0800 Subject: Drop unused method from ConsumerCoordinator --- kafka/coordinator/consumer.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 673cbaf..48d5e14 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -303,11 +303,6 @@ class ConsumerCoordinator(BaseCoordinator): time.sleep(self.config['retry_backoff_ms'] / 1000.0) - def ensure_partition_assignment(self): - """Ensure that we have a valid partition assignment from the coordinator.""" - if self._subscription.partitions_auto_assigned(): - self.ensure_active_group() - def close(self): try: self._maybe_auto_commit_offsets_sync() -- cgit v1.2.1 From df75751238f2ccc731d9881c92dfcc524c57aeaf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:12:22 -0800 Subject: Add deprecated methods to KafkaConsumer w/ notes on alternatives --- kafka/consumer/group.py | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 0293b4c..bd9d03d 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -651,3 +651,34 @@ class KafkaConsumer(six.Iterator): except StopIteration: self._iterator = None raise + + # old KafkaConsumer methods are deprecated + def configure(self, **configs): + """DEPRECATED -- initialize a new consumer""" + raise NotImplementedError( + 'deprecated -- initialize a new consumer') + + def set_topic_partitions(self, *topics): + """DEPRECATED -- use subscribe() or assign()""" + raise NotImplementedError( + 'deprecated -- use subscribe() or assign()') + + def fetch_messages(self): + """DEPRECATED -- use poll() or iterator interface""" + raise NotImplementedError( + 'deprecated -- use poll() or iterator interface') + + def get_partition_offsets(self, topic, partition, + request_time_ms, max_num_offsets): + """DEPRECATED -- send OffsetRequest with KafkaClient""" + raise NotImplementedError( + 'deprecated -- send an OffsetRequest with KafkaClient') + + def offsets(self, group=None): + """DEPRECATED -- use committed(partition)""" + raise NotImplementedError('deprecated -- use committed(partition)') + + def task_done(self, message): + """DEPRECATED -- commit manually if needed""" + raise NotImplementedError( + 'deprecated -- commit offsets manually if needed') -- cgit v1.2.1 From 0334fee5509b8cb3fd6ca3c1a6025804be978107 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:14:37 -0800 Subject: Switch to new KafkaConsumer in module imports --- kafka/consumer/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/__init__.py b/kafka/consumer/__init__.py index 935f56e..8041537 100644 --- a/kafka/consumer/__init__.py +++ b/kafka/consumer/__init__.py @@ -1,6 +1,6 @@ from .simple import SimpleConsumer from .multiprocess import MultiProcessConsumer -from .kafka import KafkaConsumer +from .group import KafkaConsumer __all__ = [ 'SimpleConsumer', 'MultiProcessConsumer', 'KafkaConsumer' -- cgit v1.2.1 From 9acb68901529a0158e37753c931ff00ccfaaaa7a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:14:52 -0800 Subject: bootstrap_servers no longer required in KafkaConsumer (localhost default) --- test/test_consumer.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/test/test_consumer.py b/test/test_consumer.py index ffce578..2c9561b 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -15,10 +15,6 @@ class TestKafkaConsumer(unittest.TestCase): with self.assertRaises(AssertionError): SimpleConsumer(MagicMock(), 'group', 'topic', partitions = [ '0' ]) - def test_broker_list_required(self): - with self.assertRaises(KafkaConfigurationError): - KafkaConsumer() - class TestMultiProcessConsumer(unittest.TestCase): def test_partition_list(self): -- cgit v1.2.1 From 50f6a25ceb0de0c1565092c40920429b9d42305e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:18:08 -0800 Subject: Update consumer integration tests to use new (group) KafkaConsumer - Remove debug call to deprecated .offsets() method - Manually assign TopicPartition to avoid group subscription overhead - Use next(consumer), not consumer.next() - consumer_timeout_ms now raises StopIteration, not ConsumerTimeout - auto_commit_enable is now enable_auto_commit - auto_offset_reset -> earliest, not smallest - new consumer does not support auto_commit_interval_messages --- test/test_consumer_integration.py | 60 ++++++++++++++++++++------------------- 1 file changed, 31 insertions(+), 29 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index cd5af5e..1104916 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -7,8 +7,8 @@ from kafka import ( KafkaConsumer, MultiProcessConsumer, SimpleConsumer, create_message ) from kafka.common import ( - ProduceRequestPayload, ConsumerFetchSizeTooSmall, ConsumerTimeout, - OffsetOutOfRangeError + ProduceRequestPayload, ConsumerFetchSizeTooSmall, + OffsetOutOfRangeError, TopicPartition ) from kafka.consumer.base import MAX_FETCH_BUFFER_SIZE_BYTES @@ -475,11 +475,10 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(1, range(100, 200)) # Start a consumer - consumer = self.kafka_consumer(auto_offset_reset='smallest', + consumer = self.kafka_consumer(auto_offset_reset='earliest', consumer_timeout_ms=5000) n = 0 messages = {0: set(), 1: set()} - logging.debug("kafka consumer offsets: %s" % consumer.offsets()) for m in consumer: logging.debug("Consumed message %s" % repr(m)) n += 1 @@ -493,13 +492,17 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): @kafka_versions("all") def test_kafka_consumer__blocking(self): TIMEOUT_MS = 500 - consumer = self.kafka_consumer(auto_offset_reset='smallest', + consumer = self.kafka_consumer(auto_offset_reset='earliest', consumer_timeout_ms=TIMEOUT_MS) + # Manual assignment avoids overhead of consumer group mgmt + consumer.unsubscribe() + consumer.assign([TopicPartition(self.topic, 0)]) + # Ask for 5 messages, nothing in queue, block 500ms with Timer() as t: - with self.assertRaises(ConsumerTimeout): - msg = consumer.next() + with self.assertRaises(StopIteration): + msg = next(consumer) self.assertGreaterEqual(t.interval, TIMEOUT_MS / 1000.0 ) self.send_messages(0, range(0, 10)) @@ -508,7 +511,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): messages = set() with Timer() as t: for i in range(5): - msg = consumer.next() + msg = next(consumer) messages.add((msg.partition, msg.offset)) self.assertEqual(len(messages), 5) self.assertLess(t.interval, TIMEOUT_MS / 1000.0 ) @@ -516,9 +519,9 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # Ask for 10 messages, get 5 back, block 500ms messages = set() with Timer() as t: - with self.assertRaises(ConsumerTimeout): + with self.assertRaises(StopIteration): for i in range(10): - msg = consumer.next() + msg = next(consumer) messages.add((msg.partition, msg.offset)) self.assertEqual(len(messages), 5) self.assertGreaterEqual(t.interval, TIMEOUT_MS / 1000.0 ) @@ -532,36 +535,35 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): # Start a consumer consumer1 = self.kafka_consumer( - group_id = GROUP_ID, - auto_commit_enable = True, - auto_commit_interval_ms = None, - auto_commit_interval_messages = 20, - auto_offset_reset='smallest', + group_id=GROUP_ID, + enable_auto_commit=True, + auto_commit_interval_ms=100, + auto_offset_reset='earliest', + consumer_timeout_ms=100 ) - # Grab the first 195 messages + # Grab the first 180 messages output_msgs1 = [] - for _ in xrange(195): - m = consumer1.next() + for _ in xrange(180): + m = next(consumer1) output_msgs1.append(m) - consumer1.task_done(m) - self.assert_message_count(output_msgs1, 195) + self.assert_message_count(output_msgs1, 180) + consumer1.close() # The total offset across both partitions should be at 180 consumer2 = self.kafka_consumer( - group_id = GROUP_ID, - auto_commit_enable = True, - auto_commit_interval_ms = None, - auto_commit_interval_messages = 20, - consumer_timeout_ms = 100, - auto_offset_reset='smallest', + group_id=GROUP_ID, + enable_auto_commit=True, + auto_commit_interval_ms=100, + auto_offset_reset='earliest', + consumer_timeout_ms=100 ) # 181-200 output_msgs2 = [] - with self.assertRaises(ConsumerTimeout): + with self.assertRaises(StopIteration): while True: - m = consumer2.next() + m = next(consumer2) output_msgs2.append(m) self.assert_message_count(output_msgs2, 20) - self.assertEqual(len(set(output_msgs1) & set(output_msgs2)), 15) + #self.assertEqual(len(set(output_msgs1) & set(output_msgs2)), 15) -- cgit v1.2.1 From fa7ecdaf3fd1ee9d54f8c39e98c99e439353d426 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:25:12 -0800 Subject: Use 2-second KafkaClient timeout in failover tests --- test/test_failover_integration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 339a08b..5ffaa04 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -34,7 +34,7 @@ class TestFailover(KafkaIntegrationTestCase): self.brokers = [KafkaFixture.instance(i, *kk_args) for i in range(replicas)] hosts = ['%s:%d' % (b.host, b.port) for b in self.brokers] - self.client = KafkaClient(hosts) + self.client = KafkaClient(hosts, timeout=2) super(TestFailover, self).setUp() def tearDown(self): -- cgit v1.2.1 From 6d48a1cc5fce549757fe306fad54a0f3a4f4444b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 16:28:49 -0800 Subject: Use parameterized pytests in test_client_async; add pytest-mocker plugin --- test/test_client_async.py | 164 ++++++++++++++++++++++++++-------------------- tox.ini | 1 + 2 files changed, 95 insertions(+), 70 deletions(-) diff --git a/test/test_client_async.py b/test/test_client_async.py index 5f0ccb0..aa8ff11 100644 --- a/test/test_client_async.py +++ b/test/test_client_async.py @@ -1,6 +1,5 @@ -from mock import patch -from . import unittest +import pytest from kafka.client_async import KafkaClient from kafka.common import BrokerMetadata @@ -9,95 +8,120 @@ from kafka.future import Future from kafka.protocol.metadata import MetadataResponse, MetadataRequest -class TestAsyncKafkaClient(unittest.TestCase): +@pytest.mark.parametrize("bootstrap,expected_hosts", [ + (None, [('localhost', 9092)]), + ('foobar:1234', [('foobar', 1234)]), + ('fizzbuzz', [('fizzbuzz', 9092)]), + ('foo:12,bar:34', [('foo', 12), ('bar', 34)]), + (['fizz:56', 'buzz'], [('fizz', 56), ('buzz', 9092)]), +]) +def test_bootstrap_servers(mocker, bootstrap, expected_hosts): + mocker.patch.object(KafkaClient, '_bootstrap') + if bootstrap is None: + KafkaClient() + else: + KafkaClient(bootstrap_servers=bootstrap) - def test_init(self): - with patch.object(KafkaClient, '_bootstrap') as bootstrap: + # host order is randomized internally, so resort before testing + (hosts,), _ = KafkaClient._bootstrap.call_args # pylint: disable=no-member + assert sorted(hosts) == sorted(expected_hosts) - KafkaClient() - bootstrap.assert_called_with([('localhost', 9092)]) - other_test_cases = [ - ('foobar:1234', [('foobar', 1234)]), - ('fizzbuzz', [('fizzbuzz', 9092)]), - ('foo:12,bar:34', [('foo', 12), ('bar', 34)]), - (['fizz:56', 'buzz'], [('fizz', 56), ('buzz', 9092)]) - ] - for arg, test in other_test_cases: - KafkaClient(bootstrap_servers=arg) - # host order is randomized internally, so resort before testing - (hosts,), _ = bootstrap.call_args - assert sorted(hosts) == sorted(test) +@pytest.fixture +def conn(mocker): + conn = mocker.patch('kafka.client_async.BrokerConnection') + conn.return_value = conn + conn.state = ConnectionStates.CONNECTED + conn.send.return_value = Future().success( + MetadataResponse( + [(0, 'foo', 12), (1, 'bar', 34)], # brokers + [])) # topics + return conn - @patch('kafka.client_async.BrokerConnection') - def test_bootstrap(self, conn): - conn.return_value = conn - conn.state = ConnectionStates.CONNECTED - conn.send.return_value = Future().success(MetadataResponse( - [(0, 'foo', 12), (1, 'bar', 34)], [])) - cli = KafkaClient() - conn.assert_called_once_with('localhost', 9092, **cli.config) - conn.connect.assert_called_with() - conn.send.assert_called_once_with(MetadataRequest([])) - assert cli._bootstrap_fails == 0 - assert cli.cluster.brokers() == set([BrokerMetadata(0, 'foo', 12), - BrokerMetadata(1, 'bar', 34)]) +def test_bootstrap_success(conn): + conn.state = ConnectionStates.CONNECTED + cli = KafkaClient() + conn.assert_called_once_with('localhost', 9092, **cli.config) + conn.connect.assert_called_with() + conn.send.assert_called_once_with(MetadataRequest([])) + assert cli._bootstrap_fails == 0 + assert cli.cluster.brokers() == set([BrokerMetadata(0, 'foo', 12), + BrokerMetadata(1, 'bar', 34)]) - conn.state = ConnectionStates.DISCONNECTED - cli = KafkaClient() - conn.connect.assert_called_with() - conn.close.assert_called_with() - assert cli._bootstrap_fails == 1 +def test_bootstrap_failure(conn): + conn.state = ConnectionStates.DISCONNECTED + cli = KafkaClient() + conn.assert_called_once_with('localhost', 9092, **cli.config) + conn.connect.assert_called_with() + conn.close.assert_called_with() + assert cli._bootstrap_fails == 1 + assert cli.cluster.brokers() == set() - def test_can_connect(self): - pass - def test_initiate_connect(self): - pass +def test_can_connect(): + pass - def test_finish_connect(self): - pass - def test_ready(self): - pass +def test_initiate_connect(): + pass - def test_close(self): - pass - def test_is_disconnected(self): - pass +def test_finish_connect(): + pass - def test_is_ready(self): - pass - def test_can_send_request(self): - pass +def test_ready(): + pass - def test_send(self): - pass - def test_poll(self): - pass +def test_close(): + pass - def test__poll(self): - pass - def test_in_flight_request_count(self): - pass +def test_is_disconnected(): + pass - def test_least_loaded_node(self): - pass - def test_set_topics(self): - pass +def test_is_ready(): + pass - def test_maybe_refresh_metadata(self): - pass - def test_schedule(self): - pass +def test_can_send_request(): + pass - def test_unschedule(self): - pass +def test_send(): + pass + + +def test_poll(): + pass + + +def test__poll(): + pass + + +def test_in_flight_request_count(): + pass + + +def test_least_loaded_node(): + pass + + +def test_set_topics(): + pass + + +def test_maybe_refresh_metadata(): + pass + + +def test_schedule(): + pass + + +def test_unschedule(): + pass diff --git a/tox.ini b/tox.ini index b00d531..8860933 100644 --- a/tox.ini +++ b/tox.ini @@ -8,6 +8,7 @@ deps = pytest-catchlog pytest-pylint pytest-sugar + pytest-mock mock python-snappy py{26,27}: six -- cgit v1.2.1 From d73f26d57fe307897b1a6d000ef971764b7f79ee Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 17:11:46 -0800 Subject: Add __hash__ method to Struct --- kafka/protocol/struct.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index 0a96c02..ca1013e 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -44,6 +44,9 @@ class Struct(AbstractType): key_vals.append('%s=%s' % (name, field.repr(self.__dict__[name]))) return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' + def __hash__(self): + return hash(self.encode()) + def __eq__(self, other): if self.SCHEMA != other.SCHEMA: return False -- cgit v1.2.1 From 8f15262c794b16ad330194f062091d0b42ac5a41 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 17:12:15 -0800 Subject: Assert Message value and (optional) key are bytes --- kafka/protocol/message.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index f893912..430ecad 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -22,6 +22,8 @@ class Message(Struct): CODEC_SNAPPY = 0x02 def __init__(self, value, key=None, magic=0, attributes=0, crc=0): + assert isinstance(value, bytes), 'value must be bytes' + assert key is None or isinstance(key, bytes), 'key must be bytes' self.crc = crc self.magic = magic self.attributes = attributes -- cgit v1.2.1 From 520d3c1484e4e856b866715739fb99a381f3f2bb Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 17:21:58 -0800 Subject: Add pytests for KafkaConsumer group assignments --- test/test_consumer_group.py | 170 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 170 insertions(+) create mode 100644 test/test_consumer_group.py diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py new file mode 100644 index 0000000..0fd65b8 --- /dev/null +++ b/test/test_consumer_group.py @@ -0,0 +1,170 @@ +import collections +import logging +import threading +import os +import time + +import pytest +import six + +from kafka import KafkaClient, SimpleProducer +from kafka.common import TopicPartition +from kafka.conn import BrokerConnection, ConnectionStates +from kafka.consumer.group import KafkaConsumer + +from test.fixtures import KafkaFixture, ZookeeperFixture +from test.testutil import random_string + + +@pytest.fixture(scope="module") +def version(): + if 'KAFKA_VERSION' not in os.environ: + return () + return tuple(map(int, os.environ['KAFKA_VERSION'].split('.'))) + + +@pytest.fixture(scope="module") +def zookeeper(version, request): + assert version + zk = ZookeeperFixture.instance() + def fin(): + zk.close() + request.addfinalizer(fin) + return zk + + +@pytest.fixture(scope="module") +def kafka_broker(version, zookeeper, request): + assert version + k = KafkaFixture.instance(0, zookeeper.host, zookeeper.port, + partitions=4) + def fin(): + k.close() + request.addfinalizer(fin) + return k + + +@pytest.fixture +def simple_client(kafka_broker): + connect_str = 'localhost:' + str(kafka_broker.port) + return KafkaClient(connect_str) + + +@pytest.fixture +def topic(simple_client): + topic = random_string(5) + simple_client.ensure_topic_exists(topic) + return topic + + +@pytest.fixture +def topic_with_messages(simple_client, topic): + producer = SimpleProducer(simple_client) + for i in six.moves.xrange(100): + producer.send_messages(topic, 'msg_%d' % i) + return topic + + +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_consumer(kafka_broker, version): + + # 0.8.2 brokers need a topic to function well + if version >= (0, 8, 2) and version < (0, 9): + topic(simple_client(kafka_broker)) + + connect_str = 'localhost:' + str(kafka_broker.port) + consumer = KafkaConsumer(bootstrap_servers=connect_str) + consumer.poll(500) + assert len(consumer._client._conns) > 0 + node_id = list(consumer._client._conns.keys())[0] + assert consumer._client._conns[node_id].state is ConnectionStates.CONNECTED + + +@pytest.mark.skipif(version() < (0, 9), reason='Unsupported Kafka Version') +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_group(kafka_broker, topic): + num_partitions = 4 + connect_str = 'localhost:' + str(kafka_broker.port) + consumers = {} + stop = {} + messages = collections.defaultdict(list) + def consumer_thread(i): + assert i not in consumers + assert i not in stop + stop[i] = threading.Event() + consumers[i] = KafkaConsumer(topic, + bootstrap_servers=connect_str, + request_timeout_ms=1000) + while not stop[i].is_set(): + for tp, records in six.itervalues(consumers[i].poll()): + messages[i][tp].extend(records) + consumers[i].close() + del consumers[i] + del stop[i] + + num_consumers = 4 + for i in range(num_consumers): + threading.Thread(target=consumer_thread, args=(i,)).start() + + try: + timeout = time.time() + 35 + while True: + for c in range(num_consumers): + if c not in consumers: + break + elif not consumers[c].assignment(): + break + else: + for c in range(num_consumers): + logging.info("%s: %s", c, consumers[c].assignment()) + break + assert time.time() < timeout, "timeout waiting for assignments" + + group_assignment = set() + for c in range(num_consumers): + assert len(consumers[c].assignment()) != 0 + assert set.isdisjoint(consumers[c].assignment(), group_assignment) + group_assignment.update(consumers[c].assignment()) + + assert group_assignment == set([ + TopicPartition(topic, partition) + for partition in range(num_partitions)]) + + finally: + for c in range(num_consumers): + stop[c].set() + + +@pytest.mark.skipif(not version(), reason="No KAFKA_VERSION set") +def test_correlation_id_rollover(kafka_broker): + logging.getLogger('kafka.conn').setLevel(logging.ERROR) + from kafka.protocol.metadata import MetadataRequest + conn = BrokerConnection('localhost', kafka_broker.port, + receive_buffer_bytes=131072, + max_in_flight_requests_per_connection=100) + req = MetadataRequest([]) + while not conn.connected(): + conn.connect() + futures = collections.deque() + start = time.time() + done = 0 + for i in six.moves.xrange(2**13): + if not conn.can_send_more(): + conn.recv(timeout=None) + futures.append(conn.send(req)) + conn.recv() + while futures and futures[0].is_done: + f = futures.popleft() + if not f.succeeded(): + raise f.exception + done += 1 + if time.time() > start + 10: + print ("%d done" % done) + start = time.time() + + while futures: + conn.recv() + if futures[0].is_done: + f = futures.popleft() + if not f.succeeded(): + raise f.exception -- cgit v1.2.1 From ca88921ee77813c83fbef003f6b9d935b9add329 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 17:29:23 -0800 Subject: Fix TRAVIS_PYTHON_VERSION magic in .travis.yml --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 64d019c..10842ae 100644 --- a/.travis.yml +++ b/.travis.yml @@ -48,7 +48,7 @@ deploy: # branch: master script: - - tox -e ${TRAVIS_PYTHON_VERSION/./} + - tox -e py${TRAVIS_PYTHON_VERSION/./} after_success: - coveralls -- cgit v1.2.1 From 95e7a65a8eb64bcb589ced003884424f94b58e8f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:00:03 -0800 Subject: Override Message __hash__ to use _encode_self and not recalc crcs --- kafka/protocol/message.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 430ecad..70da5ac 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -68,6 +68,9 @@ class Message(Struct): return MessageSet.decode(raw_bytes, bytes_to_read=len(raw_bytes)) + def __hash__(self): + return hash(self._encode_self(recalc_crc=False)) + class PartialMessage(bytes): def __repr__(self): -- cgit v1.2.1 From ece72e4745265ab71f0b6da6710accc2d8980055 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:00:45 -0800 Subject: Move pytest options to [pytest] tox section --- tox.ini | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tox.ini b/tox.ini index 8860933..d1168f1 100644 --- a/tox.ini +++ b/tox.ini @@ -1,6 +1,11 @@ [tox] envlist = py{26,27,py,33,34,35}, docs +[pytest] +testpaths = kafka test +doctest_optionflags = modules +addopts = --durations=10 + [testenv] deps = pytest @@ -14,7 +19,7 @@ deps = py{26,27}: six py26: unittest2 commands = - py.test {posargs:--pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --durations=10 --cov=kafka --doctest-modules kafka test} + py.test {posargs:--pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka} setenv = PROJECT_ROOT = {toxinidir} passenv = KAFKA_VERSION -- cgit v1.2.1 From 8d85ca2d44de39a2d591da8a675c1ab5cc2c1c67 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:01:30 -0800 Subject: Dont run travis tests against 0.8.1 (keep 0.8.1.1); remove UNIT_AND_LINT_ONLY --- .travis.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 10842ae..8d0ee02 100644 --- a/.travis.yml +++ b/.travis.yml @@ -9,9 +9,7 @@ python: - pypy env: - - UNIT_AND_LINT_ONLY=true - KAFKA_VERSION=0.8.0 - - KAFKA_VERSION=0.8.1 - KAFKA_VERSION=0.8.1.1 - KAFKA_VERSION=0.8.2.2 - KAFKA_VERSION=0.9.0.0 -- cgit v1.2.1 From b13c92306cf0b115c0aae8d1c50007bbcb1d1107 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:09:13 -0800 Subject: Dont run pylint on python2.6 (pylint no longer supports) --- tox.ini | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tox.ini b/tox.ini index d1168f1..a67af0d 100644 --- a/tox.ini +++ b/tox.ini @@ -1,5 +1,5 @@ [tox] -envlist = py{26,27,py,33,34,35}, docs +envlist = py{26,27,33,34,35,py}, docs [pytest] testpaths = kafka test @@ -11,7 +11,7 @@ deps = pytest pytest-cov pytest-catchlog - pytest-pylint + py{27,33,34,35,py}: pytest-pylint pytest-sugar pytest-mock mock @@ -24,6 +24,10 @@ setenv = PROJECT_ROOT = {toxinidir} passenv = KAFKA_VERSION +[testenv:py26] +# pylint doesn't support python2.6 +commands = py.test {posargs:--cov=kafka} + [testenv:docs] deps = sphinxcontrib-napoleon -- cgit v1.2.1 From 1bcb9f029d7179a23d2e008891cfb9e7f0534d64 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:17:23 -0800 Subject: Message value can be None --- kafka/protocol/message.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 70da5ac..2648e24 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -22,7 +22,7 @@ class Message(Struct): CODEC_SNAPPY = 0x02 def __init__(self, value, key=None, magic=0, attributes=0, crc=0): - assert isinstance(value, bytes), 'value must be bytes' + assert value is None or isinstance(value, bytes), 'value must be bytes' assert key is None or isinstance(key, bytes), 'key must be bytes' self.crc = crc self.magic = magic -- cgit v1.2.1 From 42ea4f49132ded944e10cbafbd90a754def41836 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:45:32 -0800 Subject: Catch py3 ConnectionErrors --- kafka/client_async.py | 12 +++++++++--- kafka/conn.py | 25 ++++++++++++++++++------- 2 files changed, 27 insertions(+), 10 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 8a92159..914afec 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -17,6 +17,10 @@ from .protocol.metadata import MetadataRequest from .protocol.produce import ProduceRequest from .version import __version__ +if six.PY2: + ConnectionError = None + + log = logging.getLogger(__name__) @@ -503,7 +507,6 @@ class KafkaClient(object): ('0.8.0', MetadataRequest([])), ] - for version, request in test_cases: connect() f = self.send(node_id, request) @@ -517,8 +520,11 @@ class KafkaClient(object): log.info('Broker version identifed as %s', version) return version - assert isinstance(f.exception.message, socket.error) - assert f.exception.message.errno in (32, 54) + if six.PY2: + assert isinstance(f.exception.args[0], socket.error) + assert f.exception.args[0].errno in (32, 54) + else: + assert isinstance(f.exception.args[0], ConnectionError) log.info("Broker is not v%s -- it did not recognize %s", version, request.__class__.__name__) continue diff --git a/kafka/conn.py b/kafka/conn.py index d713b56..9e8a16f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -13,7 +13,6 @@ import time import six import kafka.common as Errors -from kafka.common import ConnectionError from kafka.future import Future from kafka.protocol.api import RequestHeader from kafka.protocol.commit import GroupCoordinatorResponse @@ -21,6 +20,10 @@ from kafka.protocol.types import Int32 from kafka.version import __version__ +if six.PY2: + ConnectionError = socket.error + BlockingIOError = Exception + log = logging.getLogger(__name__) DEFAULT_SOCKET_TIMEOUT_SECONDS = 120 @@ -166,7 +169,7 @@ class BrokerConnection(object): sent_bytes = self._sock.send(message) assert sent_bytes == len(message) self._sock.setblocking(False) - except (AssertionError, socket.error) as e: + except (AssertionError, ConnectionError) as e: log.exception("Error sending %s to %s", request, self) error = Errors.ConnectionError(e) self.close(error=error) @@ -225,8 +228,8 @@ class BrokerConnection(object): # An extremely small, but non-zero, probability that there are # more than 0 but not yet 4 bytes available to read self._rbuffer.write(self._sock.recv(4 - self._rbuffer.tell())) - except socket.error as e: - if e.errno == errno.EWOULDBLOCK: + except ConnectionError as e: + if six.PY2 and e.errno == errno.EWOULDBLOCK: # This shouldn't happen after selecting above # but just in case return None @@ -234,6 +237,10 @@ class BrokerConnection(object): ' closing socket', self) self.close(error=Errors.ConnectionError(e)) return None + except BlockingIOError: + if six.PY3: + return None + raise if self._rbuffer.tell() == 4: self._rbuffer.seek(0) @@ -249,14 +256,18 @@ class BrokerConnection(object): staged_bytes = self._rbuffer.tell() try: self._rbuffer.write(self._sock.recv(self._next_payload_bytes - staged_bytes)) - except socket.error as e: + except ConnectionError as e: # Extremely small chance that we have exactly 4 bytes for a # header, but nothing to read in the body yet - if e.errno == errno.EWOULDBLOCK: + if six.PY2 and e.errno == errno.EWOULDBLOCK: return None log.exception('%s: Error in recv', self) self.close(error=Errors.ConnectionError(e)) return None + except BlockingIOError: + if six.PY3: + return None + raise staged_bytes = self._rbuffer.tell() if staged_bytes > self._next_payload_bytes: @@ -379,7 +390,7 @@ class KafkaConnection(local): self.close() # And then raise - raise ConnectionError("Kafka @ {0}:{1} went away".format(self.host, self.port)) + raise Errors.ConnectionError("Kafka @ {0}:{1} went away".format(self.host, self.port)) def _read_bytes(self, num_bytes): bytes_left = num_bytes -- cgit v1.2.1 From ed8f2212bf879831d38300212aa2251e4c4f0d8c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 18:49:16 -0800 Subject: Attempt to fix travis shell magic for pypy --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 8d0ee02..2eb91b7 100644 --- a/.travis.yml +++ b/.travis.yml @@ -46,7 +46,7 @@ deploy: # branch: master script: - - tox -e py${TRAVIS_PYTHON_VERSION/./} + - tox -e `if [ "$TRAVIS_PYTHON_VERSION" == "pypy" ]; then echo pypy; else echo py${TRAVIS_PYTHON_VERSION/./}; fi` after_success: - coveralls -- cgit v1.2.1 From 03d37dff5c707599e2f268dccb4cccafbeadd5e3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 19:50:21 -0800 Subject: Import queue from six.moves --- kafka/consumer/multiprocess.py | 6 ++---- kafka/consumer/simple.py | 5 +---- test/test_producer.py | 13 +++---------- 3 files changed, 6 insertions(+), 18 deletions(-) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index d0e2920..a348d1a 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -3,12 +3,10 @@ from __future__ import absolute_import from collections import namedtuple import logging from multiprocessing import Process, Manager as MPManager -try: - import queue # python 3 -except ImportError: - import Queue as queue # python 2 import time +from six.moves import queue + from ..common import KafkaError from .base import ( Consumer, diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index 946e9c7..abeac7b 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -5,14 +5,11 @@ try: except ImportError: from itertools import izip_longest as izip_longest, repeat # pylint: disable=E0611 import logging -try: - import queue # python 3 -except ImportError: - import Queue as queue # python 2 import sys import time import six +from six.moves import queue from .base import ( Consumer, diff --git a/test/test_producer.py b/test/test_producer.py index f62b97a..227d4ad 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -2,6 +2,7 @@ import collections import logging +import threading import time from mock import MagicMock, patch @@ -15,15 +16,7 @@ from kafka.common import ( from kafka.producer.base import Producer, _send_upstream from kafka.protocol import CODEC_NONE -import threading -try: - from queue import Empty, Queue -except ImportError: - from Queue import Empty, Queue -try: - xrange -except NameError: - xrange = range +from six.moves import queue, xrange class TestKafkaProducer(unittest.TestCase): @@ -130,7 +123,7 @@ class TestKafkaProducerSendUpstream(unittest.TestCase): def setUp(self): self.client = MagicMock() - self.queue = Queue() + self.queue = queue.Queue() def _run_process(self, retries_limit=3, sleep_timeout=1): # run _send_upstream process with the queue -- cgit v1.2.1 From 09358f9c825a5d019a25288e0a4ea8b2dd0141b3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 22:43:39 -0800 Subject: Fast heartbeats during consumer group tests --- test/test_consumer_group.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index 0fd65b8..795e127 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -94,6 +94,7 @@ def test_group(kafka_broker, topic): stop[i] = threading.Event() consumers[i] = KafkaConsumer(topic, bootstrap_servers=connect_str, + heartbeat_interval_ms=500, request_timeout_ms=1000) while not stop[i].is_set(): for tp, records in six.itervalues(consumers[i].poll()): -- cgit v1.2.1 From c8226d030a15e34538934bdaf5add090db118732 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 22:46:18 -0800 Subject: Dont use consumer_timeout_ms in kafka blocking test --- test/test_consumer_integration.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 1104916..8b5dbec 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -539,7 +539,6 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): enable_auto_commit=True, auto_commit_interval_ms=100, auto_offset_reset='earliest', - consumer_timeout_ms=100 ) # Grab the first 180 messages @@ -556,14 +555,12 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): enable_auto_commit=True, auto_commit_interval_ms=100, auto_offset_reset='earliest', - consumer_timeout_ms=100 ) # 181-200 output_msgs2 = [] - with self.assertRaises(StopIteration): - while True: - m = next(consumer2) - output_msgs2.append(m) + for _ in xrange(20): + m = next(consumer2) + output_msgs2.append(m) self.assert_message_count(output_msgs2, 20) - #self.assertEqual(len(set(output_msgs1) & set(output_msgs2)), 15) + self.assertEqual(len(set(output_msgs1) | set(output_msgs2)), 200) -- cgit v1.2.1 From 54d758e8132e60e3631aeefb5da3e9692dd7a671 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 22:47:15 -0800 Subject: Dont pylint in pypy tests - it seems to take forever and doesn't add anything --- tox.ini | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tox.ini b/tox.ini index a67af0d..0544cf8 100644 --- a/tox.ini +++ b/tox.ini @@ -28,6 +28,10 @@ passenv = KAFKA_VERSION # pylint doesn't support python2.6 commands = py.test {posargs:--cov=kafka} +[testenv:pypy] +# pylint is super slow on pypy... +commands = py.test {posargs:--cov=kafka} + [testenv:docs] deps = sphinxcontrib-napoleon -- cgit v1.2.1 From bb433e27f020597f3807cb4058d1e45a671cea6e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 3 Jan 2016 23:31:09 -0800 Subject: Call errbacks with future.exception --- kafka/future.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/future.py b/kafka/future.py index 958e85f..06b8c3a 100644 --- a/kafka/future.py +++ b/kafka/future.py @@ -45,7 +45,7 @@ class Future(object): self.is_done = True for f in self._errbacks: try: - f(e) + f(self.exception) except Exception: log.exception('Error processing errback') return self -- cgit v1.2.1 From 16c834119b6113450a59a05ce627669ba5752f37 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 4 Jan 2016 00:18:58 -0800 Subject: Fix future redefine bug in client.poll --- kafka/client_async.py | 6 +++--- kafka/coordinator/base.py | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 914afec..747a85f 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -292,14 +292,14 @@ class KafkaClient(object): metadata_timeout = self._maybe_refresh_metadata() # Send scheduled tasks - for task, future in self._delayed_tasks.pop_ready(): + for task, task_future in self._delayed_tasks.pop_ready(): try: result = task() except Exception as e: log.error("Task %s failed: %s", task, e) - future.failure(e) + task_future.failure(e) else: - future.success(result) + task_future.success(result) timeout = min(timeout_ms, metadata_timeout, self.config['request_timeout_ms']) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 4f16bb0..bcd5889 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -236,6 +236,7 @@ class BaseCoordinator(object): self.needs_join_prepare = True self.heartbeat_task.reset() else: + assert future.failed() exception = future.exception if isinstance(exception, (Errors.UnknownMemberIdError, Errors.RebalanceInProgressError, -- cgit v1.2.1 From 1a0b86625f265aa1e0edcf7f1909be3a572a2994 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 4 Jan 2016 00:19:38 -0800 Subject: Dont use consumer_timeout_ms in simple kafka_consumer test --- test/test_consumer_integration.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 8b5dbec..c1cb311 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -475,8 +475,7 @@ class TestConsumerIntegration(KafkaIntegrationTestCase): self.send_messages(1, range(100, 200)) # Start a consumer - consumer = self.kafka_consumer(auto_offset_reset='earliest', - consumer_timeout_ms=5000) + consumer = self.kafka_consumer(auto_offset_reset='earliest') n = 0 messages = {0: set(), 1: set()} for m in consumer: -- cgit v1.2.1 From 16e35c9d160dba02fc37323fa811607c5fbfe7b6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 4 Jan 2016 00:55:55 -0800 Subject: Drop request_timeout_ms override in consumer group test --- test/test_consumer_group.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index 795e127..4fd4cdf 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -94,8 +94,7 @@ def test_group(kafka_broker, topic): stop[i] = threading.Event() consumers[i] = KafkaConsumer(topic, bootstrap_servers=connect_str, - heartbeat_interval_ms=500, - request_timeout_ms=1000) + heartbeat_interval_ms=500) while not stop[i].is_set(): for tp, records in six.itervalues(consumers[i].poll()): messages[i][tp].extend(records) -- cgit v1.2.1 From 31c3d59ee3507a65533a3db58dd0fa59d9925b11 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 4 Jan 2016 11:33:08 -0800 Subject: Reduce partitions for offset commit topic in 0.9 broker configs --- servers/0.9.0.0/resources/kafka.properties | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/servers/0.9.0.0/resources/kafka.properties b/servers/0.9.0.0/resources/kafka.properties index 685aed1..d7b81c1 100644 --- a/servers/0.9.0.0/resources/kafka.properties +++ b/servers/0.9.0.0/resources/kafka.properties @@ -109,6 +109,15 @@ log.retention.check.interval.ms=60000 # If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. log.cleaner.enable=false +# tune down offset topics to reduce setup time in tests +offsets.commit.timeout.ms=500 +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=2 + +# Allow shorter session timeouts for tests +group.min.session.timeout.ms=1000 + + ############################# Zookeeper ############################# # Zookeeper connection string (see zookeeper docs for details). -- cgit v1.2.1 From c8deb0c276d57209006eebdd910017846860a38d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 7 Jan 2016 16:58:11 -0800 Subject: Reorg kafka imports - kafka.KafkaClient is new async client - kafka.SimpleClient is old sync client - update copyright / author info - add BrokerConnection; drop KafkaConnection --- kafka/__init__.py | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/kafka/__init__.py b/kafka/__init__.py index 2fc59c6..2a99847 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -1,21 +1,22 @@ __title__ = 'kafka' from .version import __version__ -__author__ = 'David Arthur' +__author__ = 'Dana Powers' __license__ = 'Apache License 2.0' -__copyright__ = 'Copyright 2015, David Arthur under Apache License, v2.0' +__copyright__ = 'Copyright 2016 Dana Powers, David Arthur, and Contributors' -from kafka.client import KafkaClient -from kafka.conn import KafkaConnection +from kafka.client import KafkaClient as SimpleClient +from kafka.client_async import KafkaClient +from kafka.conn import BrokerConnection from kafka.protocol import ( - create_message, create_gzip_message, create_snappy_message -) + create_message, create_gzip_message, create_snappy_message) from kafka.producer import SimpleProducer, KeyedProducer from kafka.partitioner import RoundRobinPartitioner, HashedPartitioner, Murmur2Partitioner -from kafka.consumer import SimpleConsumer, MultiProcessConsumer, KafkaConsumer +from kafka.consumer import KafkaConsumer, SimpleConsumer, MultiProcessConsumer __all__ = [ - 'KafkaClient', 'KafkaConnection', 'SimpleProducer', 'KeyedProducer', - 'RoundRobinPartitioner', 'HashedPartitioner', 'SimpleConsumer', - 'MultiProcessConsumer', 'create_message', 'create_gzip_message', - 'create_snappy_message', 'KafkaConsumer', + 'KafkaConsumer', 'KafkaClient', 'BrokerConnection', + 'SimpleClient', 'SimpleProducer', 'KeyedProducer', + 'RoundRobinPartitioner', 'HashedPartitioner', + 'create_message', 'create_gzip_message', 'create_snappy_message', + 'SimpleConsumer', 'MultiProcessConsumer', ] -- cgit v1.2.1 From e080c6b0cdb54563e3c5ad595d582de26561d9f0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 7 Jan 2016 17:03:08 -0800 Subject: Docstring updates --- kafka/client.py | 39 ++++---- kafka/codec.py | 32 +++--- kafka/conn.py | 11 ++- kafka/consumer/group.py | 256 ++++++++++++++++++++++++------------------------ kafka/producer/base.py | 53 ++++++---- 5 files changed, 209 insertions(+), 182 deletions(-) diff --git a/kafka/client.py b/kafka/client.py index 2f070cd..14e71bb 100644 --- a/kafka/client.py +++ b/kafka/client.py @@ -455,31 +455,28 @@ class KafkaClient(object): time.sleep(.5) def load_metadata_for_topics(self, *topics): - """ - Fetch broker and topic-partition metadata from the server, - and update internal data: - broker list, topic/partition list, and topic/parition -> broker map + """Fetch broker and topic-partition metadata from the server. + + Updates internal data: broker list, topic/partition list, and + topic/parition -> broker map. This method should be called after + receiving any error. - This method should be called after receiving any error + Note: Exceptions *will not* be raised in a full refresh (i.e. no topic + list). In this case, error codes will be logged as errors. + Partition-level errors will also not be raised here (a single partition + w/o a leader, for example). Arguments: *topics (optional): If a list of topics is provided, - the metadata refresh will be limited to the specified topics only. - - Exceptions: - ---------- - If the broker is configured to not auto-create topics, - expect UnknownTopicOrPartitionError for topics that don't exist - - If the broker is configured to auto-create topics, - expect LeaderNotAvailableError for new topics - until partitions have been initialized. - - Exceptions *will not* be raised in a full refresh (i.e. no topic list) - In this case, error codes will be logged as errors - - Partition-level errors will also not be raised here - (a single partition w/o a leader, for example) + the metadata refresh will be limited to the specified topics + only. + + Raises: + UnknownTopicOrPartitionError: Raised for topics that do not exist, + unless the broker is configured to auto-create topics. + LeaderNotAvailableError: Raised for topics that do not exist yet, + when the broker is configured to auto-create topics. Retry + after a short backoff (topics/partitions are initializing). """ if topics: self.reset_topic_metadata(*topics) diff --git a/kafka/codec.py b/kafka/codec.py index a9373c7..c27d89b 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -55,24 +55,30 @@ def gzip_decode(payload): return result -def snappy_encode(payload, xerial_compatible=False, xerial_blocksize=32 * 1024): - """Encodes the given data with snappy if xerial_compatible is set then the - stream is encoded in a fashion compatible with the xerial snappy library +def snappy_encode(payload, xerial_compatible=False, xerial_blocksize=32*1024): + """Encodes the given data with snappy compression. + + If xerial_compatible is set then the stream is encoded in a fashion + compatible with the xerial snappy library. + + The block size (xerial_blocksize) controls how frequent the blocking occurs + 32k is the default in the xerial library. + + The format winds up being: - The block size (xerial_blocksize) controls how frequent the blocking - occurs 32k is the default in the xerial library. - The format winds up being +-------------+------------+--------------+------------+--------------+ | Header | Block1 len | Block1 data | Blockn len | Blockn data | - |-------------+------------+--------------+------------+--------------| + +-------------+------------+--------------+------------+--------------+ | 16 bytes | BE int32 | snappy bytes | BE int32 | snappy bytes | +-------------+------------+--------------+------------+--------------+ - It is important to not that the blocksize is the amount of uncompressed - data presented to snappy at each block, whereas the blocklen is the - number of bytes that will be present in the stream, that is the - length will always be <= blocksize. + + It is important to note that the blocksize is the amount of uncompressed + data presented to snappy at each block, whereas the blocklen is the number + of bytes that will be present in the stream; so the length will always be + <= blocksize. + """ if not has_snappy(): @@ -109,9 +115,9 @@ def _detect_xerial_stream(payload): This mode writes a magic header of the format: +--------+--------------+------------+---------+--------+ | Marker | Magic String | Null / Pad | Version | Compat | - |--------+--------------+------------+---------+--------| + +--------+--------------+------------+---------+--------+ | byte | c-string | byte | int32 | int32 | - |--------+--------------+------------+---------+--------| + +--------+--------------+------------+---------+--------+ | -126 | 'SNAPPY' | \0 | | | +--------+--------------+------------+---------+--------+ diff --git a/kafka/conn.py b/kafka/conn.py index 9e8a16f..6ee5f5f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -126,9 +126,17 @@ class BrokerConnection(object): return False def connected(self): + """Return True iff socket is connected.""" return self.state is ConnectionStates.CONNECTED def close(self, error=None): + """Close socket and fail all in-flight-requests. + + Arguments: + error (Exception, optional): pending in-flight-requests + will be failed with this exception. + Default: kafka.common.ConnectionError. + """ if self._sock: self._sock.close() self._sock = None @@ -189,11 +197,12 @@ class BrokerConnection(object): return future def can_send_more(self): + """Return True unless there are max_in_flight_requests.""" max_ifrs = self.config['max_in_flight_requests_per_connection'] return len(self.in_flight_requests) < max_ifrs def recv(self, timeout=0): - """Non-blocking network receive + """Non-blocking network receive. Return response if available """ diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index bd9d03d..9ce1438 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -18,7 +18,114 @@ log = logging.getLogger(__name__) class KafkaConsumer(six.Iterator): - """Consumer for Kafka 0.9""" + """Consume records from a Kafka cluster. + + The consumer will transparently handle the failure of servers in the Kafka + cluster, and adapt as topic-partitions are created or migrate between + brokers. It also interacts with the assigned kafka Group Coordinator node + to allow multiple consumers to load balance consumption of topics (requires + kafka >= 0.9.0.0). + + Arguments: + *topics (str): optional list of topics to subscribe to. If not set, + call subscribe() or assign() before consuming records. + + Keyword Arguments: + bootstrap_servers: 'host[:port]' string (or list of 'host[:port]' + strings) that the consumer should contact to bootstrap initial + cluster metadata. This does not have to be the full node list. + It just needs to have at least one broker that will respond to a + Metadata API Request. Default port is 9092. If no servers are + specified, will default to localhost:9092. + client_id (str): a name for this client. This string is passed in + each request to servers and can be used to identify specific + server-side log entries that correspond to this client. Also + submitted to GroupCoordinator for logging with respect to + consumer group administration. Default: 'kafka-python-{version}' + group_id (str): name of the consumer group to join for dynamic + partition assignment (if enabled), and to use for fetching and + committing offsets. Default: 'kafka-python-default-group' + key_deserializer (callable): Any callable that takes a + raw message key and returns a deserialized key. + value_deserializer (callable, optional): Any callable that takes a + raw message value and returns a deserialized value. + fetch_min_bytes (int): Minimum amount of data the server should + return for a fetch request, otherwise wait up to + fetch_max_wait_ms for more data to accumulate. Default: 1024. + fetch_max_wait_ms (int): The maximum amount of time in milliseconds + the server will block before answering the fetch request if + there isn't sufficient data to immediately satisfy the + requirement given by fetch_min_bytes. Default: 500. + max_partition_fetch_bytes (int): The maximum amount of data + per-partition the server will return. The maximum total memory + used for a request = #partitions * max_partition_fetch_bytes. + This size must be at least as large as the maximum message size + the server allows or else it is possible for the producer to + send messages larger than the consumer can fetch. If that + happens, the consumer can get stuck trying to fetch a large + message on a certain partition. Default: 1048576. + request_timeout_ms (int): Client request timeout in milliseconds. + Default: 40000. + retry_backoff_ms (int): Milliseconds to backoff when retrying on + errors. Default: 100. + reconnect_backoff_ms (int): The amount of time in milliseconds to + wait before attempting to reconnect to a given host. + Default: 50. + max_in_flight_requests_per_connection (int): Requests are pipelined + to kafka brokers up to this number of maximum requests per + broker connection. Default: 5. + auto_offset_reset (str): A policy for resetting offsets on + OffsetOutOfRange errors: 'earliest' will move to the oldest + available message, 'latest' will move to the most recent. Any + ofther value will raise the exception. Default: 'latest'. + enable_auto_commit (bool): If true the consumer's offset will be + periodically committed in the background. Default: True. + auto_commit_interval_ms (int): milliseconds between automatic + offset commits, if enable_auto_commit is True. Default: 5000. + default_offset_commit_callback (callable): called as + callback(offsets, response) response will be either an Exception + or a OffsetCommitResponse struct. This callback can be used to + trigger custom actions when a commit request completes. + check_crcs (bool): Automatically check the CRC32 of the records + consumed. This ensures no on-the-wire or on-disk corruption to + the messages occurred. This check adds some overhead, so it may + be disabled in cases seeking extreme performance. Default: True + metadata_max_age_ms (int): The period of time in milliseconds after + which we force a refresh of metadata even if we haven't seen any + partition leadership changes to proactively discover any new + brokers or partitions. Default: 300000 + partition_assignment_strategy (list): List of objects to use to + distribute partition ownership amongst consumer instances when + group management is used. Default: [RoundRobinPartitionAssignor] + heartbeat_interval_ms (int): The expected time in milliseconds + between heartbeats to the consumer coordinator when using + Kafka's group management feature. Heartbeats are used to ensure + that the consumer's session stays active and to facilitate + rebalancing when new consumers join or leave the group. The + value must be set lower than session_timeout_ms, but typically + should be set no higher than 1/3 of that value. It can be + adjusted even lower to control the expected time for normal + rebalances. Default: 3000 + session_timeout_ms (int): The timeout used to detect failures when + using Kafka's group managementment facilities. Default: 30000 + send_buffer_bytes (int): The size of the TCP send buffer + (SO_SNDBUF) to use when sending data. Default: 131072 + receive_buffer_bytes (int): The size of the TCP receive buffer + (SO_RCVBUF) to use when reading data. Default: 32768 + consumer_timeout_ms (int): number of millisecond to throw a timeout + exception to the consumer if no message is available for + consumption. Default: -1 (dont throw exception) + api_version (str): specify which kafka API version to use. + 0.9 enables full group coordination features; 0.8.2 enables + kafka-storage offset commits; 0.8.1 enables zookeeper-storage + offset commits; 0.8.0 is what is left. If set to 'auto', will + attempt to infer the broker version by probing various APIs. + Default: auto + + Note: + Configuration parameters are described in more detail at + https://kafka.apache.org/090/configuration.html#newconsumerconfigs + """ DEFAULT_CONFIG = { 'bootstrap_servers': 'localhost', 'client_id': 'kafka-python-' + __version__, @@ -51,114 +158,6 @@ class KafkaConsumer(six.Iterator): } def __init__(self, *topics, **configs): - """A Kafka client that consumes records from a Kafka cluster. - - The consumer will transparently handle the failure of servers in the - Kafka cluster, and transparently adapt as partitions of data it fetches - migrate within the cluster. This client also interacts with the server - to allow groups of consumers to load balance consumption using consumer - groups. - - Requires Kafka Server >= 0.9.0.0 - - Configuration settings can be passed to constructor as kwargs, - otherwise defaults will be used: - - Keyword Arguments: - bootstrap_servers: 'host[:port]' string (or list of 'host[:port]' - strings) that the consumer should contact to bootstrap initial - cluster metadata. This does not have to be the full node list. - It just needs to have at least one broker that will respond to a - Metadata API Request. Default port is 9092. If no servers are - specified, will default to localhost:9092. - client_id (str): a name for this client. This string is passed in - each request to servers and can be used to identify specific - server-side log entries that correspond to this client. Also - submitted to GroupCoordinator for logging with respect to - consumer group administration. Default: 'kafka-python-{version}' - group_id (str): name of the consumer group to join for dynamic - partition assignment (if enabled), and to use for fetching and - committing offsets. Default: 'kafka-python-default-group' - key_deserializer (callable): Any callable that takes a - raw message key and returns a deserialized key. - value_deserializer (callable, optional): Any callable that takes a - raw message value and returns a deserialized value. - fetch_min_bytes (int): Minimum amount of data the server should - return for a fetch request, otherwise wait up to - fetch_max_wait_ms for more data to accumulate. Default: 1024. - fetch_max_wait_ms (int): The maximum amount of time in milliseconds - the server will block before answering the fetch request if - there isn't sufficient data to immediately satisfy the - requirement given by fetch_min_bytes. Default: 500. - max_partition_fetch_bytes (int): The maximum amount of data - per-partition the server will return. The maximum total memory - used for a request = #partitions * max_partition_fetch_bytes. - This size must be at least as large as the maximum message size - the server allows or else it is possible for the producer to - send messages larger than the consumer can fetch. If that - happens, the consumer can get stuck trying to fetch a large - message on a certain partition. Default: 1048576. - request_timeout_ms (int): Client request timeout in milliseconds. - Default: 40000. - retry_backoff_ms (int): Milliseconds to backoff when retrying on - errors. Default: 100. - reconnect_backoff_ms (int): The amount of time in milliseconds to - wait before attempting to reconnect to a given host. - Default: 50. - max_in_flight_requests_per_connection (int): Requests are pipelined - to kafka brokers up to this number of maximum requests per - broker connection. Default: 5. - auto_offset_reset (str): A policy for resetting offsets on - OffsetOutOfRange errors: 'earliest' will move to the oldest - available message, 'latest' will move to the most recent. Any - ofther value will raise the exception. Default: 'latest'. - enable_auto_commit (bool): If true the consumer's offset will be - periodically committed in the background. Default: True. - auto_commit_interval_ms (int): milliseconds between automatic - offset commits, if enable_auto_commit is True. Default: 5000. - default_offset_commit_callback (callable): called as - callback(offsets, response) response will be either an Exception - or a OffsetCommitResponse struct. This callback can be used to - trigger custom actions when a commit request completes. - check_crcs (bool): Automatically check the CRC32 of the records - consumed. This ensures no on-the-wire or on-disk corruption to - the messages occurred. This check adds some overhead, so it may - be disabled in cases seeking extreme performance. Default: True - metadata_max_age_ms (int): The period of time in milliseconds after - which we force a refresh of metadata even if we haven't seen any - partition leadership changes to proactively discover any new - brokers or partitions. Default: 300000 - partition_assignment_strategy (list): List of objects to use to - distribute partition ownership amongst consumer instances when - group management is used. Default: [RoundRobinPartitionAssignor] - heartbeat_interval_ms (int): The expected time in milliseconds - between heartbeats to the consumer coordinator when using - Kafka's group management feature. Heartbeats are used to ensure - that the consumer's session stays active and to facilitate - rebalancing when new consumers join or leave the group. The - value must be set lower than session_timeout_ms, but typically - should be set no higher than 1/3 of that value. It can be - adjusted even lower to control the expected time for normal - rebalances. Default: 3000 - session_timeout_ms (int): The timeout used to detect failures when - using Kafka's group managementment facilities. Default: 30000 - send_buffer_bytes (int): The size of the TCP send buffer - (SO_SNDBUF) to use when sending data. Default: 131072 - receive_buffer_bytes (int): The size of the TCP receive buffer - (SO_RCVBUF) to use when reading data. Default: 32768 - consumer_timeout_ms (int): number of millisecond to throw a timeout - exception to the consumer if no message is available for - consumption. Default: -1 (dont throw exception) - api_version (str): specify which kafka API version to use. - 0.9 enables full group coordination features; 0.8.2 enables - kafka-storage offset commits; 0.8.1 enables zookeeper-storage - offset commits; 0.8.0 is what is left. If set to 'auto', will - attempt to infer the broker version by probing various APIs. - Default: auto - - Configuration parameters are described in more detail at - https://kafka.apache.org/090/configuration.html#newconsumerconfigs - """ self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: @@ -204,20 +203,25 @@ class KafkaConsumer(six.Iterator): def assign(self, partitions): """Manually assign a list of TopicPartitions to this consumer. - This interface does not allow for incremental assignment and will - replace the previous assignment (if there was one). - - Manual topic assignment through this method does not use the consumer's - group management functionality. As such, there will be no rebalance - operation triggered when group membership or cluster and topic metadata - change. Note that it is not possible to use both manual partition - assignment with assign() and group assignment with subscribe(). - Arguments: partitions (list of TopicPartition): assignment for this instance. Raises: IllegalStateError: if consumer has already called subscribe() + + Warning: + It is not possible to use both manual partition assignment with + assign() and group assignment with subscribe(). + + Note: + This interface does not support incremental assignment and will + replace the previous assignment (if there was one). + + Note: + Manual topic assignment through this method does not use the + consumer's group management functionality. As such, there will be + no rebalance operation triggered when group membership or cluster + and topic metadata change. """ self._subscription.assign_from_user(partitions) self._client.set_topics([tp.topic for tp in partitions]) @@ -225,12 +229,12 @@ class KafkaConsumer(six.Iterator): def assignment(self): """Get the TopicPartitions currently assigned to this consumer. - If partitions were directly assigning using assign(), then this will - simply return the same partitions that were assigned. - If topics were subscribed to using subscribe(), then this will give the + If partitions were directly assigned using assign(), then this will + simply return the same partitions that were previously assigned. + If topics were subscribed using subscribe(), then this will give the set of topic partitions currently assigned to the consumer (which may - be none if the assignment hasn't happened yet, or the partitions are in - the process of getting reassigned). + be none if the assignment hasn't happened yet, or if the partitions are + in the process of being reassigned). Returns: set: {TopicPartition, ...} @@ -654,31 +658,25 @@ class KafkaConsumer(six.Iterator): # old KafkaConsumer methods are deprecated def configure(self, **configs): - """DEPRECATED -- initialize a new consumer""" raise NotImplementedError( 'deprecated -- initialize a new consumer') def set_topic_partitions(self, *topics): - """DEPRECATED -- use subscribe() or assign()""" raise NotImplementedError( 'deprecated -- use subscribe() or assign()') def fetch_messages(self): - """DEPRECATED -- use poll() or iterator interface""" raise NotImplementedError( 'deprecated -- use poll() or iterator interface') def get_partition_offsets(self, topic, partition, request_time_ms, max_num_offsets): - """DEPRECATED -- send OffsetRequest with KafkaClient""" raise NotImplementedError( 'deprecated -- send an OffsetRequest with KafkaClient') def offsets(self, group=None): - """DEPRECATED -- use committed(partition)""" raise NotImplementedError('deprecated -- use committed(partition)') def task_done(self, message): - """DEPRECATED -- commit manually if needed""" raise NotImplementedError( 'deprecated -- commit offsets manually if needed') diff --git a/kafka/producer/base.py b/kafka/producer/base.py index 4f5edbc..506da83 100644 --- a/kafka/producer/base.py +++ b/kafka/producer/base.py @@ -61,7 +61,8 @@ def _send_upstream(queue, client, codec, batch_time, batch_size, Arguments: queue (threading.Queue): the queue from which to get messages - client (KafkaClient): instance to use for communicating with brokers + client (kafka.SimpleClient): instance to use for communicating + with brokers codec (kafka.protocol.ALL_CODECS): compression codec to use batch_time (int): interval in seconds to send message batches batch_size (int): count of messages that will trigger an immediate send @@ -225,9 +226,9 @@ class Producer(object): Base class to be used by producers Arguments: - client (KafkaClient): instance to use for broker communications. - If async=True, the background thread will use client.copy(), - which is expected to return a thread-safe object. + client (kafka.SimpleClient): instance to use for broker + communications. If async=True, the background thread will use + client.copy(), which is expected to return a thread-safe object. codec (kafka.protocol.ALL_CODECS): compression codec to use. req_acks (int, optional): A value indicating the acknowledgements that the server must receive before responding to the request, @@ -345,20 +346,36 @@ class Producer(object): self.sync_fail_on_error = sync_fail_on_error def send_messages(self, topic, partition, *msg): - """ - Helper method to send produce requests - @param: topic, name of topic for produce request -- type str - @param: partition, partition number for produce request -- type int - @param: *msg, one or more message payloads -- type bytes - @returns: ResponseRequest returned by server - raises on error - - Note that msg type *must* be encoded to bytes by user. - Passing unicode message will not work, for example - you should encode before calling send_messages via - something like `unicode_message.encode('utf-8')` - - All messages produced via this method will set the message 'key' to Null + """Helper method to send produce requests. + + Note that msg type *must* be encoded to bytes by user. Passing unicode + message will not work, for example you should encode before calling + send_messages via something like `unicode_message.encode('utf-8')` + All messages will set the message 'key' to None. + + Arguments: + topic (str): name of topic for produce request + partition (int): partition number for produce request + *msg (bytes): one or more message payloads + + Returns: + ResponseRequest returned by server + + Raises: + FailedPayloadsError: low-level connection error, can be caused by + networking failures, or a malformed request. + ConnectionError: + KafkaUnavailableError: all known brokers are down when attempting + to refresh metadata. + LeaderNotAvailableError: topic or partition is initializing or + a broker failed and leadership election is in progress. + NotLeaderForPartitionError: metadata is out of sync; the broker + that the request was sent to is not the leader for the topic + or partition. + UnknownTopicOrPartitionError: the topic or partition has not + been created yet and auto-creation is not available. + AsyncProducerQueueFull: in async mode, if too many messages are + unsent and remain in the internal queue. """ return self._send_messages(topic, partition, *msg) -- cgit v1.2.1 From 19d403c743c99b0ecf8084bc35ee3718f624ca3f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 7 Jan 2016 17:08:32 -0800 Subject: Add metadata_max_age_ms and retry_backoff_ms options to async client --- kafka/client_async.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/kafka/client_async.py b/kafka/client_async.py index 747a85f..54d8153 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -40,6 +40,8 @@ class KafkaClient(object): 'max_in_flight_requests_per_connection': 5, 'receive_buffer_bytes': 32768, 'send_buffer_bytes': 131072, + 'retry_backoff_ms': 100, + 'metadata_max_age_ms': 300000, } def __init__(self, **configs): @@ -69,6 +71,12 @@ class KafkaClient(object): (SO_SNDBUF) to use when sending data. Default: 131072 receive_buffer_bytes (int): The size of the TCP receive buffer (SO_RCVBUF) to use when reading data. Default: 32768 + metadata_max_age_ms (int): The period of time in milliseconds after + which we force a refresh of metadata even if we haven't seen any + partition leadership changes to proactively discover any new + brokers or partitions. Default: 300000 + retry_backoff_ms (int): Milliseconds to backoff when retrying on + errors. Default: 100. """ self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: -- cgit v1.2.1 From 2a2e77aa1e5c31b3e815d573051bb2019daaa306 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 7 Jan 2016 17:10:04 -0800 Subject: Drop sphinxcontrib-napoleon in favor of sphinx.ext.napolean --- docs/conf.py | 2 +- tox.ini | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/conf.py b/docs/conf.py index dc68fd4..805c729 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -32,7 +32,7 @@ extensions = [ 'sphinx.ext.autodoc', 'sphinx.ext.intersphinx', 'sphinx.ext.viewcode', - 'sphinxcontrib.napoleon', + 'sphinx.ext.napoleon', ] # Add any paths that contain templates here, relative to this directory. diff --git a/tox.ini b/tox.ini index 0544cf8..33ec593 100644 --- a/tox.ini +++ b/tox.ini @@ -34,7 +34,6 @@ commands = py.test {posargs:--cov=kafka} [testenv:docs] deps = - sphinxcontrib-napoleon sphinx_rtd_theme sphinx -- cgit v1.2.1 From d4e85ecd1d8acac1a0f74d164b67faefd99987e4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 7 Jan 2016 17:14:49 -0800 Subject: Update docs for release w/ new async classes --- docs/apidoc/BrokerConnection.rst | 5 + docs/apidoc/KafkaClient.rst | 5 + docs/apidoc/KafkaConsumer.rst | 5 + docs/apidoc/KafkaProducer.rst | 4 + docs/apidoc/SimpleProducer.rst | 14 +++ docs/apidoc/modules.rst | 11 +- docs/compatibility.rst | 14 +++ docs/conf.py | 2 +- docs/index.rst | 106 ++++++++++------- docs/install.rst | 12 +- docs/license.rst | 10 ++ docs/support.rst | 11 ++ docs/tests.rst | 76 +++++++----- docs/usage.rst | 246 ++++++++++++++------------------------- 14 files changed, 283 insertions(+), 238 deletions(-) create mode 100644 docs/apidoc/BrokerConnection.rst create mode 100644 docs/apidoc/KafkaClient.rst create mode 100644 docs/apidoc/KafkaConsumer.rst create mode 100644 docs/apidoc/KafkaProducer.rst create mode 100644 docs/apidoc/SimpleProducer.rst create mode 100644 docs/compatibility.rst create mode 100644 docs/license.rst create mode 100644 docs/support.rst diff --git a/docs/apidoc/BrokerConnection.rst b/docs/apidoc/BrokerConnection.rst new file mode 100644 index 0000000..c56cf42 --- /dev/null +++ b/docs/apidoc/BrokerConnection.rst @@ -0,0 +1,5 @@ +BrokerConnection +================ + +.. autoclass:: kafka.BrokerConnection + :members: diff --git a/docs/apidoc/KafkaClient.rst b/docs/apidoc/KafkaClient.rst new file mode 100644 index 0000000..5c9d736 --- /dev/null +++ b/docs/apidoc/KafkaClient.rst @@ -0,0 +1,5 @@ +KafkaClient +=========== + +.. autoclass:: kafka.KafkaClient + :members: diff --git a/docs/apidoc/KafkaConsumer.rst b/docs/apidoc/KafkaConsumer.rst new file mode 100644 index 0000000..39062c6 --- /dev/null +++ b/docs/apidoc/KafkaConsumer.rst @@ -0,0 +1,5 @@ +KafkaConsumer +============= + +.. autoclass:: kafka.KafkaConsumer + :members: diff --git a/docs/apidoc/KafkaProducer.rst b/docs/apidoc/KafkaProducer.rst new file mode 100644 index 0000000..c33b2f9 --- /dev/null +++ b/docs/apidoc/KafkaProducer.rst @@ -0,0 +1,4 @@ +KafkaProducer +============= + + See :class:`kafka.producer.SimpleProducer` diff --git a/docs/apidoc/SimpleProducer.rst b/docs/apidoc/SimpleProducer.rst new file mode 100644 index 0000000..a509858 --- /dev/null +++ b/docs/apidoc/SimpleProducer.rst @@ -0,0 +1,14 @@ +SimpleProducer +============== + +.. autoclass:: kafka.producer.SimpleProducer + :members: + :show-inheritance: + +.. autoclass:: kafka.producer.KeyedProducer + :members: + :show-inheritance: + +.. automodule:: kafka.producer.base + :members: + :show-inheritance: diff --git a/docs/apidoc/modules.rst b/docs/apidoc/modules.rst index db3e580..f6eb798 100644 --- a/docs/apidoc/modules.rst +++ b/docs/apidoc/modules.rst @@ -1,7 +1,10 @@ -kafka -===== +kafka-python API +**************** .. toctree:: - :maxdepth: 4 - kafka + KafkaConsumer + KafkaProducer + KafkaClient + BrokerConnection + SimpleProducer diff --git a/docs/compatibility.rst b/docs/compatibility.rst new file mode 100644 index 0000000..ccc4b96 --- /dev/null +++ b/docs/compatibility.rst @@ -0,0 +1,14 @@ +Compatibility +------------- + +.. image:: https://img.shields.io/badge/kafka-0.9%2C%200.8.2%2C%200.8.1%2C%200.8-brightgreen.svg + :target: https://kafka-python.readthedocs.org/compatibility.html +.. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg + :target: https://pypi.python.org/pypi/kafka-python + +kafka-python is compatible with (and tested against) broker versions 0.9.0.0 +through 0.8.0 . kafka-python is not compatible with the 0.8.2-beta release. + +kafka-python is tested on python 2.6, 2.7, 3.3, 3.4, 3.5, and pypy. + +Builds and tests via Travis-CI. See https://travis-ci.org/dpkp/kafka-python diff --git a/docs/conf.py b/docs/conf.py index 805c729..66f9663 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -49,7 +49,7 @@ master_doc = 'index' # General information about the project. project = u'kafka-python' -copyright = u'2015 - David Arthur, Dana Powers, and Contributors' +copyright = u'2016 -- Dana Powes, David Arthur, and Contributors' # The version info for the project you're documenting, acts as replacement for # |version| and |release|, also used in various other places throughout the diff --git a/docs/index.rst b/docs/index.rst index fa77a8e..f65d4db 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,66 +1,86 @@ kafka-python -============ +############ -This module provides low-level protocol support for Apache Kafka as well as -high-level consumer and producer classes. Request batching is supported by the -protocol as well as broker-aware request routing. Gzip and Snappy compression -is also supported for message sets. +.. image:: https://img.shields.io/badge/kafka-0.9%2C%200.8.2%2C%200.8.1%2C%200.8-brightgreen.svg + :target: https://kafka-python.readthedocs.org/compatibility.html +.. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg + :target: https://pypi.python.org/pypi/kafka-python +.. image:: https://coveralls.io/repos/dpkp/kafka-python/badge.svg?branch=master&service=github + :target: https://coveralls.io/github/dpkp/kafka-python?branch=master +.. image:: https://travis-ci.org/dpkp/kafka-python.svg?branch=master + :target: https://travis-ci.org/dpkp/kafka-python +.. image:: https://img.shields.io/badge/license-Apache%202-blue.svg + :target: https://github.com/dpkp/kafka-python/blob/master/LICENSE -Coordinated Consumer Group support is under development - see Issue #38. +>>> pip install kafka-python -On Freenode IRC at #kafka-python, as well as #apache-kafka +kafka-python is a client for the Apache Kafka distributed stream processing +system. It is designed to function much like the official java client, with a +sprinkling of pythonic interfaces (e.g., iterators). -For general discussion of kafka-client design and implementation (not python specific), -see https://groups.google.com/forum/m/#!forum/kafka-clients -For information about Apache Kafka generally, see https://kafka.apache.org/ +KafkaConsumer +************* -Status ------- +>>> from kafka import KafkaConsumer +>>> consumer = KafkaConsumer('my_favorite_topic') +>>> for msg in consumer: +... print (msg) -The current stable version of this package is `0.9.5 `_ and is compatible with: +:class:`~kafka.consumer.KafkaConsumer` is a full-featured, +high-level message consumer class that is similar in design and function to the +new 0.9 java consumer. Most configuration parameters defined by the official +java client are supported as optional kwargs, with generally similar behavior. +Gzip and Snappy compressed messages are supported transparently. -Kafka broker versions +In addition to the standard +:meth:`~kafka.consumer.KafkaConsumer.poll` interface (which returns +micro-batches of messages, grouped by topic-partition), kafka-python supports +single-message iteration, yielding :class:`~kafka.consumer.ConsumerRecord` +namedtuples, which include the topic, partition, offset, key, and value of each +message. -* 0.9.0.0 -* 0.8.2.2 -* 0.8.2.1 -* 0.8.1.1 -* 0.8.1 -* 0.8.0 +By default, :class:`~kafka.consumer.KafkaConsumer` will attempt to auto-commit +message offsets every 5 seconds. When used with 0.9 kafka brokers, +:class:`~kafka.consumer.KafkaConsumer` will dynamically assign partitions using +the kafka GroupCoordinator APIs and a +:class:`~kafka.coordinator.assignors.roundrobin.RoundRobinPartitionAssignor` +partitioning strategy, enabling relatively straightforward parallel consumption +patterns. See :doc:`usage` for examples. -Python versions -* 3.5 (tested on 3.5.0) -* 3.4 (tested on 3.4.2) -* 3.3 (tested on 3.3.5) -* 2.7 (tested on 2.7.9) -* 2.6 (tested on 2.6.9) -* pypy (tested on pypy 2.5.0 / python 2.7.8) +KafkaProducer +************* -License -------- +TBD -Apache License, v2.0. See `LICENSE `_. -Copyright 2015, David Arthur, Dana Powers, and Contributors -(See `AUTHORS `_). +Protocol +******** +A secondary goal of kafka-python is to provide an easy-to-use protocol layer +for interacting with kafka brokers via the python repl. This is useful for +testing, probing, and general experimentation. The protocol support is +leveraged to enable a :meth:`~kafka.KafkaClient.check_version()` +method that probes a kafka broker and +attempts to identify which version it is running (0.8.0 to 0.9). + + +Low-level +********* + +Legacy support is maintained for low-level consumer and producer classes, +SimpleConsumer and SimpleProducer. -Contents --------- .. toctree:: + :hidden: :maxdepth: 2 - usage + Usage Overview + API install tests - API reference - -Indices and tables -================== - -* :ref:`genindex` -* :ref:`modindex` -* :ref:`search` + compatibility + support + license diff --git a/docs/install.rst b/docs/install.rst index 2bc6911..bf49c3f 100644 --- a/docs/install.rst +++ b/docs/install.rst @@ -1,10 +1,10 @@ Install -======= +####### Install with your favorite package manager Latest Release --------------- +************** Pip: .. code:: bash @@ -15,7 +15,7 @@ Releases are also listed at https://github.com/dpkp/kafka-python/releases Bleeding-Edge -------------- +************* .. code:: bash @@ -39,10 +39,10 @@ Using `setup.py` directly: Optional Snappy install ------------------------ +*********************** Install Development Libraries -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +============================= Download and build Snappy from http://code.google.com/p/snappy/downloads/list @@ -70,7 +70,7 @@ From Source: sudo make install Install Python Module -^^^^^^^^^^^^^^^^^^^^^ +===================== Install the `python-snappy` module diff --git a/docs/license.rst b/docs/license.rst new file mode 100644 index 0000000..13df48c --- /dev/null +++ b/docs/license.rst @@ -0,0 +1,10 @@ +License +------- + +.. image:: https://img.shields.io/badge/license-Apache%202-blue.svg + :target: https://github.com/dpkp/kafka-python/blob/master/LICENSE + +Apache License, v2.0. See `LICENSE `_. + +Copyright 2016, David Arthur, Dana Powers, and Contributors +(See `AUTHORS `_). diff --git a/docs/support.rst b/docs/support.rst new file mode 100644 index 0000000..63d4a86 --- /dev/null +++ b/docs/support.rst @@ -0,0 +1,11 @@ +Support +------- + +For support, see github issues at https://github.com/dpkp/kafka-python + +Limited IRC chat at #kafka-python on freenode (general chat is #apache-kafka). + +For information about Apache Kafka generally, see https://kafka.apache.org/ + +For general discussion of kafka-client design and implementation (not python +specific), see https://groups.google.com/forum/m/#!forum/kafka-clients diff --git a/docs/tests.rst b/docs/tests.rst index df9a3ef..e5dd269 100644 --- a/docs/tests.rst +++ b/docs/tests.rst @@ -1,59 +1,83 @@ Tests ===== -Run the unit tests ------------------- +.. image:: https://coveralls.io/repos/dpkp/kafka-python/badge.svg?branch=master&service=github + :target: https://coveralls.io/github/dpkp/kafka-python?branch=master +.. image:: https://travis-ci.org/dpkp/kafka-python.svg?branch=master + :target: https://travis-ci.org/dpkp/kafka-python -.. code:: bash +Test environments are managed via tox. The test suite is run via pytest. +Individual tests are written using unittest, pytest, and in some cases, +doctest. + +Linting is run via pylint, but is generally skipped on python2.6 and pypy +due to pylint compatibility / performance issues. + +For test coverage details, see https://coveralls.io/github/dpkp/kafka-python - tox +The test suite includes unit tests that mock network interfaces, as well as +integration tests that setup and teardown kafka broker (and zookeeper) +fixtures for client / consumer / producer testing. + + +Unit tests +------------------ +To run the tests locally, install tox -- `pip install tox` +See http://tox.readthedocs.org/en/latest/install.html -Run a subset of unit tests --------------------------- +Then simply run tox, optionally setting the python environment. +If unset, tox will loop through all environments. .. code:: bash + tox -e py27 + tox -e py35 + # run protocol tests only tox -- -v test.test_protocol - # test with pypy only - tox -e pypy + # re-run the last failing test, dropping into pdb + tox -e py27 -- --lf --pdb + + # see available (pytest) options + tox -e py27 -- --help - # Run only 1 test, and use python 2.7 - tox -e py27 -- -v --with-id --collect-only - # pick a test number from the list like #102 - tox -e py27 -- -v --with-id 102 +Integration tests +----------------- +.. code:: bash -Run the integration tests -------------------------- + KAFKA_VERSION=0.9.0.0 tox -e py27 + KAFKA_VERSION=0.8.2.2 tox -e py35 -The integration tests will actually start up real local Zookeeper -instance and Kafka brokers, and send messages in using the client. -First, get the kafka binaries for integration testing: +Integration tests start Kafka and Zookeeper fixtures. This requires downloading +kafka server binaries: .. code:: bash ./build_integration.sh -By default, the build_integration.sh script will download binary -distributions for all supported kafka versions. -To test against the latest source build, set KAFKA_VERSION=trunk -and optionally set SCALA_VERSION (defaults to 2.8.0, but 2.10.1 is recommended) +By default, this will install 0.8.1.1, 0.8.2.2, and 0.9.0.0 brokers into the +servers/ directory. To install a specific version, set `KAFKA_VERSION=1.2.3`: .. code:: bash - SCALA_VERSION=2.10.1 KAFKA_VERSION=trunk ./build_integration.sh + KAFKA_VERSION=0.8.0 ./build_integration.sh Then run the tests against supported Kafka versions, simply set the `KAFKA_VERSION` env variable to the server build you want to use for testing: .. code:: bash - KAFKA_VERSION=0.8.0 tox - KAFKA_VERSION=0.8.1 tox - KAFKA_VERSION=0.8.1.1 tox - KAFKA_VERSION=trunk tox + KAFKA_VERSION=0.9.0.0 tox -e py27 + +To test against the kafka source tree, set KAFKA_VERSION=trunk +[optionally set SCALA_VERSION (defaults to 2.10)] + +.. code:: bash + + SCALA_VERSION=2.11 KAFKA_VERSION=trunk ./build_integration.sh + KAFKA_VERSION=trunk tox -e py35 diff --git a/docs/usage.rst b/docs/usage.rst index 6417cd8..e74e5af 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -1,68 +1,126 @@ Usage -===== +***** -SimpleProducer --------------- + +KafkaConsumer +============= .. code:: python - from kafka import SimpleProducer, KafkaClient + from kafka import KafkaConsumer - # To send messages synchronously - kafka = KafkaClient('localhost:9092') - producer = SimpleProducer(kafka) + # To consume latest messages and auto-commit offsets + consumer = KafkaConsumer('my-topic', + group_id='my-group', + bootstrap_servers=['localhost:9092']) + for message in consumer: + # message value and key are raw bytes -- decode if necessary! + # e.g., for unicode: `message.value.decode('utf-8')` + print ("%s:%d:%d: key=%s value=%s" % (message.topic, message.partition, + message.offset, message.key, + message.value)) - # Note that the application is responsible for encoding messages to type bytes - producer.send_messages(b'my-topic', b'some message') - producer.send_messages(b'my-topic', b'this method', b'is variadic') + # consume earliest available messages, dont commit offsets + KafkaConsumer(auto_offset_reset='earliest', enable_auto_commit=False) - # Send unicode message - producer.send_messages(b'my-topic', u'你怎么样?'.encode('utf-8')) + # consume json messages + KafkaConsumer(value_deserializer=lambda m: json.loads(m.decode('ascii'))) + + # consume msgpack + KafkaConsumer(value_deserializer=msgpack.unpackb) + + # StopIteration if no message after 1sec + KafkaConsumer(consumer_timeout_ms=1000) + + # Subscribe to a regex topic pattern + consumer = KafkaConsumer() + consumer.subscribe(pattern='^awesome.*') + + # Use multiple consumers in parallel w/ 0.9 kafka brokers + # typically you would run each on a different server / process / CPU + consumer1 = KafkaConsumer('my-topic', + group_id='my-group', + bootstrap_servers='my.server.com') + consumer2 = KafkaConsumer('my-topic', + group_id='my-group', + bootstrap_servers='my.server.com') + + +There are many configuration options for the consumer class. See +:class:`~kafka.KafkaConsumer` API documentation for more details. + + +SimpleProducer +============== Asynchronous Mode ----------------- .. code:: python + from kafka import SimpleProducer, SimpleClient + # To send messages asynchronously - producer = SimpleProducer(kafka, async=True) - producer.send_messages(b'my-topic', b'async message') + client = SimpleClient('localhost:9092') + producer = SimpleProducer(client, async=True) + producer.send_messages('my-topic', b'async message') + + # To send messages in batch. You can use any of the available + # producers for doing this. The following producer will collect + # messages in batch and send them to Kafka after 20 messages are + # collected or every 60 seconds + # Notes: + # * If the producer dies before the messages are sent, there will be losses + # * Call producer.stop() to send the messages and cleanup + producer = SimpleProducer(client, + async=True, + batch_send_every_n=20, + batch_send_every_t=60) + +Synchronous Mode +---------------- + +.. code:: python + + from kafka import SimpleProducer, SimpleClient + + # To send messages synchronously + client = SimpleClient('localhost:9092') + producer = SimpleProducer(client, async=False) + + # Note that the application is responsible for encoding messages to type bytes + producer.send_messages('my-topic', b'some message') + producer.send_messages('my-topic', b'this method', b'is variadic') + + # Send unicode message + producer.send_messages('my-topic', u'你怎么样?'.encode('utf-8')) # To wait for acknowledgements # ACK_AFTER_LOCAL_WRITE : server will wait till the data is written to # a local log before sending response # ACK_AFTER_CLUSTER_COMMIT : server will block until the message is committed # by all in sync replicas before sending a response - producer = SimpleProducer(kafka, async=False, + producer = SimpleProducer(client, + async=False, req_acks=SimpleProducer.ACK_AFTER_LOCAL_WRITE, ack_timeout=2000, sync_fail_on_error=False) - responses = producer.send_messages(b'my-topic', b'another message') + responses = producer.send_messages('my-topic', b'another message') for r in responses: logging.info(r.offset) - # To send messages in batch. You can use any of the available - # producers for doing this. The following producer will collect - # messages in batch and send them to Kafka after 20 messages are - # collected or every 60 seconds - # Notes: - # * If the producer dies before the messages are sent, there will be losses - # * Call producer.stop() to send the messages and cleanup - producer = SimpleProducer(kafka, async=True, - batch_send_every_n=20, - batch_send_every_t=60) -Keyed messages --------------- +KeyedProducer +============= .. code:: python from kafka import ( - KafkaClient, KeyedProducer, + SimpleClient, KeyedProducer, Murmur2Partitioner, RoundRobinPartitioner) - kafka = KafkaClient('localhost:9092') + kafka = SimpleClient('localhost:9092') # HashedPartitioner is default (currently uses python hash()) producer = KeyedProducer(kafka) @@ -74,131 +132,3 @@ Keyed messages # Or just produce round-robin (or just use SimpleProducer) producer = KeyedProducer(kafka, partitioner=RoundRobinPartitioner) - - - -KafkaConsumer -------------- - -.. code:: python - - from kafka import KafkaConsumer - - # To consume messages - consumer = KafkaConsumer('my-topic', - group_id='my_group', - bootstrap_servers=['localhost:9092']) - for message in consumer: - # message value is raw byte string -- decode if necessary! - # e.g., for unicode: `message.value.decode('utf-8')` - print("%s:%d:%d: key=%s value=%s" % (message.topic, message.partition, - message.offset, message.key, - message.value)) - - -messages (m) are namedtuples with attributes: - - * `m.topic`: topic name (str) - * `m.partition`: partition number (int) - * `m.offset`: message offset on topic-partition log (int) - * `m.key`: key (bytes - can be None) - * `m.value`: message (output of deserializer_class - default is raw bytes) - - -.. code:: python - - from kafka import KafkaConsumer - - # more advanced consumer -- multiple topics w/ auto commit offset - # management - consumer = KafkaConsumer('topic1', 'topic2', - bootstrap_servers=['localhost:9092'], - group_id='my_consumer_group', - auto_commit_enable=True, - auto_commit_interval_ms=30 * 1000, - auto_offset_reset='smallest') - - # Infinite iteration - for m in consumer: - do_some_work(m) - - # Mark this message as fully consumed - # so it can be included in the next commit - # - # **messages that are not marked w/ task_done currently do not commit! - consumer.task_done(m) - - # If auto_commit_enable is False, remember to commit() periodically - consumer.commit() - - # Batch process interface - while True: - for m in kafka.fetch_messages(): - process_message(m) - consumer.task_done(m) - - - Configuration settings can be passed to constructor, - otherwise defaults will be used: - -.. code:: python - - client_id='kafka.consumer.kafka', - group_id=None, - fetch_message_max_bytes=1024*1024, - fetch_min_bytes=1, - fetch_wait_max_ms=100, - refresh_leader_backoff_ms=200, - bootstrap_servers=[], - socket_timeout_ms=30*1000, - auto_offset_reset='largest', - deserializer_class=lambda msg: msg, - auto_commit_enable=False, - auto_commit_interval_ms=60 * 1000, - consumer_timeout_ms=-1 - - Configuration parameters are described in more detail at - http://kafka.apache.org/documentation.html#highlevelconsumerapi - -Multiprocess consumer ---------------------- - -.. code:: python - - from kafka import KafkaClient, MultiProcessConsumer - - kafka = KafkaClient('localhost:9092') - - # This will split the number of partitions among two processes - consumer = MultiProcessConsumer(kafka, b'my-group', b'my-topic', num_procs=2) - - # This will spawn processes such that each handles 2 partitions max - consumer = MultiProcessConsumer(kafka, b'my-group', b'my-topic', - partitions_per_proc=2) - - for message in consumer: - print(message) - - for message in consumer.get_messages(count=5, block=True, timeout=4): - print(message) - -Low level ---------- - -.. code:: python - - from kafka import KafkaClient, create_message - from kafka.protocol import KafkaProtocol - from kafka.common import ProduceRequest - - kafka = KafkaClient('localhost:9092') - - req = ProduceRequest(topic=b'my-topic', partition=1, - messages=[create_message(b'some message')]) - resps = kafka.send_produce_request(payloads=[req], fail_on_error=True) - kafka.close() - - resps[0].topic # b'my-topic' - resps[0].partition # 1 - resps[0].error # 0 (hopefully) - resps[0].offset # offset of the first message sent in this request -- cgit v1.2.1 From 9a8af1499ca425366d934487469d9977fae7fe5f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 7 Jan 2016 17:57:24 -0800 Subject: Fix KafkaClient->SimpleClient references --- kafka/consumer/base.py | 2 +- kafka/consumer/kafka.py | 4 +-- kafka/consumer/multiprocess.py | 2 +- kafka/consumer/simple.py | 2 +- test/test_client.py | 64 +++++++++++++++++++-------------------- test/test_consumer_group.py | 4 +-- test/test_failover_integration.py | 6 ++-- test/test_producer.py | 10 +++--- test/testutil.py | 4 +-- 9 files changed, 49 insertions(+), 49 deletions(-) diff --git a/kafka/consumer/base.py b/kafka/consumer/base.py index a90038f..2059d92 100644 --- a/kafka/consumer/base.py +++ b/kafka/consumer/base.py @@ -94,7 +94,7 @@ class Consumer(object): def fetch_last_known_offsets(self, partitions=None): if self.group is None: - raise ValueError('KafkaClient.group must not be None') + raise ValueError('SimpleClient.group must not be None') if partitions is None: partitions = self.client.get_partition_ids_for_topic(self.topic) diff --git a/kafka/consumer/kafka.py b/kafka/consumer/kafka.py index 3f14444..29ddd0e 100644 --- a/kafka/consumer/kafka.py +++ b/kafka/consumer/kafka.py @@ -9,7 +9,7 @@ import time import six -from kafka.client import KafkaClient +from kafka import SimpleClient from kafka.common import ( OffsetFetchRequestPayload, OffsetCommitRequestPayload, OffsetRequestPayload, FetchRequestPayload, @@ -136,7 +136,7 @@ class KafkaConsumer(object): 'bootstrap_servers required to configure KafkaConsumer' ) - self._client = KafkaClient( + self._client = SimpleClient( self._config['bootstrap_servers'], client_id=self._config['client_id'], timeout=(self._config['socket_timeout_ms'] / 1000.0) diff --git a/kafka/consumer/multiprocess.py b/kafka/consumer/multiprocess.py index a348d1a..9358b09 100644 --- a/kafka/consumer/multiprocess.py +++ b/kafka/consumer/multiprocess.py @@ -102,7 +102,7 @@ class MultiProcessConsumer(Consumer): parallel using multiple processes Arguments: - client: a connected KafkaClient + client: a connected SimpleClient group: a name for this consumer, used for offset storage and must be unique If you are connecting to a server that does not support offset commit/fetch (any prior to 0.8.1.1), then you *must* set this to None diff --git a/kafka/consumer/simple.py b/kafka/consumer/simple.py index abeac7b..29eb480 100644 --- a/kafka/consumer/simple.py +++ b/kafka/consumer/simple.py @@ -70,7 +70,7 @@ class SimpleConsumer(Consumer): for a topic Arguments: - client: a connected KafkaClient + client: a connected SimpleClient group: a name for this consumer, used for offset storage and must be unique If you are connecting to a server that does not support offset commit/fetch (any prior to 0.8.1.1), then you *must* set this to None diff --git a/test/test_client.py b/test/test_client.py index 8c62eb9..5a35c83 100644 --- a/test/test_client.py +++ b/test/test_client.py @@ -5,7 +5,7 @@ from mock import ANY, MagicMock, patch import six from . import unittest -from kafka import KafkaClient +from kafka import SimpleClient from kafka.common import ( ProduceRequestPayload, BrokerMetadata, @@ -35,33 +35,33 @@ def mock_conn(conn, success=True): conn.return_value = mocked -class TestKafkaClient(unittest.TestCase): +class TestSimpleClient(unittest.TestCase): def test_init_with_list(self): - with patch.object(KafkaClient, 'load_metadata_for_topics'): - client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092', 'kafka03:9092']) + with patch.object(SimpleClient, 'load_metadata_for_topics'): + client = SimpleClient(hosts=['kafka01:9092', 'kafka02:9092', 'kafka03:9092']) self.assertEqual( sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]), sorted(client.hosts)) def test_init_with_csv(self): - with patch.object(KafkaClient, 'load_metadata_for_topics'): - client = KafkaClient(hosts='kafka01:9092,kafka02:9092,kafka03:9092') + with patch.object(SimpleClient, 'load_metadata_for_topics'): + client = SimpleClient(hosts='kafka01:9092,kafka02:9092,kafka03:9092') self.assertEqual( sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]), sorted(client.hosts)) def test_init_with_unicode_csv(self): - with patch.object(KafkaClient, 'load_metadata_for_topics'): - client = KafkaClient(hosts=u'kafka01:9092,kafka02:9092,kafka03:9092') + with patch.object(SimpleClient, 'load_metadata_for_topics'): + client = SimpleClient(hosts=u'kafka01:9092,kafka02:9092,kafka03:9092') self.assertEqual( sorted([('kafka01', 9092), ('kafka02', 9092), ('kafka03', 9092)]), sorted(client.hosts)) - @patch.object(KafkaClient, '_get_conn') - @patch.object(KafkaClient, 'load_metadata_for_topics') + @patch.object(SimpleClient, '_get_conn') + @patch.object(SimpleClient, 'load_metadata_for_topics') def test_send_broker_unaware_request_fail(self, load_metadata, conn): mocked_conns = { ('kafka01', 9092): MagicMock(), @@ -74,7 +74,7 @@ class TestKafkaClient(unittest.TestCase): return mocked_conns[(host, port)] conn.side_effect = mock_get_conn - client = KafkaClient(hosts=['kafka01:9092', 'kafka02:9092']) + client = SimpleClient(hosts=['kafka01:9092', 'kafka02:9092']) req = KafkaProtocol.encode_metadata_request() with self.assertRaises(KafkaUnavailableError): @@ -102,10 +102,10 @@ class TestKafkaClient(unittest.TestCase): return mocked_conns[(host, port)] # patch to avoid making requests before we want it - with patch.object(KafkaClient, 'load_metadata_for_topics'): - with patch.object(KafkaClient, '_get_conn', side_effect=mock_get_conn): + with patch.object(SimpleClient, 'load_metadata_for_topics'): + with patch.object(SimpleClient, '_get_conn', side_effect=mock_get_conn): - client = KafkaClient(hosts='kafka01:9092,kafka02:9092') + client = SimpleClient(hosts='kafka01:9092,kafka02:9092') resp = client._send_broker_unaware_request(payloads=['fake request'], encoder_fn=MagicMock(), decoder_fn=lambda x: x) @@ -113,7 +113,7 @@ class TestKafkaClient(unittest.TestCase): self.assertEqual('valid response', resp) mocked_conns[('kafka02', 9092)].recv.assert_called_once_with() - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_load_metadata(self, protocol, conn): @@ -143,7 +143,7 @@ class TestKafkaClient(unittest.TestCase): protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) # client loads metadata at init - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) self.assertDictEqual({ TopicPartition('topic_1', 0): brokers[1], TopicPartition('topic_noleader', 0): None, @@ -163,7 +163,7 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.load_metadata_for_topics('topic_no_leader') - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_has_metadata_for_topic(self, protocol, conn): @@ -184,7 +184,7 @@ class TestKafkaClient(unittest.TestCase): ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) # Topics with no partitions return False self.assertFalse(client.has_metadata_for_topic('topic_still_creating')) @@ -193,7 +193,7 @@ class TestKafkaClient(unittest.TestCase): # Topic with partition metadata, but no leaders return True self.assertTrue(client.has_metadata_for_topic('topic_noleaders')) - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol.decode_metadata_response') def test_ensure_topic_exists(self, decode_metadata_response, conn): @@ -214,7 +214,7 @@ class TestKafkaClient(unittest.TestCase): ] decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) with self.assertRaises(UnknownTopicOrPartitionError): client.ensure_topic_exists('topic_doesnt_exist', timeout=1) @@ -225,7 +225,7 @@ class TestKafkaClient(unittest.TestCase): # This should not raise client.ensure_topic_exists('topic_noleaders', timeout=1) - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_get_leader_for_partitions_reloads_metadata(self, protocol, conn): "Get leader for partitions reload metadata if it is not available" @@ -242,7 +242,7 @@ class TestKafkaClient(unittest.TestCase): ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) # topic metadata is loaded but empty self.assertDictEqual({}, client.topics_to_brokers) @@ -263,7 +263,7 @@ class TestKafkaClient(unittest.TestCase): TopicPartition('topic_one_partition', 0): brokers[0]}, client.topics_to_brokers) - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_get_leader_for_unassigned_partitions(self, protocol, conn): @@ -280,7 +280,7 @@ class TestKafkaClient(unittest.TestCase): ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) self.assertDictEqual({}, client.topics_to_brokers) @@ -290,7 +290,7 @@ class TestKafkaClient(unittest.TestCase): with self.assertRaises(UnknownTopicOrPartitionError): client._get_leader_for_partition('topic_unknown', 0) - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_get_leader_exceptions_when_noleader(self, protocol, conn): @@ -309,7 +309,7 @@ class TestKafkaClient(unittest.TestCase): ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) self.assertDictEqual( { TopicPartition('topic_noleader', 0): None, @@ -337,7 +337,7 @@ class TestKafkaClient(unittest.TestCase): self.assertEqual(brokers[0], client._get_leader_for_partition('topic_noleader', 0)) self.assertEqual(brokers[1], client._get_leader_for_partition('topic_noleader', 1)) - @patch.object(KafkaClient, '_get_conn') + @patch.object(SimpleClient, '_get_conn') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_noleader(self, protocol, conn): mock_conn(conn) @@ -355,7 +355,7 @@ class TestKafkaClient(unittest.TestCase): ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) requests = [ProduceRequestPayload( "topic_noleader", 0, @@ -364,7 +364,7 @@ class TestKafkaClient(unittest.TestCase): with self.assertRaises(LeaderNotAvailableError): client.send_produce_request(requests) - @patch('kafka.client.KafkaClient._get_conn') + @patch('kafka.SimpleClient._get_conn') @patch('kafka.client.KafkaProtocol') def test_send_produce_request_raises_when_topic_unknown(self, protocol, conn): @@ -380,7 +380,7 @@ class TestKafkaClient(unittest.TestCase): ] protocol.decode_metadata_response.return_value = MetadataResponse(brokers, topics) - client = KafkaClient(hosts=['broker_1:4567']) + client = SimpleClient(hosts=['broker_1:4567']) requests = [ProduceRequestPayload( "topic_doesnt_exist", 0, @@ -403,9 +403,9 @@ class TestKafkaClient(unittest.TestCase): self.assertGreaterEqual(t.interval, 1.0) def test_correlation_rollover(self): - with patch.object(KafkaClient, 'load_metadata_for_topics'): + with patch.object(SimpleClient, 'load_metadata_for_topics'): big_num = 2**31 - 3 - client = KafkaClient(hosts=[], correlation_id=big_num) + client = SimpleClient(hosts=[], correlation_id=big_num) self.assertEqual(big_num + 1, client._next_id()) self.assertEqual(big_num + 2, client._next_id()) self.assertEqual(0, client._next_id()) diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index 4fd4cdf..6160372 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -7,7 +7,7 @@ import time import pytest import six -from kafka import KafkaClient, SimpleProducer +from kafka import SimpleClient, SimpleProducer from kafka.common import TopicPartition from kafka.conn import BrokerConnection, ConnectionStates from kafka.consumer.group import KafkaConsumer @@ -47,7 +47,7 @@ def kafka_broker(version, zookeeper, request): @pytest.fixture def simple_client(kafka_broker): connect_str = 'localhost:' + str(kafka_broker.port) - return KafkaClient(connect_str) + return SimpleClient(connect_str) @pytest.fixture diff --git a/test/test_failover_integration.py b/test/test_failover_integration.py index 5ffaa04..b54ace0 100644 --- a/test/test_failover_integration.py +++ b/test/test_failover_integration.py @@ -2,7 +2,7 @@ import logging import os import time -from kafka import KafkaClient, SimpleConsumer, KeyedProducer +from kafka import SimpleClient, SimpleConsumer, KeyedProducer from kafka.common import ( TopicPartition, FailedPayloadsError, ConnectionError, RequestTimedOutError ) @@ -34,7 +34,7 @@ class TestFailover(KafkaIntegrationTestCase): self.brokers = [KafkaFixture.instance(i, *kk_args) for i in range(replicas)] hosts = ['%s:%d' % (b.host, b.port) for b in self.brokers] - self.client = KafkaClient(hosts, timeout=2) + self.client = SimpleClient(hosts, timeout=2) super(TestFailover, self).setUp() def tearDown(self): @@ -214,7 +214,7 @@ class TestFailover(KafkaIntegrationTestCase): hosts = ','.join(['%s:%d' % (broker.host, broker.port) for broker in self.brokers]) - client = KafkaClient(hosts) + client = SimpleClient(hosts) consumer = SimpleConsumer(client, None, topic, partitions=partitions, auto_commit=False, diff --git a/test/test_producer.py b/test/test_producer.py index 227d4ad..aa4f0be 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -8,7 +8,7 @@ import time from mock import MagicMock, patch from . import unittest -from kafka import KafkaClient, SimpleProducer, KeyedProducer +from kafka import SimpleClient, SimpleProducer, KeyedProducer from kafka.common import ( AsyncProducerQueueFull, FailedPayloadsError, NotLeaderForPartitionError, ProduceResponsePayload, RetryOptions, TopicPartition @@ -89,11 +89,11 @@ class TestKafkaProducer(unittest.TestCase): def test_producer_sync_fail_on_error(self): error = FailedPayloadsError('failure') - with patch.object(KafkaClient, 'load_metadata_for_topics'): - with patch.object(KafkaClient, 'get_partition_ids_for_topic', return_value=[0, 1]): - with patch.object(KafkaClient, '_send_broker_aware_request', return_value = [error]): + with patch.object(SimpleClient, 'load_metadata_for_topics'): + with patch.object(SimpleClient, 'get_partition_ids_for_topic', return_value=[0, 1]): + with patch.object(SimpleClient, '_send_broker_aware_request', return_value = [error]): - client = KafkaClient(MagicMock()) + client = SimpleClient(MagicMock()) producer = SimpleProducer(client, async=False, sync_fail_on_error=False) # This should not raise diff --git a/test/testutil.py b/test/testutil.py index 98fe805..2f3770e 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -10,7 +10,7 @@ import uuid from six.moves import xrange from . import unittest -from kafka import KafkaClient +from kafka import SimpleClient from kafka.common import OffsetRequestPayload __all__ = [ @@ -62,7 +62,7 @@ class KafkaIntegrationTestCase(unittest.TestCase): self.topic = topic if self.create_client: - self.client = KafkaClient('%s:%d' % (self.server.host, self.server.port)) + self.client = SimpleClient('%s:%d' % (self.server.host, self.server.port)) self.client.ensure_topic_exists(self.topic) -- cgit v1.2.1